mirror of
https://github.com/fluencelabs/rust-libp2p
synced 2025-06-26 00:01:33 +00:00
feat(libp2p): add SwarmBuilder
Introduce the new `libp2p::SwarmBuilder`. Users should use the new `libp2p::SwarmBuilder` instead of the now deprecated `libp2p::swarm::SwarmBuilder`. See `libp2p::SwarmBuilder` docs on how to use the new builder. Fixes #3657. Fixes #3563. Fixes #3179. Pull-Request: #4120.
This commit is contained in:
@ -748,6 +748,8 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn max_negotiating_inbound_streams() {
|
||||
let _ = env_logger::try_init();
|
||||
|
||||
fn prop(max_negotiating_inbound_streams: u8) {
|
||||
let max_negotiating_inbound_streams: usize = max_negotiating_inbound_streams.into();
|
||||
|
||||
@ -756,7 +758,7 @@ mod tests {
|
||||
StreamMuxerBox::new(DummyStreamMuxer {
|
||||
counter: alive_substream_counter.clone(),
|
||||
}),
|
||||
MockConnectionHandler::new(Duration::ZERO),
|
||||
MockConnectionHandler::new(Duration::from_secs(10)),
|
||||
None,
|
||||
max_negotiating_inbound_streams,
|
||||
Duration::ZERO,
|
||||
|
293
swarm/src/lib.rs
293
swarm/src/lib.rs
@ -359,6 +359,26 @@ impl<TBehaviour> Swarm<TBehaviour>
|
||||
where
|
||||
TBehaviour: NetworkBehaviour,
|
||||
{
|
||||
/// Creates a new [`Swarm`] from the given [`Transport`], [`NetworkBehaviour`], [`PeerId`] and
|
||||
/// [`Config`].
|
||||
pub fn new(
|
||||
transport: transport::Boxed<(PeerId, StreamMuxerBox)>,
|
||||
behaviour: TBehaviour,
|
||||
local_peer_id: PeerId,
|
||||
config: Config,
|
||||
) -> Self {
|
||||
Swarm {
|
||||
local_peer_id,
|
||||
transport,
|
||||
pool: Pool::new(local_peer_id, config.pool_config),
|
||||
behaviour,
|
||||
supported_protocols: Default::default(),
|
||||
confirmed_external_addr: Default::default(),
|
||||
listened_addrs: HashMap::new(),
|
||||
pending_event: None,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns information about the connections underlying the [`Swarm`].
|
||||
pub fn network_info(&self) -> NetworkInfo {
|
||||
let num_peers = self.pool.num_peers();
|
||||
@ -401,7 +421,9 @@ where
|
||||
/// # use libp2p_swarm::dummy;
|
||||
/// # use libp2p_identity::PeerId;
|
||||
/// #
|
||||
/// let mut swarm = SwarmBuilder::without_executor(
|
||||
/// # #[tokio::main]
|
||||
/// # async fn main() {
|
||||
/// let mut swarm = SwarmBuilder::with_tokio_executor(
|
||||
/// DummyTransport::new().boxed(),
|
||||
/// dummy::Behaviour,
|
||||
/// PeerId::random(),
|
||||
@ -412,6 +434,7 @@ where
|
||||
///
|
||||
/// // Dial an unknown peer.
|
||||
/// swarm.dial("/ip6/::1/tcp/12345".parse::<Multiaddr>().unwrap());
|
||||
/// # }
|
||||
/// ```
|
||||
pub fn dial(&mut self, opts: impl Into<DialOpts>) -> Result<(), DialError> {
|
||||
let dial_opts = opts.into();
|
||||
@ -1346,7 +1369,132 @@ impl<'a> PollParameters for SwarmPollParameters<'a> {
|
||||
}
|
||||
}
|
||||
|
||||
pub struct Config {
|
||||
pool_config: PoolConfig,
|
||||
}
|
||||
|
||||
impl Config {
|
||||
/// Creates a new [`Config`] from the given executor. The [`Swarm`] is obtained via
|
||||
/// [`Swarm::new`].
|
||||
pub fn with_executor(executor: impl Executor + Send + 'static) -> Self {
|
||||
Self {
|
||||
pool_config: PoolConfig::new(Some(Box::new(executor))),
|
||||
}
|
||||
}
|
||||
|
||||
/// Sets executor to the `wasm` executor.
|
||||
/// Background tasks will be executed by the browser on the next micro-tick.
|
||||
///
|
||||
/// Spawning a task is similar too:
|
||||
/// ```typescript
|
||||
/// function spawn(task: () => Promise<void>) {
|
||||
/// task()
|
||||
/// }
|
||||
/// ```
|
||||
#[cfg(feature = "wasm-bindgen")]
|
||||
pub fn with_wasm_executor() -> Self {
|
||||
Self::with_executor(crate::executor::WasmBindgenExecutor)
|
||||
}
|
||||
|
||||
/// Builds a new [`Config`] from the given `tokio` executor.
|
||||
#[cfg(all(
|
||||
feature = "tokio",
|
||||
not(any(target_os = "emscripten", target_os = "wasi", target_os = "unknown"))
|
||||
))]
|
||||
pub fn with_tokio_executor() -> Self {
|
||||
Self::with_executor(crate::executor::TokioExecutor)
|
||||
}
|
||||
|
||||
/// Builds a new [`Config`] from the given `async-std` executor.
|
||||
#[cfg(all(
|
||||
feature = "async-std",
|
||||
not(any(target_os = "emscripten", target_os = "wasi", target_os = "unknown"))
|
||||
))]
|
||||
pub fn with_async_std_executor() -> Self {
|
||||
Self::with_executor(crate::executor::AsyncStdExecutor)
|
||||
}
|
||||
|
||||
/// Configures the number of events from the [`NetworkBehaviour`] in
|
||||
/// destination to the [`ConnectionHandler`] that can be buffered before
|
||||
/// the [`Swarm`] has to wait. An individual buffer with this number of
|
||||
/// events exists for each individual connection.
|
||||
///
|
||||
/// The ideal value depends on the executor used, the CPU speed, and the
|
||||
/// volume of events. If this value is too low, then the [`Swarm`] will
|
||||
/// be sleeping more often than necessary. Increasing this value increases
|
||||
/// the overall memory usage.
|
||||
pub fn with_notify_handler_buffer_size(mut self, n: NonZeroUsize) -> Self {
|
||||
self.pool_config = self.pool_config.with_notify_handler_buffer_size(n);
|
||||
self
|
||||
}
|
||||
|
||||
/// Configures the size of the buffer for events sent by a [`ConnectionHandler`] to the
|
||||
/// [`NetworkBehaviour`].
|
||||
///
|
||||
/// Each connection has its own buffer.
|
||||
///
|
||||
/// The ideal value depends on the executor used, the CPU speed and the volume of events.
|
||||
/// If this value is too low, then the [`ConnectionHandler`]s will be sleeping more often
|
||||
/// than necessary. Increasing this value increases the overall memory
|
||||
/// usage, and more importantly the latency between the moment when an
|
||||
/// event is emitted and the moment when it is received by the
|
||||
/// [`NetworkBehaviour`].
|
||||
pub fn with_per_connection_event_buffer_size(mut self, n: usize) -> Self {
|
||||
self.pool_config = self.pool_config.with_per_connection_event_buffer_size(n);
|
||||
self
|
||||
}
|
||||
|
||||
/// Number of addresses concurrently dialed for a single outbound connection attempt.
|
||||
pub fn with_dial_concurrency_factor(mut self, factor: NonZeroU8) -> Self {
|
||||
self.pool_config = self.pool_config.with_dial_concurrency_factor(factor);
|
||||
self
|
||||
}
|
||||
|
||||
/// Configures an override for the substream upgrade protocol to use.
|
||||
///
|
||||
/// The subtream upgrade protocol is the multistream-select protocol
|
||||
/// used for protocol negotiation on substreams. Since a listener
|
||||
/// supports all existing versions, the choice of upgrade protocol
|
||||
/// only effects the "dialer", i.e. the peer opening a substream.
|
||||
///
|
||||
/// > **Note**: If configured, specific upgrade protocols for
|
||||
/// > individual [`SubstreamProtocol`]s emitted by the `NetworkBehaviour`
|
||||
/// > are ignored.
|
||||
pub fn with_substream_upgrade_protocol_override(
|
||||
mut self,
|
||||
v: libp2p_core::upgrade::Version,
|
||||
) -> Self {
|
||||
self.pool_config = self.pool_config.with_substream_upgrade_protocol_override(v);
|
||||
self
|
||||
}
|
||||
|
||||
/// The maximum number of inbound streams concurrently negotiating on a
|
||||
/// connection. New inbound streams exceeding the limit are dropped and thus
|
||||
/// reset.
|
||||
///
|
||||
/// Note: This only enforces a limit on the number of concurrently
|
||||
/// negotiating inbound streams. The total number of inbound streams on a
|
||||
/// connection is the sum of negotiating and negotiated streams. A limit on
|
||||
/// the total number of streams can be enforced at the
|
||||
/// [`StreamMuxerBox`] level.
|
||||
pub fn with_max_negotiating_inbound_streams(mut self, v: usize) -> Self {
|
||||
self.pool_config = self.pool_config.with_max_negotiating_inbound_streams(v);
|
||||
self
|
||||
}
|
||||
|
||||
/// How long to keep a connection alive once it is idling.
|
||||
///
|
||||
/// Defaults to 0.
|
||||
pub fn with_idle_connection_timeout(mut self, timeout: Duration) -> Self {
|
||||
self.pool_config.idle_connection_timeout = timeout;
|
||||
self
|
||||
}
|
||||
}
|
||||
|
||||
/// A [`SwarmBuilder`] provides an API for configuring and constructing a [`Swarm`].
|
||||
#[deprecated(
|
||||
note = "Use the new `libp2p::SwarmBuilder` instead of `libp2p::swarm::SwarmBuilder` or create a `Swarm` directly via `Swarm::new`."
|
||||
)]
|
||||
pub struct SwarmBuilder<TBehaviour> {
|
||||
local_peer_id: PeerId,
|
||||
transport: transport::Boxed<(PeerId, StreamMuxerBox)>,
|
||||
@ -1354,6 +1502,7 @@ pub struct SwarmBuilder<TBehaviour> {
|
||||
pool_config: PoolConfig,
|
||||
}
|
||||
|
||||
#[allow(deprecated)]
|
||||
impl<TBehaviour> SwarmBuilder<TBehaviour>
|
||||
where
|
||||
TBehaviour: NetworkBehaviour,
|
||||
@ -1820,9 +1969,7 @@ mod tests {
|
||||
use super::*;
|
||||
use crate::dummy;
|
||||
use crate::test::{CallTraceBehaviour, MockBehaviour};
|
||||
use futures::executor::block_on;
|
||||
use futures::executor::ThreadPool;
|
||||
use futures::{executor, future};
|
||||
use futures::future;
|
||||
use libp2p_core::multiaddr::multiaddr;
|
||||
use libp2p_core::transport::memory::MemoryTransportError;
|
||||
use libp2p_core::transport::TransportEvent;
|
||||
@ -1841,7 +1988,8 @@ mod tests {
|
||||
}
|
||||
|
||||
fn new_test_swarm(
|
||||
) -> SwarmBuilder<CallTraceBehaviour<MockBehaviour<dummy::ConnectionHandler, ()>>> {
|
||||
config: Config,
|
||||
) -> Swarm<CallTraceBehaviour<MockBehaviour<dummy::ConnectionHandler, ()>>> {
|
||||
let id_keys = identity::Keypair::generate_ed25519();
|
||||
let local_public_key = id_keys.public();
|
||||
let transport = transport::MemoryTransport::default()
|
||||
@ -1850,14 +1998,13 @@ mod tests {
|
||||
.multiplex(yamux::Config::default())
|
||||
.boxed();
|
||||
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))
|
||||
Swarm::new(
|
||||
transport,
|
||||
behaviour,
|
||||
local_public_key.into(),
|
||||
config.with_idle_connection_timeout(Duration::from_secs(5)),
|
||||
)
|
||||
}
|
||||
|
||||
fn swarms_connected<TBehaviour>(
|
||||
@ -1906,10 +2053,10 @@ mod tests {
|
||||
///
|
||||
/// The test expects both behaviours to be notified via calls to [`NetworkBehaviour::on_swarm_event`]
|
||||
/// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`]
|
||||
#[test]
|
||||
fn test_swarm_disconnect() {
|
||||
let mut swarm1 = new_test_swarm().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
#[tokio::test]
|
||||
async fn test_swarm_disconnect() {
|
||||
let mut swarm1 = new_test_swarm(Config::with_tokio_executor());
|
||||
let mut swarm2 = new_test_swarm(Config::with_tokio_executor());
|
||||
|
||||
let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
@ -1927,7 +2074,7 @@ mod tests {
|
||||
}
|
||||
let mut state = State::Connecting;
|
||||
|
||||
executor::block_on(future::poll_fn(move |cx| loop {
|
||||
future::poll_fn(move |cx| loop {
|
||||
let poll1 = Swarm::poll_next_event(Pin::new(&mut swarm1), cx);
|
||||
let poll2 = Swarm::poll_next_event(Pin::new(&mut swarm2), cx);
|
||||
match state {
|
||||
@ -1959,7 +2106,8 @@ mod tests {
|
||||
if poll1.is_pending() && poll2.is_pending() {
|
||||
return Poll::Pending;
|
||||
}
|
||||
}))
|
||||
})
|
||||
.await
|
||||
}
|
||||
|
||||
/// Establishes multiple connections between two peers,
|
||||
@ -1968,10 +2116,10 @@ mod tests {
|
||||
///
|
||||
/// The test expects both behaviours to be notified via calls to [`NetworkBehaviour::on_swarm_event`]
|
||||
/// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`]
|
||||
#[test]
|
||||
fn test_behaviour_disconnect_all() {
|
||||
let mut swarm1 = new_test_swarm().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
#[tokio::test]
|
||||
async fn test_behaviour_disconnect_all() {
|
||||
let mut swarm1 = new_test_swarm(Config::with_tokio_executor());
|
||||
let mut swarm2 = new_test_swarm(Config::with_tokio_executor());
|
||||
|
||||
let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
@ -1989,7 +2137,7 @@ mod tests {
|
||||
}
|
||||
let mut state = State::Connecting;
|
||||
|
||||
executor::block_on(future::poll_fn(move |cx| loop {
|
||||
future::poll_fn(move |cx| loop {
|
||||
let poll1 = Swarm::poll_next_event(Pin::new(&mut swarm1), cx);
|
||||
let poll2 = Swarm::poll_next_event(Pin::new(&mut swarm2), cx);
|
||||
match state {
|
||||
@ -2025,7 +2173,8 @@ mod tests {
|
||||
if poll1.is_pending() && poll2.is_pending() {
|
||||
return Poll::Pending;
|
||||
}
|
||||
}))
|
||||
})
|
||||
.await
|
||||
}
|
||||
|
||||
/// Establishes multiple connections between two peers,
|
||||
@ -2034,10 +2183,10 @@ mod tests {
|
||||
///
|
||||
/// The test expects both behaviours to be notified via calls to [`NetworkBehaviour::on_swarm_event`]
|
||||
/// with pairs of [`FromSwarm::ConnectionEstablished`] / [`FromSwarm::ConnectionClosed`]
|
||||
#[test]
|
||||
fn test_behaviour_disconnect_one() {
|
||||
let mut swarm1 = new_test_swarm().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
#[tokio::test]
|
||||
async fn test_behaviour_disconnect_one() {
|
||||
let mut swarm1 = new_test_swarm(Config::with_tokio_executor());
|
||||
let mut swarm2 = new_test_swarm(Config::with_tokio_executor());
|
||||
|
||||
let addr1: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
let addr2: Multiaddr = multiaddr::Protocol::Memory(rand::random::<u64>()).into();
|
||||
@ -2055,7 +2204,7 @@ mod tests {
|
||||
let mut state = State::Connecting;
|
||||
let mut disconnected_conn_id = None;
|
||||
|
||||
executor::block_on(future::poll_fn(move |cx| loop {
|
||||
future::poll_fn(move |cx| loop {
|
||||
let poll1 = Swarm::poll_next_event(Pin::new(&mut swarm1), cx);
|
||||
let poll2 = Swarm::poll_next_event(Pin::new(&mut swarm2), cx);
|
||||
match state {
|
||||
@ -2099,7 +2248,8 @@ mod tests {
|
||||
if poll1.is_pending() && poll2.is_pending() {
|
||||
return Poll::Pending;
|
||||
}
|
||||
}))
|
||||
})
|
||||
.await
|
||||
}
|
||||
|
||||
#[test]
|
||||
@ -2114,10 +2264,11 @@ mod tests {
|
||||
}
|
||||
|
||||
fn prop(concurrency_factor: DialConcurrencyFactor) {
|
||||
block_on(async {
|
||||
let mut swarm = new_test_swarm()
|
||||
.dial_concurrency_factor(concurrency_factor.0)
|
||||
.build();
|
||||
tokio::runtime::Runtime::new().unwrap().block_on(async {
|
||||
let mut swarm = new_test_swarm(
|
||||
Config::with_tokio_executor()
|
||||
.with_dial_concurrency_factor(concurrency_factor.0),
|
||||
);
|
||||
|
||||
// Listen on `concurrency_factor + 1` addresses.
|
||||
//
|
||||
@ -2173,31 +2324,29 @@ mod tests {
|
||||
QuickCheck::new().tests(10).quickcheck(prop as fn(_) -> _);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn invalid_peer_id() {
|
||||
#[tokio::test]
|
||||
async fn invalid_peer_id() {
|
||||
// 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().build();
|
||||
let mut swarm2 = new_test_swarm().build();
|
||||
let mut swarm1 = new_test_swarm(Config::with_tokio_executor());
|
||||
let mut swarm2 = new_test_swarm(Config::with_tokio_executor());
|
||||
|
||||
swarm1.listen_on("/memory/0".parse().unwrap()).unwrap();
|
||||
|
||||
let address =
|
||||
futures::executor::block_on(future::poll_fn(|cx| match swarm1.poll_next_unpin(cx) {
|
||||
Poll::Ready(Some(SwarmEvent::NewListenAddr { address, .. })) => {
|
||||
Poll::Ready(address)
|
||||
}
|
||||
Poll::Pending => Poll::Pending,
|
||||
_ => panic!("Was expecting the listen address to be reported"),
|
||||
}));
|
||||
let address = future::poll_fn(|cx| match swarm1.poll_next_unpin(cx) {
|
||||
Poll::Ready(Some(SwarmEvent::NewListenAddr { address, .. })) => Poll::Ready(address),
|
||||
Poll::Pending => Poll::Pending,
|
||||
_ => panic!("Was expecting the listen address to be reported"),
|
||||
})
|
||||
.await;
|
||||
|
||||
let other_id = PeerId::random();
|
||||
let other_addr = address.with(multiaddr::Protocol::P2p(other_id));
|
||||
|
||||
swarm2.dial(other_addr.clone()).unwrap();
|
||||
|
||||
let (peer_id, error) = futures::executor::block_on(future::poll_fn(|cx| {
|
||||
let (peer_id, error) = future::poll_fn(|cx| {
|
||||
if let Poll::Ready(Some(SwarmEvent::IncomingConnection { .. })) =
|
||||
swarm1.poll_next_unpin(cx)
|
||||
{}
|
||||
@ -2209,7 +2358,8 @@ mod tests {
|
||||
Poll::Ready(x) => panic!("unexpected {x:?}"),
|
||||
Poll::Pending => Poll::Pending,
|
||||
}
|
||||
}));
|
||||
})
|
||||
.await;
|
||||
assert_eq!(peer_id.unwrap(), other_id);
|
||||
match error {
|
||||
DialError::WrongPeerId { obtained, endpoint } => {
|
||||
@ -2226,8 +2376,8 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn dial_self() {
|
||||
#[tokio::test]
|
||||
async fn dial_self() {
|
||||
// Check whether dialing ourselves correctly fails.
|
||||
//
|
||||
// Dialing the same address we're listening should result in three events:
|
||||
@ -2238,17 +2388,15 @@ mod tests {
|
||||
//
|
||||
// The last two can happen in any order.
|
||||
|
||||
let mut swarm = new_test_swarm().build();
|
||||
let mut swarm = new_test_swarm(Config::with_tokio_executor());
|
||||
swarm.listen_on("/memory/0".parse().unwrap()).unwrap();
|
||||
|
||||
let local_address =
|
||||
futures::executor::block_on(future::poll_fn(|cx| match swarm.poll_next_unpin(cx) {
|
||||
Poll::Ready(Some(SwarmEvent::NewListenAddr { address, .. })) => {
|
||||
Poll::Ready(address)
|
||||
}
|
||||
Poll::Pending => Poll::Pending,
|
||||
_ => panic!("Was expecting the listen address to be reported"),
|
||||
}));
|
||||
let local_address = future::poll_fn(|cx| match swarm.poll_next_unpin(cx) {
|
||||
Poll::Ready(Some(SwarmEvent::NewListenAddr { address, .. })) => Poll::Ready(address),
|
||||
Poll::Pending => Poll::Pending,
|
||||
_ => panic!("Was expecting the listen address to be reported"),
|
||||
})
|
||||
.await;
|
||||
|
||||
swarm.listened_addrs.clear(); // This is a hack to actually execute the dial to ourselves which would otherwise be filtered.
|
||||
|
||||
@ -2256,7 +2404,7 @@ mod tests {
|
||||
|
||||
let mut got_dial_err = false;
|
||||
let mut got_inc_err = false;
|
||||
futures::executor::block_on(future::poll_fn(|cx| -> Poll<Result<(), io::Error>> {
|
||||
future::poll_fn(|cx| -> Poll<Result<(), io::Error>> {
|
||||
loop {
|
||||
match swarm.poll_next_unpin(cx) {
|
||||
Poll::Ready(Some(SwarmEvent::OutgoingConnectionError {
|
||||
@ -2290,26 +2438,27 @@ mod tests {
|
||||
Poll::Pending => break Poll::Pending,
|
||||
}
|
||||
}
|
||||
}))
|
||||
})
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn dial_self_by_id() {
|
||||
#[tokio::test]
|
||||
async 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().build();
|
||||
let swarm = new_test_swarm(Config::with_tokio_executor());
|
||||
let peer_id = *swarm.local_peer_id();
|
||||
assert!(!swarm.is_connected(&peer_id));
|
||||
}
|
||||
|
||||
#[async_std::test]
|
||||
#[tokio::test]
|
||||
async fn multiple_addresses_err() {
|
||||
// Tries dialing multiple addresses, and makes sure there's one dialing error per address.
|
||||
|
||||
let target = PeerId::random();
|
||||
|
||||
let mut swarm = new_test_swarm().build();
|
||||
let mut swarm = new_test_swarm(Config::with_tokio_executor());
|
||||
|
||||
let addresses = HashSet::from([
|
||||
multiaddr![Ip4([0, 0, 0, 0]), Tcp(rand::random::<u16>())],
|
||||
@ -2351,16 +2500,16 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn aborting_pending_connection_surfaces_error() {
|
||||
#[tokio::test]
|
||||
async fn aborting_pending_connection_surfaces_error() {
|
||||
let _ = env_logger::try_init();
|
||||
|
||||
let mut dialer = new_test_swarm().build();
|
||||
let mut listener = new_test_swarm().build();
|
||||
let mut dialer = new_test_swarm(Config::with_tokio_executor());
|
||||
let mut listener = new_test_swarm(Config::with_tokio_executor());
|
||||
|
||||
let listener_peer_id = *listener.local_peer_id();
|
||||
listener.listen_on(multiaddr![Memory(0u64)]).unwrap();
|
||||
let listener_address = match block_on(listener.next()).unwrap() {
|
||||
let listener_address = match listener.next().await.unwrap() {
|
||||
SwarmEvent::NewListenAddr { address, .. } => address,
|
||||
e => panic!("Unexpected network event: {e:?}"),
|
||||
};
|
||||
@ -2377,7 +2526,7 @@ mod tests {
|
||||
.disconnect_peer_id(listener_peer_id)
|
||||
.expect_err("Expect peer to not yet be connected.");
|
||||
|
||||
match block_on(dialer.next()).unwrap() {
|
||||
match dialer.next().await.unwrap() {
|
||||
SwarmEvent::OutgoingConnectionError {
|
||||
error: DialError::Aborted,
|
||||
..
|
||||
|
Reference in New Issue
Block a user