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:
Max Inden
2023-10-10 08:55:14 +02:00
committed by GitHub
parent 3ae72557ab
commit d605255fec
62 changed files with 2970 additions and 940 deletions

View File

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

View File

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