mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-26 08:11:39 +00:00
feat(swarm): allow configuration to idle connection timeout
Previously, a connection would be shut down immediately as soon as its `ConnectionHandler` reports `KeepAlive::No`. As we have gained experience with libp2p, it turned out that this isn't ideal. For one, tests often need to keep connections alive longer than the configured protocols require. Plus, some usecases require connections to be kept alive in general. Both of these needs are currently served by the `keep_alive::Behaviour`. That one does essentially nothing other than statically returning `KeepAlive::Yes` from its `ConnectionHandler`. It makes much more sense to deprecate `keep_alive::Behaviour` and instead allow users to globally configure an `idle_conncetion_timeout` on the `Swarm`. This timeout comes into effect once a `ConnectionHandler` reports `KeepAlive::No`. To start with, this timeout is 0. Together with https://github.com/libp2p/rust-libp2p/issues/3844, this will allow us to move towards a much more aggressive closing of idle connections, together with a more ergonomic way of opting out of this behaviour. Fixes #4121. Pull-Request: #4161.
This commit is contained in:
@ -52,6 +52,7 @@ use libp2p_core::upgrade;
|
||||
use libp2p_core::upgrade::{NegotiationError, ProtocolError};
|
||||
use libp2p_core::Endpoint;
|
||||
use libp2p_identity::PeerId;
|
||||
use std::cmp::max;
|
||||
use std::collections::HashSet;
|
||||
use std::fmt::{Display, Formatter};
|
||||
use std::future::Future;
|
||||
@ -156,6 +157,7 @@ where
|
||||
|
||||
local_supported_protocols: HashSet<StreamProtocol>,
|
||||
remote_supported_protocols: HashSet<StreamProtocol>,
|
||||
idle_timeout: Duration,
|
||||
}
|
||||
|
||||
impl<THandler> fmt::Debug for Connection<THandler>
|
||||
@ -183,9 +185,9 @@ where
|
||||
mut handler: THandler,
|
||||
substream_upgrade_protocol_override: Option<upgrade::Version>,
|
||||
max_negotiating_inbound_streams: usize,
|
||||
idle_timeout: Duration,
|
||||
) -> Self {
|
||||
let initial_protocols = gather_supported_protocols(&handler);
|
||||
|
||||
if !initial_protocols.is_empty() {
|
||||
handler.on_connection_event(ConnectionEvent::LocalProtocolsChange(
|
||||
ProtocolsChange::Added(ProtocolsAdded::from_set(&initial_protocols)),
|
||||
@ -203,6 +205,7 @@ where
|
||||
requested_substreams: Default::default(),
|
||||
local_supported_protocols: initial_protocols,
|
||||
remote_supported_protocols: Default::default(),
|
||||
idle_timeout,
|
||||
}
|
||||
}
|
||||
|
||||
@ -234,6 +237,7 @@ where
|
||||
substream_upgrade_protocol_override,
|
||||
local_supported_protocols: supported_protocols,
|
||||
remote_supported_protocols,
|
||||
idle_timeout,
|
||||
} = self.get_mut();
|
||||
|
||||
loop {
|
||||
@ -348,17 +352,36 @@ where
|
||||
(Shutdown::Later(timer, deadline), KeepAlive::Until(t)) => {
|
||||
if *deadline != t {
|
||||
*deadline = t;
|
||||
if let Some(dur) = deadline.checked_duration_since(Instant::now()) {
|
||||
timer.reset(dur)
|
||||
if let Some(new_duration) = deadline.checked_duration_since(Instant::now())
|
||||
{
|
||||
let effective_keep_alive = max(new_duration, *idle_timeout);
|
||||
|
||||
timer.reset(effective_keep_alive)
|
||||
}
|
||||
}
|
||||
}
|
||||
(_, KeepAlive::Until(t)) => {
|
||||
if let Some(dur) = t.checked_duration_since(Instant::now()) {
|
||||
*shutdown = Shutdown::Later(Delay::new(dur), t)
|
||||
(_, KeepAlive::Until(earliest_shutdown)) => {
|
||||
if let Some(requested_keep_alive) =
|
||||
earliest_shutdown.checked_duration_since(Instant::now())
|
||||
{
|
||||
let effective_keep_alive = max(requested_keep_alive, *idle_timeout);
|
||||
|
||||
// Important: We store the _original_ `Instant` given by the `ConnectionHandler` in the `Later` instance to ensure we can compare it in the above branch.
|
||||
// This is quite subtle but will hopefully become simpler soon once `KeepAlive::Until` is fully deprecated. See <https://github.com/libp2p/rust-libp2p/issues/3844>/
|
||||
*shutdown =
|
||||
Shutdown::Later(Delay::new(effective_keep_alive), earliest_shutdown)
|
||||
}
|
||||
}
|
||||
(_, KeepAlive::No) => *shutdown = Shutdown::Asap,
|
||||
(_, KeepAlive::No) if idle_timeout == &Duration::ZERO => {
|
||||
*shutdown = Shutdown::Asap;
|
||||
}
|
||||
(Shutdown::Later(_, _), KeepAlive::No) => {
|
||||
// Do nothing, i.e. let the shutdown timer continue to tick.
|
||||
}
|
||||
(_, KeepAlive::No) => {
|
||||
let deadline = Instant::now() + *idle_timeout;
|
||||
*shutdown = Shutdown::Later(Delay::new(*idle_timeout), deadline);
|
||||
}
|
||||
(_, KeepAlive::Yes) => *shutdown = Shutdown::None,
|
||||
};
|
||||
|
||||
@ -696,7 +719,7 @@ enum Shutdown {
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::keep_alive;
|
||||
use crate::dummy;
|
||||
use futures::future;
|
||||
use futures::AsyncRead;
|
||||
use futures::AsyncWrite;
|
||||
@ -704,6 +727,7 @@ mod tests {
|
||||
use libp2p_core::StreamMuxer;
|
||||
use quickcheck::*;
|
||||
use std::sync::{Arc, Weak};
|
||||
use std::time::Instant;
|
||||
use void::Void;
|
||||
|
||||
#[test]
|
||||
@ -712,14 +736,14 @@ mod tests {
|
||||
let max_negotiating_inbound_streams: usize = max_negotiating_inbound_streams.into();
|
||||
|
||||
let alive_substream_counter = Arc::new(());
|
||||
|
||||
let mut connection = Connection::new(
|
||||
StreamMuxerBox::new(DummyStreamMuxer {
|
||||
counter: alive_substream_counter.clone(),
|
||||
}),
|
||||
keep_alive::ConnectionHandler,
|
||||
MockConnectionHandler::new(Duration::ZERO),
|
||||
None,
|
||||
max_negotiating_inbound_streams,
|
||||
Duration::ZERO,
|
||||
);
|
||||
|
||||
let result = connection.poll_noop_waker();
|
||||
@ -743,6 +767,7 @@ mod tests {
|
||||
MockConnectionHandler::new(upgrade_timeout),
|
||||
None,
|
||||
2,
|
||||
Duration::ZERO,
|
||||
);
|
||||
|
||||
connection.handler.open_new_outbound();
|
||||
@ -765,6 +790,7 @@ mod tests {
|
||||
ConfigurableProtocolConnectionHandler::default(),
|
||||
None,
|
||||
0,
|
||||
Duration::ZERO,
|
||||
);
|
||||
|
||||
// First, start listening on a single protocol.
|
||||
@ -803,6 +829,7 @@ mod tests {
|
||||
ConfigurableProtocolConnectionHandler::default(),
|
||||
None,
|
||||
0,
|
||||
Duration::ZERO,
|
||||
);
|
||||
|
||||
// First, remote supports a single protocol.
|
||||
@ -846,6 +873,141 @@ mod tests {
|
||||
assert_eq!(connection.handler.remote_removed, vec![vec!["/bar"]]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn idle_timeout_with_keep_alive_no() {
|
||||
let idle_timeout = Duration::from_millis(100);
|
||||
|
||||
let mut connection = Connection::new(
|
||||
StreamMuxerBox::new(PendingStreamMuxer),
|
||||
dummy::ConnectionHandler,
|
||||
None,
|
||||
0,
|
||||
idle_timeout,
|
||||
);
|
||||
|
||||
assert!(connection.poll_noop_waker().is_pending());
|
||||
|
||||
tokio::time::sleep(idle_timeout).await;
|
||||
|
||||
assert!(matches!(
|
||||
connection.poll_noop_waker(),
|
||||
Poll::Ready(Err(ConnectionError::KeepAliveTimeout))
|
||||
));
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn idle_timeout_with_keep_alive_until_greater_than_idle_timeout() {
|
||||
let idle_timeout = Duration::from_millis(100);
|
||||
|
||||
let mut connection = Connection::new(
|
||||
StreamMuxerBox::new(PendingStreamMuxer),
|
||||
KeepAliveUntilConnectionHandler {
|
||||
until: Instant::now() + idle_timeout * 2,
|
||||
},
|
||||
None,
|
||||
0,
|
||||
idle_timeout,
|
||||
);
|
||||
|
||||
assert!(connection.poll_noop_waker().is_pending());
|
||||
|
||||
tokio::time::sleep(idle_timeout).await;
|
||||
|
||||
assert!(
|
||||
connection.poll_noop_waker().is_pending(),
|
||||
"`KeepAlive::Until` is greater than idle-timeout, continue sleeping"
|
||||
);
|
||||
|
||||
tokio::time::sleep(idle_timeout).await;
|
||||
|
||||
assert!(matches!(
|
||||
connection.poll_noop_waker(),
|
||||
Poll::Ready(Err(ConnectionError::KeepAliveTimeout))
|
||||
));
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn idle_timeout_with_keep_alive_until_less_than_idle_timeout() {
|
||||
let idle_timeout = Duration::from_millis(100);
|
||||
|
||||
let mut connection = Connection::new(
|
||||
StreamMuxerBox::new(PendingStreamMuxer),
|
||||
KeepAliveUntilConnectionHandler {
|
||||
until: Instant::now() + idle_timeout / 2,
|
||||
},
|
||||
None,
|
||||
0,
|
||||
idle_timeout,
|
||||
);
|
||||
|
||||
assert!(connection.poll_noop_waker().is_pending());
|
||||
|
||||
tokio::time::sleep(idle_timeout / 2).await;
|
||||
|
||||
assert!(
|
||||
connection.poll_noop_waker().is_pending(),
|
||||
"`KeepAlive::Until` is less than idle-timeout, honor idle-timeout"
|
||||
);
|
||||
|
||||
tokio::time::sleep(idle_timeout / 2).await;
|
||||
|
||||
assert!(matches!(
|
||||
connection.poll_noop_waker(),
|
||||
Poll::Ready(Err(ConnectionError::KeepAliveTimeout))
|
||||
));
|
||||
}
|
||||
|
||||
struct KeepAliveUntilConnectionHandler {
|
||||
until: Instant,
|
||||
}
|
||||
|
||||
impl ConnectionHandler for KeepAliveUntilConnectionHandler {
|
||||
type FromBehaviour = Void;
|
||||
type ToBehaviour = Void;
|
||||
type Error = Void;
|
||||
type InboundProtocol = DeniedUpgrade;
|
||||
type OutboundProtocol = DeniedUpgrade;
|
||||
type InboundOpenInfo = ();
|
||||
type OutboundOpenInfo = Void;
|
||||
|
||||
fn listen_protocol(
|
||||
&self,
|
||||
) -> SubstreamProtocol<Self::InboundProtocol, Self::InboundOpenInfo> {
|
||||
SubstreamProtocol::new(DeniedUpgrade, ())
|
||||
}
|
||||
|
||||
fn connection_keep_alive(&self) -> KeepAlive {
|
||||
KeepAlive::Until(self.until)
|
||||
}
|
||||
|
||||
fn poll(
|
||||
&mut self,
|
||||
_: &mut Context<'_>,
|
||||
) -> Poll<
|
||||
ConnectionHandlerEvent<
|
||||
Self::OutboundProtocol,
|
||||
Self::OutboundOpenInfo,
|
||||
Self::ToBehaviour,
|
||||
Self::Error,
|
||||
>,
|
||||
> {
|
||||
Poll::Pending
|
||||
}
|
||||
|
||||
fn on_behaviour_event(&mut self, _: Self::FromBehaviour) {}
|
||||
|
||||
fn on_connection_event(
|
||||
&mut self,
|
||||
_: ConnectionEvent<
|
||||
Self::InboundProtocol,
|
||||
Self::OutboundProtocol,
|
||||
Self::InboundOpenInfo,
|
||||
Self::OutboundOpenInfo,
|
||||
>,
|
||||
) {
|
||||
}
|
||||
}
|
||||
|
||||
struct DummyStreamMuxer {
|
||||
counter: Arc<()>,
|
||||
}
|
||||
|
@ -37,7 +37,7 @@ use futures::{
|
||||
ready,
|
||||
stream::FuturesUnordered,
|
||||
};
|
||||
use instant::Instant;
|
||||
use instant::{Duration, Instant};
|
||||
use libp2p_core::connection::Endpoint;
|
||||
use libp2p_core::muxing::{StreamMuxerBox, StreamMuxerExt};
|
||||
use std::task::Waker;
|
||||
@ -135,6 +135,9 @@ where
|
||||
|
||||
/// Receivers for [`NewConnection`] objects that are dropped.
|
||||
new_connection_dropped_listeners: FuturesUnordered<oneshot::Receiver<StreamMuxerBox>>,
|
||||
|
||||
/// How long a connection should be kept alive once it starts idling.
|
||||
idle_connection_timeout: Duration,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
@ -322,6 +325,7 @@ where
|
||||
substream_upgrade_protocol_override: config.substream_upgrade_protocol_override,
|
||||
max_negotiating_inbound_streams: config.max_negotiating_inbound_streams,
|
||||
per_connection_event_buffer_size: config.per_connection_event_buffer_size,
|
||||
idle_connection_timeout: config.idle_connection_timeout,
|
||||
executor,
|
||||
pending_connection_events_tx,
|
||||
pending_connection_events_rx,
|
||||
@ -518,6 +522,7 @@ where
|
||||
handler,
|
||||
self.substream_upgrade_protocol_override,
|
||||
self.max_negotiating_inbound_streams,
|
||||
self.idle_connection_timeout,
|
||||
);
|
||||
|
||||
self.executor.spawn(task::new_for_established_connection(
|
||||
@ -947,6 +952,8 @@ pub(crate) struct PoolConfig {
|
||||
pub(crate) per_connection_event_buffer_size: usize,
|
||||
/// Number of addresses concurrently dialed for a single outbound connection attempt.
|
||||
pub(crate) dial_concurrency_factor: NonZeroU8,
|
||||
/// How long a connection should be kept alive once it is idling.
|
||||
pub(crate) idle_connection_timeout: Duration,
|
||||
/// The configured override for substream protocol upgrades, if any.
|
||||
substream_upgrade_protocol_override: Option<libp2p_core::upgrade::Version>,
|
||||
|
||||
@ -963,6 +970,7 @@ impl PoolConfig {
|
||||
task_command_buffer_size: 32,
|
||||
per_connection_event_buffer_size: 7,
|
||||
dial_concurrency_factor: NonZeroU8::new(8).expect("8 > 0"),
|
||||
idle_connection_timeout: Duration::ZERO,
|
||||
substream_upgrade_protocol_override: None,
|
||||
max_negotiating_inbound_streams: 128,
|
||||
}
|
||||
|
@ -67,6 +67,9 @@ pub mod behaviour;
|
||||
pub mod dial_opts;
|
||||
pub mod dummy;
|
||||
pub mod handler;
|
||||
#[deprecated(
|
||||
note = "Configure an appropriate idle connection timeout via `SwarmBuilder::idle_connection_timeout` instead. To keep connections alive 'forever', use `Duration::from_secs(u64::MAX)`."
|
||||
)]
|
||||
pub mod keep_alive;
|
||||
mod listen_opts;
|
||||
|
||||
@ -146,6 +149,7 @@ use libp2p_identity::PeerId;
|
||||
use smallvec::SmallVec;
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::num::{NonZeroU32, NonZeroU8, NonZeroUsize};
|
||||
use std::time::Duration;
|
||||
use std::{
|
||||
convert::TryFrom,
|
||||
error, fmt, io,
|
||||
@ -1518,6 +1522,14 @@ where
|
||||
self
|
||||
}
|
||||
|
||||
/// How long to keep a connection alive once it is idling.
|
||||
///
|
||||
/// Defaults to 0.
|
||||
pub fn idle_connection_timeout(mut self, timeout: Duration) -> Self {
|
||||
self.pool_config.idle_connection_timeout = timeout;
|
||||
self
|
||||
}
|
||||
|
||||
/// Builds a `Swarm` with the current configuration.
|
||||
pub fn build(self) -> Swarm<TBehaviour> {
|
||||
log::info!("Local peer id: {}", self.local_peer_id);
|
||||
@ -1808,6 +1820,7 @@ fn p2p_addr(peer: Option<PeerId>, addr: Multiaddr) -> Result<Multiaddr, Multiadd
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
use crate::dummy;
|
||||
use crate::test::{CallTraceBehaviour, MockBehaviour};
|
||||
use futures::executor::block_on;
|
||||
use futures::executor::ThreadPool;
|
||||
@ -1829,14 +1842,8 @@ mod tests {
|
||||
Disconnecting,
|
||||
}
|
||||
|
||||
fn new_test_swarm<T, O>(
|
||||
handler_proto: T,
|
||||
) -> SwarmBuilder<CallTraceBehaviour<MockBehaviour<T, O>>>
|
||||
where
|
||||
T: ConnectionHandler + Clone,
|
||||
T::ToBehaviour: Clone,
|
||||
O: Send + 'static,
|
||||
{
|
||||
fn new_test_swarm(
|
||||
) -> SwarmBuilder<CallTraceBehaviour<MockBehaviour<dummy::ConnectionHandler, ()>>> {
|
||||
let id_keys = identity::Keypair::generate_ed25519();
|
||||
let local_public_key = id_keys.public();
|
||||
let transport = transport::MemoryTransport::default()
|
||||
@ -1846,13 +1853,15 @@ mod tests {
|
||||
})
|
||||
.multiplex(yamux::Config::default())
|
||||
.boxed();
|
||||
let behaviour = CallTraceBehaviour::new(MockBehaviour::new(handler_proto));
|
||||
match ThreadPool::new().ok() {
|
||||
let behaviour = CallTraceBehaviour::new(MockBehaviour::new(dummy::ConnectionHandler));
|
||||
let builder = match ThreadPool::new().ok() {
|
||||
Some(tp) => {
|
||||
SwarmBuilder::with_executor(transport, behaviour, local_public_key.into(), tp)
|
||||
}
|
||||
None => SwarmBuilder::without_executor(transport, behaviour, local_public_key.into()),
|
||||
}
|
||||
};
|
||||
|
||||
builder.idle_connection_timeout(Duration::from_secs(5))
|
||||
}
|
||||
|
||||
fn swarms_connected<TBehaviour>(
|
||||
@ -1903,12 +1912,8 @@ mod tests {
|
||||
/// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`]
|
||||
#[test]
|
||||
fn test_swarm_disconnect() {
|
||||
// Since the test does not try to open any substreams, we can
|
||||
// use the dummy protocols handler.
|
||||
let handler_proto = keep_alive::ConnectionHandler;
|
||||
|
||||
let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build();
|
||||
let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build();
|
||||
let mut swarm1 = new_test_swarm().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
|
||||
let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
@ -1969,12 +1974,8 @@ mod tests {
|
||||
/// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`]
|
||||
#[test]
|
||||
fn test_behaviour_disconnect_all() {
|
||||
// Since the test does not try to open any substreams, we can
|
||||
// use the dummy protocols handler.
|
||||
let handler_proto = keep_alive::ConnectionHandler;
|
||||
|
||||
let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build();
|
||||
let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build();
|
||||
let mut swarm1 = new_test_swarm().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
|
||||
let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
@ -2039,12 +2040,8 @@ mod tests {
|
||||
/// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`]
|
||||
#[test]
|
||||
fn test_behaviour_disconnect_one() {
|
||||
// Since the test does not try to open any substreams, we can
|
||||
// use the dummy protocols handler.
|
||||
let handler_proto = keep_alive::ConnectionHandler;
|
||||
|
||||
let mut swarm1 = new_test_swarm::<_, ()>(handler_proto.clone()).build();
|
||||
let mut swarm2 = new_test_swarm::<_, ()>(handler_proto).build();
|
||||
let mut swarm1 = new_test_swarm().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
|
||||
let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
@ -2122,7 +2119,7 @@ mod tests {
|
||||
|
||||
fn prop(concurrency_factor: DialConcurrencyFactor) {
|
||||
block_on(async {
|
||||
let mut swarm = new_test_swarm::<_, ()>(keep_alive::ConnectionHandler)
|
||||
let mut swarm = new_test_swarm()
|
||||
.dial_concurrency_factor(concurrency_factor.0)
|
||||
.build();
|
||||
|
||||
@ -2190,8 +2187,8 @@ mod tests {
|
||||
// Checks whether dialing an address containing the wrong peer id raises an error
|
||||
// for the expected peer id instead of the obtained peer id.
|
||||
|
||||
let mut swarm1 = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build();
|
||||
let mut swarm2 = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build();
|
||||
let mut swarm1 = new_test_swarm().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
|
||||
swarm1.listen_on("/memory/0".parse().unwrap()).unwrap();
|
||||
|
||||
@ -2250,7 +2247,7 @@ mod tests {
|
||||
//
|
||||
// The last two can happen in any order.
|
||||
|
||||
let mut swarm = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build();
|
||||
let mut swarm = new_test_swarm().build();
|
||||
swarm.listen_on("/memory/0".parse().unwrap()).unwrap();
|
||||
|
||||
let local_address =
|
||||
@ -2310,7 +2307,7 @@ mod tests {
|
||||
fn dial_self_by_id() {
|
||||
// Trying to dial self by passing the same `PeerId` shouldn't even be possible in the first
|
||||
// place.
|
||||
let swarm = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build();
|
||||
let swarm = new_test_swarm().build();
|
||||
let peer_id = *swarm.local_peer_id();
|
||||
assert!(!swarm.is_connected(&peer_id));
|
||||
}
|
||||
@ -2321,7 +2318,7 @@ mod tests {
|
||||
|
||||
let target = PeerId::random();
|
||||
|
||||
let mut swarm = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build();
|
||||
let mut swarm = new_test_swarm().build();
|
||||
|
||||
let addresses = HashSet::from([
|
||||
multiaddr![Ip4([0, 0, 0, 0]), Tcp(rand::random::<u16>())],
|
||||
@ -2367,8 +2364,8 @@ mod tests {
|
||||
fn aborting_pending_connection_surfaces_error() {
|
||||
let _ = env_logger::try_init();
|
||||
|
||||
let mut dialer = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build();
|
||||
let mut listener = new_test_swarm::<_, ()>(dummy::ConnectionHandler).build();
|
||||
let mut dialer = new_test_swarm().build();
|
||||
let mut listener = new_test_swarm().build();
|
||||
|
||||
let listener_peer_id = *listener.local_peer_id();
|
||||
listener.listen_on(multiaddr![Memory(0u64)]).unwrap();
|
||||
|
Reference in New Issue
Block a user