From 7431703b7e2617b731d4d63d2228018601337939 Mon Sep 17 00:00:00 2001 From: David Craven Date: Fri, 16 Jul 2021 16:01:46 +0200 Subject: [PATCH 01/14] Add libp2p-quic. --- Cargo.toml | 8 +- src/lib.rs | 4 + transports/quic/Cargo.toml | 34 +++ transports/quic/src/endpoint.rs | 399 ++++++++++++++++++++++++++++++ transports/quic/src/lib.rs | 117 +++++++++ transports/quic/src/muxer.rs | 411 +++++++++++++++++++++++++++++++ transports/quic/src/transport.rs | 292 ++++++++++++++++++++++ transports/quic/tests/smoke.rs | 241 ++++++++++++++++++ 8 files changed, 1504 insertions(+), 2 deletions(-) create mode 100644 transports/quic/Cargo.toml create mode 100644 transports/quic/src/endpoint.rs create mode 100644 transports/quic/src/lib.rs create mode 100644 transports/quic/src/muxer.rs create mode 100644 transports/quic/src/transport.rs create mode 100644 transports/quic/tests/smoke.rs diff --git a/Cargo.toml b/Cargo.toml index a8519947a8e..e5d32316c2e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -14,15 +14,16 @@ default = [ "deflate", "dns-async-std", "floodsub", + "gossipsub", "identify", "kad", - "gossipsub", "mdns", "mplex", "noise", "ping", "plaintext", "pnet", + "quic", "relay", "request-response", "secp256k1", @@ -36,15 +37,16 @@ deflate = ["libp2p-deflate"] dns-async-std = ["libp2p-dns", "libp2p-dns/async-std"] dns-tokio = ["libp2p-dns", "libp2p-dns/tokio"] floodsub = ["libp2p-floodsub"] +gossipsub = ["libp2p-gossipsub"] identify = ["libp2p-identify"] kad = ["libp2p-kad"] -gossipsub = ["libp2p-gossipsub"] mdns = ["libp2p-mdns"] mplex = ["libp2p-mplex"] noise = ["libp2p-noise"] ping = ["libp2p-ping"] plaintext = ["libp2p-plaintext"] pnet = ["libp2p-pnet"] +quic = ["libp2p-quic"] relay = ["libp2p-relay"] request-response = ["libp2p-request-response"] tcp-async-io = ["libp2p-tcp", "libp2p-tcp/async-io"] @@ -91,6 +93,7 @@ wasm-timer = "0.2.4" libp2p-deflate = { version = "0.29.0", path = "transports/deflate", optional = true } libp2p-dns = { version = "0.29.0", path = "transports/dns", optional = true, default-features = false } libp2p-mdns = { version = "0.31.0", path = "protocols/mdns", optional = true } +libp2p-quic = { version = "0.5.0", path = "transports/quic", optional = true } libp2p-tcp = { version = "0.29.0", path = "transports/tcp", default-features = false, optional = true } libp2p-websocket = { version = "0.30.0", path = "transports/websocket", optional = true } @@ -122,6 +125,7 @@ members = [ "transports/noise", "transports/plaintext", "transports/pnet", + "transports/quic", "transports/tcp", "transports/uds", "transports/websocket", diff --git a/src/lib.rs b/src/lib.rs index e675b40e7f0..f384e9d021b 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -89,6 +89,10 @@ pub use libp2p_ping as ping; #[cfg_attr(docsrs, doc(cfg(feature = "plaintext")))] #[doc(inline)] pub use libp2p_plaintext as plaintext; +#[cfg(feature = "quic")] +#[cfg_attr(docsrs, doc(cfg(feature = "quic")))] +#[doc(inline)] +pub use libp2p_quic as quic; #[doc(inline)] pub use libp2p_swarm as swarm; #[cfg(any(feature = "tcp-async-io", feature = "tcp-tokio"))] diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml new file mode 100644 index 00000000000..528e2bc9af4 --- /dev/null +++ b/transports/quic/Cargo.toml @@ -0,0 +1,34 @@ +[package] +name = "libp2p-quic" +version = "0.5.0" +authors = ["David Craven "] +edition = "2018" +description = "libp2p-quic is a noise based quic implementation for rust-libp2p." +repository = "https://github.com/libp2p/rust-libp2p" +license = "MIT" + +[dependencies] +anyhow = "1.0.41" +async-global-executor = "2.0.2" +async-io = "1.6.0" +bytes = "1.0.1" +fnv = "1.0.7" +futures = "0.3.15" +if-watch = "0.2.2" +libp2p-core = { path = "../../core", default-features = false } +multihash = { version = "0.14.0", default-features = false } +parking_lot = "0.11.1" +quinn-proto = { version = "0.7.3", default-features = false } +quinn-noise = "0.2.1" +rand_core = "0.5.1" +thiserror = "1.0.26" +tracing = "0.1.26" +udp-socket = "0.1.5" + +[dev-dependencies] +async-std = { version = "1.9.0", features = ["attributes"] } +async-trait = "0.1.50" +libp2p = { path = "../..", default-features = false, features = ["request-response"] } +log-panics = "2.0.0" +rand = "0.8.4" +tracing-subscriber = "0.2.19" diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs new file mode 100644 index 00000000000..5c8a2fd6503 --- /dev/null +++ b/transports/quic/src/endpoint.rs @@ -0,0 +1,399 @@ +use crate::muxer::QuicMuxer; +use crate::{PublicKey, QuicConfig, QuicError}; +use fnv::FnvHashMap; +use futures::channel::{mpsc, oneshot}; +use futures::prelude::*; +use quinn_noise::{NoiseConfig, NoiseSession}; +use quinn_proto::generic::{ClientConfig, ServerConfig}; +use quinn_proto::{ + ConnectionEvent, ConnectionHandle, DatagramEvent, EcnCodepoint, EndpointEvent, Transmit, +}; +use std::collections::VecDeque; +use std::io::IoSliceMut; +use std::mem::MaybeUninit; +use std::net::SocketAddr; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; +use std::time::Instant; +use udp_socket::{RecvMeta, SocketType, UdpCapabilities, UdpSocket, BATCH_SIZE}; + +/// Message sent to the endpoint background task. +#[derive(Debug)] +enum ToEndpoint { + /// Instructs the endpoint to start connecting to the given address. + Dial { + /// UDP address to connect to. + addr: SocketAddr, + /// The remotes public key. + public_key: PublicKey, + /// Channel to return the result of the dialing to. + tx: oneshot::Sender>, + }, + /// Sent by a `quinn_proto` connection when the endpoint needs to process an event generated + /// by a connection. The event itself is opaque to us. + ConnectionEvent { + connection_id: ConnectionHandle, + event: EndpointEvent, + }, + /// Instruct the endpoint to transmit a packet on its UDP socket. + Transmit(Transmit), +} + +#[derive(Debug)] +pub struct TransportChannel { + tx: mpsc::UnboundedSender, + rx: mpsc::UnboundedReceiver>, + port: u16, + ty: SocketType, +} + +impl TransportChannel { + pub fn dial( + &mut self, + addr: SocketAddr, + public_key: PublicKey, + ) -> oneshot::Receiver> { + let (tx, rx) = oneshot::channel(); + let msg = ToEndpoint::Dial { + addr, + public_key, + tx, + }; + self.tx.unbounded_send(msg).expect("endpoint has crashed"); + rx + } + + pub fn poll_incoming( + &mut self, + cx: &mut Context, + ) -> Poll>> { + Pin::new(&mut self.rx).poll_next(cx) + } + + pub fn port(&self) -> u16 { + self.port + } + + pub fn ty(&self) -> SocketType { + self.ty + } +} + +#[derive(Debug)] +pub struct ConnectionChannel { + id: ConnectionHandle, + tx: mpsc::UnboundedSender, + rx: mpsc::UnboundedReceiver, + port: u16, + max_datagrams: usize, +} + +impl ConnectionChannel { + pub fn poll_channel_events(&mut self, cx: &mut Context) -> Poll { + match Pin::new(&mut self.rx).poll_next(cx) { + Poll::Ready(Some(event)) => Poll::Ready(event), + Poll::Ready(None) => panic!("endpoint has crashed"), + Poll::Pending => Poll::Pending, + } + } + + pub fn send_endpoint_event(&mut self, event: EndpointEvent) { + let msg = ToEndpoint::ConnectionEvent { + connection_id: self.id, + event, + }; + self.tx.unbounded_send(msg).expect("endpoint has crashed") + } + + pub fn send_transmit(&mut self, transmit: Transmit) { + let msg = ToEndpoint::Transmit(transmit); + self.tx.unbounded_send(msg).expect("endpoint has crashed") + } + + pub fn port(&self) -> u16 { + self.port + } + + pub fn max_datagrams(&self) -> usize { + self.max_datagrams + } +} + +#[derive(Debug)] +struct EndpointChannel { + rx: mpsc::UnboundedReceiver, + tx: mpsc::UnboundedSender>, + port: u16, + max_datagrams: usize, + connection_tx: mpsc::UnboundedSender, +} + +impl EndpointChannel { + pub fn send_incoming(&mut self, muxer: QuicMuxer) { + self.tx.unbounded_send(Ok(muxer)).ok(); + } + + pub fn poll_next_event(&mut self, cx: &mut Context) -> Poll> { + Pin::new(&mut self.rx).poll_next(cx) + } + + pub fn create_connection( + &self, + id: ConnectionHandle, + ) -> (ConnectionChannel, mpsc::UnboundedSender) { + let (tx, rx) = mpsc::unbounded(); + let channel = ConnectionChannel { + id, + tx: self.connection_tx.clone(), + rx, + port: self.port, + max_datagrams: self.max_datagrams, + }; + (channel, tx) + } +} + +type QuinnEndpointConfig = quinn_proto::generic::EndpointConfig; +type QuinnEndpoint = quinn_proto::generic::Endpoint; + +pub struct EndpointConfig { + socket: UdpSocket, + endpoint: QuinnEndpoint, + port: u16, + client_config: ClientConfig, + capabilities: UdpCapabilities, +} + +impl EndpointConfig { + pub fn new(mut config: QuicConfig, addr: SocketAddr) -> Result { + config.transport.max_concurrent_uni_streams(0)?; + config.transport.datagram_receive_buffer_size(None); + let transport = Arc::new(config.transport); + + let noise_config = NoiseConfig { + keypair: Some(config.keypair), + psk: config.psk, + remote_public_key: None, + keylogger: config.keylogger, + }; + + let mut server_config = ServerConfig::::default(); + server_config.transport = transport.clone(); + server_config.crypto = Arc::new(noise_config.clone()); + + let client_config = ClientConfig:: { + transport, + crypto: noise_config, + }; + + let mut endpoint_config = QuinnEndpointConfig::default(); + endpoint_config.supported_versions( + quinn_noise::SUPPORTED_QUIC_VERSIONS.to_vec(), + quinn_noise::DEFAULT_QUIC_VERSION, + )?; + + let socket = UdpSocket::bind(addr)?; + let port = socket.local_addr()?.port(); + let endpoint = quinn_proto::generic::Endpoint::new( + Arc::new(endpoint_config), + Some(Arc::new(server_config)), + ); + let capabilities = UdpSocket::capabilities()?; + Ok(Self { + socket, + endpoint, + port, + client_config, + capabilities, + }) + } + + pub fn spawn(self) -> TransportChannel { + let (tx1, rx1) = mpsc::unbounded(); + let (tx2, rx2) = mpsc::unbounded(); + let transport = TransportChannel { + tx: tx1, + rx: rx2, + port: self.port, + ty: self.socket.socket_type(), + }; + let endpoint = EndpointChannel { + tx: tx2, + rx: rx1, + port: self.port, + max_datagrams: self.capabilities.max_gso_segments, + connection_tx: transport.tx.clone(), + }; + async_global_executor::spawn(Endpoint::new(endpoint, self)).detach(); + transport + } +} + +struct Endpoint { + channel: EndpointChannel, + endpoint: QuinnEndpoint, + socket: UdpSocket, + client_config: ClientConfig, + connections: FnvHashMap>, + outgoing: VecDeque, + recv_buf: Box<[u8]>, +} + +impl Endpoint { + pub fn new(channel: EndpointChannel, config: EndpointConfig) -> Self { + let max_udp_payload_size = config + .endpoint + .config() + .get_max_udp_payload_size() + .min(u16::MAX as _) as usize; + let recv_buf = vec![0; max_udp_payload_size * BATCH_SIZE].into_boxed_slice(); + Self { + channel, + endpoint: config.endpoint, + socket: config.socket, + client_config: config.client_config, + connections: Default::default(), + outgoing: Default::default(), + recv_buf, + } + } + + pub fn transmit(&mut self, transmit: Transmit) { + let ecn = transmit + .ecn + .map(|ecn| udp_socket::EcnCodepoint::from_bits(ecn as u8)) + .unwrap_or_default(); + let transmit = udp_socket::Transmit { + destination: transmit.destination, + contents: transmit.contents, + ecn, + segment_size: transmit.segment_size, + src_ip: transmit.src_ip, + }; + self.outgoing.push_back(transmit); + } +} + +impl Future for Endpoint { + type Output = (); + + fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { + let me = Pin::into_inner(self); + + while let Some(transmit) = me.endpoint.poll_transmit() { + me.transmit(transmit); + } + + while let Poll::Ready(event) = me.channel.poll_next_event(cx) { + match event { + Some(ToEndpoint::Dial { + addr, + public_key, + tx, + }) => { + let mut client_config = me.client_config.clone(); + client_config.crypto.remote_public_key = Some(public_key); + let (id, connection) = + match me.endpoint.connect(client_config, addr, "server_name") { + Ok(c) => c, + Err(err) => { + tracing::error!("dial failure: {}", err); + let _ = tx.send(Err(err.into())); + continue; + } + }; + let (channel, conn) = me.channel.create_connection(id); + me.connections.insert(id, conn); + let muxer = QuicMuxer::new(channel, connection); + tx.send(Ok(muxer)).ok(); + } + Some(ToEndpoint::ConnectionEvent { + connection_id, + event, + }) => { + let is_drained_event = event.is_drained(); + if is_drained_event { + me.connections.remove(&connection_id); + } + if let Some(event) = me.endpoint.handle_event(connection_id, event) { + me.connections + .get_mut(&connection_id) + .unwrap() + .unbounded_send(event) + .ok(); + } + } + Some(ToEndpoint::Transmit(transmit)) => { + me.transmit(transmit); + } + None => { + me.endpoint.reject_new_connections(); + return Poll::Ready(()); + } + } + } + + while !me.outgoing.is_empty() { + me.outgoing.make_contiguous(); + match me.socket.poll_send(cx, me.outgoing.as_slices().0) { + Poll::Ready(Ok(n)) => { + me.outgoing.drain(..n); + } + Poll::Ready(Err(err)) => tracing::error!("send_to: {}", err), + Poll::Pending => break, + } + } + + let mut metas = [RecvMeta::default(); BATCH_SIZE]; + let mut iovs = MaybeUninit::<[IoSliceMut; BATCH_SIZE]>::uninit(); + me.recv_buf + .chunks_mut(me.recv_buf.len() / BATCH_SIZE) + .enumerate() + .for_each(|(i, buf)| unsafe { + iovs.as_mut_ptr() + .cast::() + .add(i) + .write(IoSliceMut::new(buf)); + }); + let mut iovs = unsafe { iovs.assume_init() }; + while let Poll::Ready(result) = me.socket.poll_recv(cx, &mut iovs, &mut metas) { + let n = match result { + Ok(n) => n, + Err(err) => { + tracing::error!("recv_from: {}", err); + continue; + } + }; + for i in 0..n { + let meta = &metas[i]; + let packet = From::from(&iovs[i][..meta.len]); + let ecn = meta + .ecn + .map(|ecn| EcnCodepoint::from_bits(ecn as u8)) + .unwrap_or_default(); + match me + .endpoint + .handle(Instant::now(), meta.source, meta.dst_ip, ecn, packet) + { + None => {} + Some((id, DatagramEvent::ConnectionEvent(event))) => { + me.connections + .get_mut(&id) + .unwrap() + .unbounded_send(event) + .ok(); + } + Some((id, DatagramEvent::NewConnection(connection))) => { + let (channel, tx) = me.channel.create_connection(id); + me.connections.insert(id, tx); + let muxer = QuicMuxer::new(channel, connection); + let _ = me.channel.send_incoming(muxer); + } + } + } + } + + Poll::Pending + } +} diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs new file mode 100644 index 00000000000..254d78631d2 --- /dev/null +++ b/transports/quic/src/lib.rs @@ -0,0 +1,117 @@ +mod endpoint; +mod muxer; +mod transport; + +pub use crate::muxer::{QuicMuxer, QuicMuxerError}; +pub use crate::transport::{QuicDial, QuicTransport}; +pub use quinn_noise::{KeyLog, KeyLogFile, Keypair, PublicKey, SecretKey}; +pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; + +use libp2p_core::identity; +use libp2p_core::transport::TransportError; +use libp2p_core::{Multiaddr, PeerId}; +use std::sync::Arc; +use thiserror::Error; + +pub fn generate_keypair() -> Keypair { + Keypair::generate(&mut rand_core::OsRng {}) +} + +/// Quic configuration. +pub struct QuicConfig { + pub keypair: Keypair, + pub psk: Option<[u8; 32]>, + pub transport: TransportConfig, + pub keylogger: Option>, +} + +impl Default for QuicConfig { + fn default() -> Self { + Self { + keypair: Keypair::generate(&mut rand_core::OsRng {}), + psk: None, + transport: TransportConfig::default(), + keylogger: None, + } + } +} + +impl std::fmt::Debug for QuicConfig { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("QuicConfig") + .field("keypair", &self.keypair.public) + .field("psk", &self.psk) + .field("transport", &self.transport) + .finish() + } +} + +impl QuicConfig { + /// Creates a new config from a keypair. + pub fn new(keypair: Keypair) -> Self { + Self { + keypair, + ..Default::default() + } + } + + /// Enable keylogging. + pub fn enable_keylogger(&mut self) -> &mut Self { + self.keylogger = Some(Arc::new(KeyLogFile::new())); + self + } + + /// Spawns a new endpoint. + pub async fn listen_on( + self, + addr: Multiaddr, + ) -> Result> { + QuicTransport::new(self, addr).await + } +} + +#[derive(Debug, Error)] +pub enum QuicError { + #[error("{0}")] + Config(#[from] ConfigError), + #[error("{0}")] + Connect(#[from] ConnectError), + #[error("{0}")] + Muxer(#[from] QuicMuxerError), + #[error("{0}")] + Io(#[from] std::io::Error), + #[error("a `StreamMuxerEvent` was generated before the handshake was complete.")] + UpgradeError, +} + +pub trait ToLibp2p { + fn to_keypair(&self) -> identity::Keypair; + fn to_public(&self) -> identity::PublicKey; + fn to_peer_id(&self) -> PeerId { + self.to_public().into_peer_id() + } +} + +impl ToLibp2p for Keypair { + fn to_keypair(&self) -> identity::Keypair { + let mut secret_key = self.secret.to_bytes(); + let secret_key = identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); + identity::Keypair::Ed25519(secret_key.into()) + } + + fn to_public(&self) -> identity::PublicKey { + self.public.to_public() + } +} + +impl ToLibp2p for PublicKey { + fn to_keypair(&self) -> identity::Keypair { + panic!("wtf?"); + } + + fn to_public(&self) -> identity::PublicKey { + let public_key = self.to_bytes(); + let public_key = identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); + identity::PublicKey::Ed25519(public_key.into()) + } +} diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs new file mode 100644 index 00000000000..e2d0f2b6d55 --- /dev/null +++ b/transports/quic/src/muxer.rs @@ -0,0 +1,411 @@ +use crate::endpoint::ConnectionChannel; +use crate::ToLibp2p; +use async_io::Timer; +use fnv::FnvHashMap; +use futures::prelude::*; +use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; +use libp2p_core::{Multiaddr, PeerId}; +use parking_lot::Mutex; +use quinn_noise::NoiseSession; +use quinn_proto::crypto::Session; +use quinn_proto::generic::Connection; +use quinn_proto::{ + ConnectionError, Dir, Event, FinishError, ReadError, ReadableError, StreamEvent, StreamId, + VarInt, WriteError, +}; +use std::collections::VecDeque; +use std::io::Write; +use std::net::{IpAddr, Ipv4Addr, SocketAddr}; +use std::pin::Pin; +use std::task::{Context, Poll, Waker}; +use std::time::Instant; +use thiserror::Error; + +/// State for a single opened QUIC connection. +#[derive(Debug)] +pub struct QuicMuxer { + inner: Mutex, +} + +/// Mutex protected fields of [`QuicMuxer`]. +#[derive(Debug)] +struct QuicMuxerInner { + /// Endpoint channel. + endpoint: ConnectionChannel, + /// Inner connection object that yields events. + connection: Connection, + /// Connection waker. + waker: Option, + /// Connection timer. + timer: Option, + /// State of all open substreams. + substreams: FnvHashMap, + /// Pending substreams. + pending_substreams: VecDeque, + /// Close waker. + close_waker: Option, +} + +/// State of a single substream. +#[derive(Debug, Default)] +struct SubstreamState { + /// Waker to wake if the substream becomes readable. + read_waker: Option, + /// Waker to wake if the substream becomes writable. + write_waker: Option, +} + +impl QuicMuxer { + pub fn new(endpoint: ConnectionChannel, connection: Connection) -> Self { + Self { + inner: Mutex::new(QuicMuxerInner { + endpoint, + connection, + waker: None, + timer: None, + substreams: Default::default(), + pending_substreams: Default::default(), + close_waker: None, + }), + } + } + + pub fn is_handshaking(&self) -> bool { + self.inner.lock().connection.is_handshaking() + } + + pub fn peer_id(&self) -> PeerId { + self.inner + .lock() + .connection + .crypto_session() + .peer_identity() + .expect("In an IK handshake the PeerId is always available") + .to_peer_id() + } + + pub fn local_addr(&self) -> Multiaddr { + let inner = self.inner.lock(); + let ip = inner + .connection + .local_ip() + .unwrap_or(IpAddr::V4(Ipv4Addr::UNSPECIFIED)); + let addr = SocketAddr::new(ip, inner.endpoint.port()); + crate::transport::socketaddr_to_multiaddr(&addr) + } + + pub fn remote_addr(&self) -> Multiaddr { + let inner = self.inner.lock(); + let addr = inner.connection.remote_address(); + crate::transport::socketaddr_to_multiaddr(&addr) + } +} + +impl StreamMuxer for QuicMuxer { + type Substream = StreamId; + type OutboundSubstream = (); + type Error = QuicMuxerError; + + fn poll_event( + &self, + cx: &mut Context, + ) -> Poll, Self::Error>> { + let mut inner = self.inner.lock(); + let now = Instant::now(); + + while let Poll::Ready(event) = inner.endpoint.poll_channel_events(cx) { + inner.connection.handle_event(event); + } + + let _max_datagrams = inner.endpoint.max_datagrams(); + while let Some(transmit) = inner.connection.poll_transmit(now) { + inner.endpoint.send_transmit(transmit); + } + + loop { + if let Some(timer) = inner.timer.as_mut() { + match Pin::new(timer).poll(cx) { + Poll::Ready(expired) => { + inner.connection.handle_timeout(expired); + inner.timer = None; + } + Poll::Pending => break, + } + } else if let Some(when) = inner.connection.poll_timeout() { + inner.timer = Some(Timer::at(when)); + } else { + break; + } + } + + while let Some(event) = inner.connection.poll_endpoint_events() { + inner.endpoint.send_endpoint_event(event); + } + + while let Some(event) = inner.connection.poll() { + match event { + Event::HandshakeDataReady => {} + Event::Connected => { + // Break here so that the noise upgrade can finish. + return Poll::Pending; + } + Event::ConnectionLost { reason } => { + tracing::debug!("connection lost because of {}", reason); + inner.substreams.clear(); + if let Some(waker) = inner.close_waker.take() { + waker.wake(); + } + return Poll::Ready(Err(QuicMuxerError::ConnectionLost { reason })); + } + Event::Stream(StreamEvent::Opened { dir: Dir::Bi }) => { + // handled at end. + } + Event::Stream(StreamEvent::Readable { id }) => { + tracing::trace!("stream readable {}", id); + if let Some(substream) = inner.substreams.get_mut(&id) { + if let Some(waker) = substream.read_waker.take() { + waker.wake(); + } + } + } + Event::Stream(StreamEvent::Writable { id }) => { + tracing::trace!("stream writable {}", id); + if let Some(substream) = inner.substreams.get_mut(&id) { + if let Some(waker) = substream.write_waker.take() { + waker.wake(); + } + } + } + Event::Stream(StreamEvent::Finished { id }) => { + tracing::trace!("stream finished {}", id); + if let Some(substream) = inner.substreams.get_mut(&id) { + if let Some(waker) = substream.read_waker.take() { + waker.wake(); + } + if let Some(waker) = substream.write_waker.take() { + waker.wake(); + } + } + } + Event::Stream(StreamEvent::Stopped { id, error_code }) => { + tracing::debug!("substream {} stopped with error {}", id, error_code); + inner.substreams.remove(&id); + return Poll::Ready(Err(QuicMuxerError::StreamStopped { id, error_code })); + } + Event::Stream(StreamEvent::Available { dir: Dir::Bi }) => { + tracing::trace!("stream available"); + if let Some(waker) = inner.pending_substreams.pop_front() { + waker.wake(); + } + } + Event::Stream(StreamEvent::Opened { dir: Dir::Uni }) + | Event::Stream(StreamEvent::Available { dir: Dir::Uni }) + | Event::DatagramReceived => { + // We don't use datagrams or unidirectional streams. If these events + // happen, it is by some code not compatible with libp2p-quic. + inner + .connection + .close(Instant::now(), From::from(0u32), Default::default()); + return Poll::Ready(Err(QuicMuxerError::ProtocolViolation)); + } + } + } + + // TODO quinn doesn't support `StreamMuxerEvent::AddressChange`. + + if let Some(id) = inner.connection.streams().accept(Dir::Bi) { + inner.substreams.insert(id, Default::default()); + tracing::trace!("opened incoming substream {}", id); + return Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(id))); + } + + if inner.substreams.is_empty() { + if let Some(waker) = inner.close_waker.take() { + waker.wake(); + } + } + + inner.waker = Some(cx.waker().clone()); + Poll::Pending + } + + fn open_outbound(&self) -> Self::OutboundSubstream {} + + fn poll_outbound( + &self, + cx: &mut Context, + _: &mut Self::OutboundSubstream, + ) -> Poll> { + let mut inner = self.inner.lock(); + if let Some(id) = inner.connection.streams().open(Dir::Bi) { + tracing::trace!("opened outgoing substream {}", id); + inner.substreams.insert(id, Default::default()); + if let Some(waker) = inner.pending_substreams.pop_front() { + waker.wake(); + } + Poll::Ready(Ok(id)) + } else { + inner.pending_substreams.push_back(cx.waker().clone()); + Poll::Pending + } + } + + fn destroy_outbound(&self, _: Self::OutboundSubstream) {} + + fn read_substream( + &self, + cx: &mut Context, + id: &mut Self::Substream, + mut buf: &mut [u8], + ) -> Poll> { + let mut inner = self.inner.lock(); + let mut stream = inner.connection.recv_stream(*id); + let mut chunks = match stream.read(true) { + Ok(chunks) => chunks, + Err(ReadableError::UnknownStream) => { + return Poll::Ready(Err(QuicMuxerError::UnknownStream { id: *id })) + } + Err(ReadableError::IllegalOrderedRead) => { + panic!("Illegal ordered read can only happen if `stream.read(false)` is used."); + } + }; + let mut bytes = 0; + let mut pending = false; + loop { + if buf.is_empty() { + break; + } + match chunks.next(buf.len()) { + Ok(Some(chunk)) => { + buf.write_all(&chunk.bytes).expect("enough buffer space"); + bytes += chunk.bytes.len(); + } + Ok(None) => break, + Err(ReadError::Reset(error_code)) => { + tracing::debug!("substream {} was reset with error code {}", id, error_code); + bytes = 0; + break; + } + Err(ReadError::Blocked) => { + pending = true; + break; + } + } + } + if chunks.finalize().should_transmit() { + if let Some(waker) = inner.waker.take() { + waker.wake(); + } + } + let substream = inner.substreams.get_mut(&id).unwrap(); + if pending && bytes == 0 { + substream.read_waker = Some(cx.waker().clone()); + Poll::Pending + } else { + Poll::Ready(Ok(bytes)) + } + } + + fn write_substream( + &self, + cx: &mut Context, + id: &mut Self::Substream, + buf: &[u8], + ) -> Poll> { + let mut inner = self.inner.lock(); + match inner.connection.send_stream(*id).write(buf) { + Ok(bytes) => Poll::Ready(Ok(bytes)), + Err(WriteError::Blocked) => { + let mut substream = inner.substreams.get_mut(id).unwrap(); + substream.write_waker = Some(cx.waker().clone()); + Poll::Pending + } + Err(WriteError::Stopped(_)) => Poll::Ready(Ok(0)), + Err(WriteError::UnknownStream) => { + Poll::Ready(Err(QuicMuxerError::UnknownStream { id: *id })) + } + } + } + + fn shutdown_substream( + &self, + _: &mut Context, + id: &mut Self::Substream, + ) -> Poll> { + tracing::trace!("closing substream {}", id); + // closes the write end of the substream without waiting for the remote to receive the + // event. use flush substream to wait for the remote to receive the event. + let mut inner = self.inner.lock(); + match inner.connection.send_stream(*id).finish() { + Ok(()) => Poll::Ready(Ok(())), + Err(FinishError::Stopped(_)) => Poll::Ready(Ok(())), + Err(FinishError::UnknownStream) => { + Poll::Ready(Err(QuicMuxerError::UnknownStream { id: *id })) + } + } + } + + fn destroy_substream(&self, id: Self::Substream) { + tracing::trace!("destroying substream {}", id); + let mut inner = self.inner.lock(); + inner.substreams.remove(&id); + let mut stream = inner.connection.recv_stream(id); + let should_transmit = if let Ok(mut chunks) = stream.read(true) { + while let Ok(Some(_)) = chunks.next(usize::MAX) {} + chunks.finalize().should_transmit() + } else { + false + }; + if should_transmit { + if let Some(waker) = inner.waker.take() { + waker.wake(); + } + } + } + + fn flush_substream( + &self, + _cx: &mut Context, + _id: &mut Self::Substream, + ) -> Poll> { + // quinn doesn't support flushing, calling close will flush all substreams. + Poll::Ready(Ok(())) + } + + fn flush_all(&self, _cx: &mut Context) -> Poll> { + // quinn doesn't support flushing, calling close will flush all substreams. + Poll::Ready(Ok(())) + } + + fn close(&self, cx: &mut Context) -> Poll> { + tracing::trace!("closing muxer"); + let mut inner = self.inner.lock(); + if inner.substreams.is_empty() { + return Poll::Ready(Ok(())); + } + inner.close_waker = Some(cx.waker().clone()); + let inner = &mut *inner; + for id in inner.substreams.keys() { + let _ = inner.connection.send_stream(*id).finish(); + } + Poll::Pending + } +} + +#[derive(Debug, Error)] +pub enum QuicMuxerError { + #[error("connection was lost because of {reason}")] + ConnectionLost { reason: ConnectionError }, + #[error("unsupported quic feature used")] + ProtocolViolation, + #[error("stream {id} stopped with error {error_code}")] + StreamStopped { id: StreamId, error_code: VarInt }, + #[error("unknown stream {id}")] + UnknownStream { id: StreamId }, +} + +impl From for std::io::Error { + fn from(err: QuicMuxerError) -> Self { + std::io::Error::new(std::io::ErrorKind::Other, err) + } +} diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs new file mode 100644 index 00000000000..ffcee2b780e --- /dev/null +++ b/transports/quic/src/transport.rs @@ -0,0 +1,292 @@ +use crate::endpoint::{EndpointConfig, TransportChannel}; +use crate::muxer::QuicMuxer; +use crate::{PublicKey, QuicConfig, QuicError}; +use futures::channel::oneshot; +use futures::future::{ready, Ready}; +use futures::prelude::*; +use if_watch::{IfEvent, IfWatcher}; +use libp2p_core::multiaddr::{Multiaddr, Protocol}; +use libp2p_core::muxing::StreamMuxerBox; +use libp2p_core::transport::{Boxed, ListenerEvent, Transport, TransportError}; +use libp2p_core::PeerId; +use parking_lot::Mutex; +use std::net::{IpAddr, SocketAddr}; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; +use udp_socket::SocketType; + +#[derive(Clone)] +pub struct QuicTransport { + inner: Arc>, +} + +impl QuicTransport { + /// Creates a new quic transport. + pub async fn new( + config: QuicConfig, + addr: Multiaddr, + ) -> Result> { + let socket_addr = multiaddr_to_socketaddr(&addr) + .map_err(|_| TransportError::MultiaddrNotSupported(addr.clone()))? + .0; + let addresses = if socket_addr.ip().is_unspecified() { + let watcher = IfWatcher::new() + .await + .map_err(|err| TransportError::Other(err.into()))?; + Addresses::Unspecified(watcher) + } else { + Addresses::Ip(Some(socket_addr.ip())) + }; + let endpoint = EndpointConfig::new(config, socket_addr).map_err(TransportError::Other)?; + Ok(Self { + inner: Arc::new(Mutex::new(QuicTransportInner { + channel: endpoint.spawn(), + addresses, + })), + }) + } + + /// Creates a boxed libp2p transport. + pub fn boxed(self) -> Boxed<(PeerId, StreamMuxerBox)> { + Transport::map(self, |(peer_id, muxer), _| { + (peer_id, StreamMuxerBox::new(muxer)) + }) + .boxed() + } +} + +impl std::fmt::Debug for QuicTransport { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("QuicTransport").finish() + } +} + +struct QuicTransportInner { + channel: TransportChannel, + addresses: Addresses, +} + +enum Addresses { + Unspecified(IfWatcher), + Ip(Option), +} + +impl Transport for QuicTransport { + type Output = (PeerId, QuicMuxer); + type Error = QuicError; + type Listener = Self; + type ListenerUpgrade = Ready>; + type Dial = QuicDial; + + fn listen_on(self, addr: Multiaddr) -> Result> { + multiaddr_to_socketaddr(&addr).map_err(|_| TransportError::MultiaddrNotSupported(addr))?; + Ok(self) + } + + fn dial(self, addr: Multiaddr) -> Result> { + let (socket_addr, public_key) = + if let Ok((socket_addr, Some(public_key))) = multiaddr_to_socketaddr(&addr) { + (socket_addr, public_key) + } else { + tracing::debug!("invalid multiaddr"); + return Err(TransportError::MultiaddrNotSupported(addr.clone())); + }; + if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { + tracing::debug!("invalid multiaddr"); + return Err(TransportError::MultiaddrNotSupported(addr)); + } + tracing::debug!("dialing {}", socket_addr); + let rx = self.inner.lock().channel.dial(socket_addr, public_key); + Ok(QuicDial { rx }) + } + + fn address_translation(&self, _listen: &Multiaddr, observed: &Multiaddr) -> Option { + Some(observed.clone()) + } +} + +impl Stream for QuicTransport { + type Item = + Result>, QuicError>, QuicError>; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let mut inner = self.inner.lock(); + match &mut inner.addresses { + Addresses::Ip(ip) => { + if let Some(ip) = ip.take() { + let addr = socketaddr_to_multiaddr(&SocketAddr::new(ip, inner.channel.port())); + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(addr)))); + } + } + Addresses::Unspecified(watcher) => match Pin::new(watcher).poll(cx) { + Poll::Ready(Ok(IfEvent::Up(net))) => { + if inner.channel.ty() == SocketType::Ipv4 && net.addr().is_ipv4() + || inner.channel.ty() != SocketType::Ipv4 && net.addr().is_ipv6() + { + let addr = socketaddr_to_multiaddr(&SocketAddr::new( + net.addr(), + inner.channel.port(), + )); + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(addr)))); + } + } + Poll::Ready(Ok(IfEvent::Down(net))) => { + if inner.channel.ty() == SocketType::Ipv4 && net.addr().is_ipv4() + || inner.channel.ty() != SocketType::Ipv4 && net.addr().is_ipv6() + { + let addr = socketaddr_to_multiaddr(&SocketAddr::new( + net.addr(), + inner.channel.port(), + )); + return Poll::Ready(Some(Ok(ListenerEvent::AddressExpired(addr)))); + } + } + Poll::Ready(Err(err)) => return Poll::Ready(Some(Err(err.into()))), + Poll::Pending => {} + }, + } + match inner.channel.poll_incoming(cx) { + Poll::Ready(Some(Ok(muxer))) => Poll::Ready(Some(Ok(ListenerEvent::Upgrade { + local_addr: muxer.local_addr(), + remote_addr: muxer.remote_addr(), + upgrade: ready(Ok((muxer.peer_id(), muxer))), + }))), + Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(err))), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + } + } +} + +#[allow(clippy::large_enum_variant)] +pub struct QuicDial { + rx: oneshot::Receiver>, +} + +impl Future for QuicDial { + type Output = Result<(PeerId, QuicMuxer), QuicError>; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { + loop { + match Pin::new(&mut self.rx).poll(cx) { + Poll::Ready(Ok(Ok(muxer))) => { + return Poll::Ready(Ok((muxer.peer_id(), muxer))); + } + Poll::Ready(Ok(Err(err))) => return Poll::Ready(Err(err)), + Poll::Ready(Err(_)) => panic!("endpoint crashed"), + Poll::Pending => return Poll::Pending, + } + } + } +} + +/// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns an error if the format +/// of the multiaddr is wrong. +fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result<(SocketAddr, Option), ()> { + let mut iter = addr.iter().peekable(); + let proto1 = iter.next().ok_or(())?; + let proto2 = iter.next().ok_or(())?; + let proto3 = iter.next().ok_or(())?; + + let peer_id = if let Some(Protocol::P2p(peer_id)) = iter.peek() { + if peer_id.code() != multihash::Code::Identity.into() { + return Err(()); + } + let public_key = + libp2p_core::PublicKey::from_protobuf_encoding(peer_id.digest()).map_err(|_| ())?; + let public_key = if let libp2p_core::PublicKey::Ed25519(public_key) = public_key { + public_key.encode() + } else { + return Err(()); + }; + let public_key = PublicKey::from_bytes(&public_key).map_err(|_| ())?; + iter.next(); + Some(public_key) + } else { + None + }; + + if iter.next().is_some() { + return Err(()); + } + + match (proto1, proto2, proto3) { + (Protocol::Ip4(ip), Protocol::Udp(port), Protocol::Quic) => { + Ok((SocketAddr::new(ip.into(), port), peer_id)) + } + (Protocol::Ip6(ip), Protocol::Udp(port), Protocol::Quic) => { + Ok((SocketAddr::new(ip.into(), port), peer_id)) + } + _ => Err(()), + } +} + +/// Turns an IP address and port into the corresponding QUIC multiaddr. +pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { + Multiaddr::empty() + .with(socket_addr.ip().into()) + .with(Protocol::Udp(socket_addr.port())) + .with(Protocol::Quic) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn multiaddr_to_udp_conversion() { + use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; + + assert!( + multiaddr_to_socketaddr(&"/ip4/127.0.0.1/udp/1234".parse::().unwrap()) + .is_err() + ); + + assert_eq!( + multiaddr_to_socketaddr( + &"/ip4/127.0.0.1/udp/12345/quic" + .parse::() + .unwrap() + ), + Ok(( + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 12345,), + None + )) + ); + assert_eq!( + multiaddr_to_socketaddr( + &"/ip4/255.255.255.255/udp/8080/quic" + .parse::() + .unwrap() + ), + Ok(( + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(255, 255, 255, 255)), 8080,), + None + )) + ); + assert_eq!( + multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), + Ok(( + SocketAddr::new(IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), 12345,), + None + )) + ); + assert_eq!( + multiaddr_to_socketaddr( + &"/ip6/ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/udp/8080/quic" + .parse::() + .unwrap() + ), + Ok(( + SocketAddr::new( + IpAddr::V6(Ipv6Addr::new( + 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, + )), + 8080, + ), + None + )) + ); + } +} diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs new file mode 100644 index 00000000000..26d9933d7fa --- /dev/null +++ b/transports/quic/tests/smoke.rs @@ -0,0 +1,241 @@ +use anyhow::Result; +use async_trait::async_trait; +use futures::future::FutureExt; +use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; +use futures::stream::StreamExt; +use libp2p::core::upgrade; +use libp2p::multiaddr::Protocol; +use libp2p::quic::{Keypair, QuicConfig, ToLibp2p}; +use libp2p::request_response::{ + ProtocolName, ProtocolSupport, RequestResponse, RequestResponseCodec, RequestResponseConfig, + RequestResponseEvent, RequestResponseMessage, +}; +use libp2p::swarm::{Swarm, SwarmEvent}; +use rand::RngCore; +use std::{io, iter}; + +async fn create_swarm(keylog: bool) -> Result>> { + let keypair = Keypair::generate(&mut rand_core::OsRng {}); + let peer_id = keypair.to_peer_id(); + let mut transport = QuicConfig::new(keypair); + if keylog { + transport.enable_keylogger(); + } + let transport = transport + .listen_on("/ip4/127.0.0.1/udp/0/quic".parse()?) + .await? + .boxed(); + + let protocols = iter::once((PingProtocol(), ProtocolSupport::Full)); + let cfg = RequestResponseConfig::default(); + let behaviour = RequestResponse::new(PingCodec(), protocols, cfg); + tracing::info!("{}", peer_id); + Ok(Swarm::new(transport, behaviour, peer_id)) +} + +#[async_std::test] +async fn smoke() -> Result<()> { + tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .try_init() + .ok(); + log_panics::init(); + let mut rng = rand::thread_rng(); + + let mut a = create_swarm(true).await?; + let mut b = create_swarm(false).await?; + + Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; + + let mut addr = match a.next().await { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + }; + addr.push(Protocol::P2p((*a.local_peer_id()).into())); + + let mut data = vec![0; 4096 * 10]; + rng.fill_bytes(&mut data); + + b.behaviour_mut() + .add_address(&Swarm::local_peer_id(&a), addr); + b.behaviour_mut() + .send_request(&Swarm::local_peer_id(&a), Ping(data.clone())); + + match b.next().await { + Some(SwarmEvent::Dialing(_)) => {} + e => panic!("{:?}", e), + } + + match a.next().await { + Some(SwarmEvent::IncomingConnection { .. }) => {} + e => panic!("{:?}", e), + }; + + match b.next().await { + Some(SwarmEvent::ConnectionEstablished { .. }) => {} + e => panic!("{:?}", e), + }; + + match a.next().await { + Some(SwarmEvent::ConnectionEstablished { .. }) => {} + e => panic!("{:?}", e), + }; + + assert!(b.next().now_or_never().is_none()); + + match a.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { + message: + RequestResponseMessage::Request { + request: Ping(ping), + channel, + .. + }, + .. + })) => { + a.behaviour_mut() + .send_response(channel, Pong(ping)) + .unwrap(); + } + e => panic!("{:?}", e), + } + + match a.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::ResponseSent { .. })) => {} + e => panic!("{:?}", e), + } + + match b.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { + message: + RequestResponseMessage::Response { + response: Pong(pong), + .. + }, + .. + })) => assert_eq!(data, pong), + e => panic!("{:?}", e), + } + + a.behaviour_mut().send_request( + &Swarm::local_peer_id(&b), + Ping(b"another substream".to_vec()), + ); + + assert!(a.next().now_or_never().is_none()); + + match b.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { + message: + RequestResponseMessage::Request { + request: Ping(data), + channel, + .. + }, + .. + })) => { + b.behaviour_mut() + .send_response(channel, Pong(data)) + .unwrap(); + } + e => panic!("{:?}", e), + } + + match b.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::ResponseSent { .. })) => {} + e => panic!("{:?}", e), + } + + match a.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { + message: + RequestResponseMessage::Response { + response: Pong(data), + .. + }, + .. + })) => assert_eq!(data, b"another substream".to_vec()), + e => panic!("{:?}", e), + } + + Ok(()) +} + +#[derive(Debug, Clone)] +struct PingProtocol(); + +#[derive(Clone)] +struct PingCodec(); + +#[derive(Debug, Clone, PartialEq, Eq)] +struct Ping(Vec); + +#[derive(Debug, Clone, PartialEq, Eq)] +struct Pong(Vec); + +impl ProtocolName for PingProtocol { + fn protocol_name(&self) -> &[u8] { + "/ping/1".as_bytes() + } +} + +#[async_trait] +impl RequestResponseCodec for PingCodec { + type Protocol = PingProtocol; + type Request = Ping; + type Response = Pong; + + async fn read_request(&mut self, _: &PingProtocol, io: &mut T) -> io::Result + where + T: AsyncRead + Unpin + Send, + { + upgrade::read_length_prefixed(io, 4096 * 10) + .map(|res| match res { + Err(e) => Err(io::Error::new(io::ErrorKind::InvalidData, e)), + Ok(vec) if vec.is_empty() => Err(io::ErrorKind::UnexpectedEof.into()), + Ok(vec) => Ok(Ping(vec)), + }) + .await + } + + async fn read_response(&mut self, _: &PingProtocol, io: &mut T) -> io::Result + where + T: AsyncRead + Unpin + Send, + { + upgrade::read_length_prefixed(io, 4096 * 10) + .map(|res| match res { + Err(e) => Err(io::Error::new(io::ErrorKind::InvalidData, e)), + Ok(vec) if vec.is_empty() => Err(io::ErrorKind::UnexpectedEof.into()), + Ok(vec) => Ok(Pong(vec)), + }) + .await + } + + async fn write_request( + &mut self, + _: &PingProtocol, + io: &mut T, + Ping(data): Ping, + ) -> io::Result<()> + where + T: AsyncWrite + Unpin + Send, + { + upgrade::write_length_prefixed(io, data).await?; + io.close().await?; + Ok(()) + } + + async fn write_response( + &mut self, + _: &PingProtocol, + io: &mut T, + Pong(data): Pong, + ) -> io::Result<()> + where + T: AsyncWrite + Unpin + Send, + { + upgrade::write_length_prefixed(io, data).await?; + io.close().await?; + Ok(()) + } +} From 7c658064fdb001aa376cf2f5553c993c5911bf3c Mon Sep 17 00:00:00 2001 From: David Craven Date: Sat, 17 Jul 2021 13:02:24 +0200 Subject: [PATCH 02/14] Disable build on wasm. --- src/lib.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/lib.rs b/src/lib.rs index f384e9d021b..e41adb77d85 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -91,6 +91,7 @@ pub use libp2p_ping as ping; pub use libp2p_plaintext as plaintext; #[cfg(feature = "quic")] #[cfg_attr(docsrs, doc(cfg(feature = "quic")))] +#[cfg(not(any(target_os = "emscripten", target_os = "wasi", target_os = "unknown")))] #[doc(inline)] pub use libp2p_quic as quic; #[doc(inline)] From eb3bc68bbca7cbe70e24437166ddaf2901b5bd5c Mon Sep 17 00:00:00 2001 From: David Craven Date: Mon, 26 Jul 2021 12:17:26 +0200 Subject: [PATCH 03/14] Update to v0.6.0 --- Cargo.toml | 2 +- transports/quic/Cargo.toml | 26 +- transports/quic/src/crypto.rs | 147 ++++++++++ transports/quic/src/endpoint.rs | 353 +++++++++++++++---------- transports/quic/src/lib.rs | 38 ++- transports/quic/src/muxer.rs | 58 ++-- transports/quic/src/tls/certificate.rs | 85 ++++++ transports/quic/src/tls/mod.rs | 89 +++++++ transports/quic/src/tls/verifier.rs | 238 +++++++++++++++++ transports/quic/src/transport.rs | 116 ++++++-- transports/quic/tests/smoke.rs | 36 ++- 11 files changed, 961 insertions(+), 227 deletions(-) create mode 100644 transports/quic/src/crypto.rs create mode 100644 transports/quic/src/tls/certificate.rs create mode 100644 transports/quic/src/tls/mod.rs create mode 100644 transports/quic/src/tls/verifier.rs diff --git a/Cargo.toml b/Cargo.toml index e5d32316c2e..e1b1b0d92ab 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -93,7 +93,7 @@ wasm-timer = "0.2.4" libp2p-deflate = { version = "0.29.0", path = "transports/deflate", optional = true } libp2p-dns = { version = "0.29.0", path = "transports/dns", optional = true, default-features = false } libp2p-mdns = { version = "0.31.0", path = "protocols/mdns", optional = true } -libp2p-quic = { version = "0.5.0", path = "transports/quic", optional = true } +libp2p-quic = { version = "0.6.0", path = "transports/quic", optional = true } libp2p-tcp = { version = "0.29.0", path = "transports/tcp", default-features = false, optional = true } libp2p-websocket = { version = "0.30.0", path = "transports/websocket", optional = true } diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 528e2bc9af4..b7a526d096f 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -1,34 +1,46 @@ [package] name = "libp2p-quic" -version = "0.5.0" -authors = ["David Craven "] +version = "0.6.0" +authors = ["David Craven ", "Parity Technologies "] edition = "2018" description = "libp2p-quic is a noise based quic implementation for rust-libp2p." -repository = "https://github.com/libp2p/rust-libp2p" -license = "MIT" +repository = "https://github.com/ipfs-rust/libp2p-quic" +license = "ISC" + +[features] +noise = ["quinn-noise"] +tls = ["barebones-x509", "quinn-proto/tls-rustls", "rcgen", "ring", "rustls", "untrusted", "webpki", "yasna"] [dependencies] anyhow = "1.0.41" async-global-executor = "2.0.2" async-io = "1.6.0" +barebones-x509 = { version = "0.5.0", optional = true, features = ["webpki", "rustls", "std"] } bytes = "1.0.1" +ed25519-dalek = "1.0.1" fnv = "1.0.7" futures = "0.3.15" if-watch = "0.2.2" -libp2p-core = { path = "../../core", default-features = false } +libp2p-core = { version = "0.29.0", path = "../../core" } multihash = { version = "0.14.0", default-features = false } parking_lot = "0.11.1" +quinn-noise = { version = "0.3.0", optional = true } quinn-proto = { version = "0.7.3", default-features = false } -quinn-noise = "0.2.1" rand_core = "0.5.1" +rcgen = { version = "0.8.11", optional = true } +ring = { version = "0.16.20", optional = true } +rustls = { version = "0.19.1", optional = true, features = ["dangerous_configuration"] } thiserror = "1.0.26" tracing = "0.1.26" udp-socket = "0.1.5" +untrusted = { version = "0.7.1", optional = true } +webpki = { version = "0.21.4", optional = true, features = ["std"] } +yasna = { version = "0.4.0", optional = true } [dev-dependencies] async-std = { version = "1.9.0", features = ["attributes"] } async-trait = "0.1.50" -libp2p = { path = "../..", default-features = false, features = ["request-response"] } +libp2p = { version = "0.39.1", default-features = false, features = ["request-response"], path = "../.." } log-panics = "2.0.0" rand = "0.8.4" tracing-subscriber = "0.2.19" diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs new file mode 100644 index 00000000000..2d95e7c8eb6 --- /dev/null +++ b/transports/quic/src/crypto.rs @@ -0,0 +1,147 @@ +use ed25519_dalek::{Keypair, PublicKey}; +use libp2p_core::PeerId; +use quinn_proto::crypto::Session; +use quinn_proto::TransportConfig; +use std::sync::Arc; + +pub struct CryptoConfig { + pub keypair: Keypair, + pub psk: Option<[u8; 32]>, + pub keylogger: Option, + pub transport: Arc, +} + +#[cfg(feature = "noise")] +impl CryptoConfig { + fn clone_keypair(&self) -> Keypair { + Keypair::from_bytes(&self.keypair.to_bytes()).expect("serde works") + } +} + +pub trait Crypto: std::fmt::Debug + Clone + 'static { + type Session: Session + Unpin; + type Keylogger: Send + Sync; + + fn new_server_config( + config: &Arc>, + ) -> ::ServerConfig; + fn new_client_config( + config: &Arc>, + remote_public: PublicKey, + ) -> ::ClientConfig; + fn supported_quic_versions() -> Vec; + fn default_quic_version() -> u32; + fn peer_id(session: &Self::Session) -> Option; + fn keylogger() -> Self::Keylogger; +} + +#[cfg(feature = "noise")] +#[derive(Clone, Copy, Debug)] +pub struct NoiseCrypto; + +#[cfg(feature = "noise")] +impl Crypto for NoiseCrypto { + type Session = quinn_noise::NoiseSession; + type Keylogger = Arc; + + fn new_server_config( + config: &Arc>, + ) -> ::ServerConfig { + Arc::new( + quinn_noise::NoiseServerConfig { + keypair: config.clone_keypair(), + psk: config.psk, + keylogger: config.keylogger.clone(), + supported_protocols: vec![b"libp2p".to_vec()], + } + .into(), + ) + } + + fn new_client_config( + config: &Arc>, + remote_public_key: PublicKey, + ) -> ::ClientConfig { + quinn_noise::NoiseClientConfig { + keypair: config.clone_keypair(), + psk: config.psk, + alpn: b"libp2p".to_vec(), + remote_public_key, + keylogger: config.keylogger.clone(), + } + .into() + } + + fn supported_quic_versions() -> Vec { + quinn_noise::SUPPORTED_QUIC_VERSIONS.to_vec() + } + + fn default_quic_version() -> u32 { + quinn_noise::DEFAULT_QUIC_VERSION + } + + fn peer_id(session: &Self::Session) -> Option { + use crate::ToLibp2p; + Some(session.peer_identity()?.to_peer_id()) + } + + fn keylogger() -> Self::Keylogger { + Arc::new(quinn_noise::KeyLogFile::new()) + } +} + +#[cfg(feature = "tls")] +#[derive(Clone, Copy, Debug)] +pub struct TlsCrypto; + +#[cfg(feature = "tls")] +impl Crypto for TlsCrypto { + type Session = quinn_proto::crypto::rustls::TlsSession; + type Keylogger = Arc; + + fn new_server_config( + config: &Arc>, + ) -> ::ServerConfig { + assert!(config.psk.is_none(), "invalid config"); + use crate::ToLibp2p; + let (_, mut server) = + crate::tls::make_tls_config(&config.keypair.to_keypair()).expect("invalid config"); + if let Some(key_log) = config.keylogger.clone() { + server.key_log = key_log; + } + Arc::new(server) + } + + fn new_client_config( + config: &Arc>, + _remote_public: PublicKey, + ) -> ::ClientConfig { + assert!(config.psk.is_none(), "invalid config"); + use crate::ToLibp2p; + let (mut client, _) = + crate::tls::make_tls_config(&config.keypair.to_keypair()).expect("invalid config"); + if let Some(key_log) = config.keylogger.clone() { + client.key_log = key_log; + } + Arc::new(client) + } + + fn supported_quic_versions() -> Vec { + quinn_proto::DEFAULT_SUPPORTED_VERSIONS.to_vec() + } + + fn default_quic_version() -> u32 { + quinn_proto::DEFAULT_SUPPORTED_VERSIONS[0] + } + + fn peer_id(session: &Self::Session) -> Option { + let certificate = session.get_peer_certificates()?.into_iter().next()?; + Some(crate::tls::extract_peerid_or_panic( + quinn_proto::Certificate::from(certificate).as_der(), + )) + } + + fn keylogger() -> Self::Keylogger { + Arc::new(rustls::KeyLogFile::new()) + } +} diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 5c8a2fd6503..9d44cb21606 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -1,9 +1,11 @@ +use crate::crypto::{Crypto, CryptoConfig}; use crate::muxer::QuicMuxer; -use crate::{PublicKey, QuicConfig, QuicError}; +use crate::{QuicConfig, QuicError}; +use ed25519_dalek::PublicKey; use fnv::FnvHashMap; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; -use quinn_noise::{NoiseConfig, NoiseSession}; +use quinn_proto::crypto::Session; use quinn_proto::generic::{ClientConfig, ServerConfig}; use quinn_proto::{ ConnectionEvent, ConnectionHandle, DatagramEvent, EcnCodepoint, EndpointEvent, Transmit, @@ -20,7 +22,7 @@ use udp_socket::{RecvMeta, SocketType, UdpCapabilities, UdpSocket, BATCH_SIZE}; /// Message sent to the endpoint background task. #[derive(Debug)] -enum ToEndpoint { +enum ToEndpoint { /// Instructs the endpoint to start connecting to the given address. Dial { /// UDP address to connect to. @@ -28,7 +30,7 @@ enum ToEndpoint { /// The remotes public key. public_key: PublicKey, /// Channel to return the result of the dialing to. - tx: oneshot::Sender>, + tx: oneshot::Sender, QuicError>>, }, /// Sent by a `quinn_proto` connection when the endpoint needs to process an event generated /// by a connection. The event itself is opaque to us. @@ -41,19 +43,19 @@ enum ToEndpoint { } #[derive(Debug)] -pub struct TransportChannel { - tx: mpsc::UnboundedSender, - rx: mpsc::UnboundedReceiver>, +pub struct TransportChannel { + tx: mpsc::UnboundedSender>, + rx: mpsc::Receiver, QuicError>>, port: u16, ty: SocketType, } -impl TransportChannel { +impl TransportChannel { pub fn dial( &mut self, addr: SocketAddr, public_key: PublicKey, - ) -> oneshot::Receiver> { + ) -> oneshot::Receiver, QuicError>> { let (tx, rx) = oneshot::channel(); let msg = ToEndpoint::Dial { addr, @@ -67,7 +69,7 @@ impl TransportChannel { pub fn poll_incoming( &mut self, cx: &mut Context, - ) -> Poll>> { + ) -> Poll, QuicError>>> { Pin::new(&mut self.rx).poll_next(cx) } @@ -81,15 +83,15 @@ impl TransportChannel { } #[derive(Debug)] -pub struct ConnectionChannel { +pub struct ConnectionChannel { id: ConnectionHandle, - tx: mpsc::UnboundedSender, - rx: mpsc::UnboundedReceiver, + tx: mpsc::UnboundedSender>, + rx: mpsc::Receiver, port: u16, max_datagrams: usize, } -impl ConnectionChannel { +impl ConnectionChannel { pub fn poll_channel_events(&mut self, cx: &mut Context) -> Poll { match Pin::new(&mut self.rx).poll_next(cx) { Poll::Ready(Some(event)) => Poll::Ready(event), @@ -121,28 +123,24 @@ impl ConnectionChannel { } #[derive(Debug)] -struct EndpointChannel { - rx: mpsc::UnboundedReceiver, - tx: mpsc::UnboundedSender>, +struct EndpointChannel { + rx: mpsc::UnboundedReceiver>, + tx: mpsc::Sender, QuicError>>, port: u16, max_datagrams: usize, - connection_tx: mpsc::UnboundedSender, + connection_tx: mpsc::UnboundedSender>, } -impl EndpointChannel { - pub fn send_incoming(&mut self, muxer: QuicMuxer) { - self.tx.unbounded_send(Ok(muxer)).ok(); - } - - pub fn poll_next_event(&mut self, cx: &mut Context) -> Poll> { +impl EndpointChannel { + pub fn poll_next_event(&mut self, cx: &mut Context) -> Poll>> { Pin::new(&mut self.rx).poll_next(cx) } pub fn create_connection( &self, id: ConnectionHandle, - ) -> (ConnectionChannel, mpsc::UnboundedSender) { - let (tx, rx) = mpsc::unbounded(); + ) -> (ConnectionChannel, mpsc::Sender) { + let (tx, rx) = mpsc::channel(12); let channel = ConnectionChannel { id, tx: self.connection_tx.clone(), @@ -154,44 +152,37 @@ impl EndpointChannel { } } -type QuinnEndpointConfig = quinn_proto::generic::EndpointConfig; -type QuinnEndpoint = quinn_proto::generic::Endpoint; +type QuinnEndpointConfig = quinn_proto::generic::EndpointConfig; +type QuinnEndpoint = quinn_proto::generic::Endpoint; -pub struct EndpointConfig { +pub struct EndpointConfig { socket: UdpSocket, - endpoint: QuinnEndpoint, + endpoint: QuinnEndpoint, port: u16, - client_config: ClientConfig, + crypto_config: Arc>, capabilities: UdpCapabilities, } -impl EndpointConfig { - pub fn new(mut config: QuicConfig, addr: SocketAddr) -> Result { +impl EndpointConfig { + pub fn new(mut config: QuicConfig, addr: SocketAddr) -> Result { config.transport.max_concurrent_uni_streams(0)?; config.transport.datagram_receive_buffer_size(None); let transport = Arc::new(config.transport); - let noise_config = NoiseConfig { - keypair: Some(config.keypair), + let crypto_config = Arc::new(CryptoConfig { + keypair: config.keypair, psk: config.psk, - remote_public_key: None, keylogger: config.keylogger, - }; + transport: transport.clone(), + }); - let mut server_config = ServerConfig::::default(); - server_config.transport = transport.clone(); - server_config.crypto = Arc::new(noise_config.clone()); - - let client_config = ClientConfig:: { - transport, - crypto: noise_config, - }; + let mut server_config = ServerConfig::::default(); + server_config.transport = transport; + server_config.crypto = C::new_server_config(&crypto_config); let mut endpoint_config = QuinnEndpointConfig::default(); - endpoint_config.supported_versions( - quinn_noise::SUPPORTED_QUIC_VERSIONS.to_vec(), - quinn_noise::DEFAULT_QUIC_VERSION, - )?; + endpoint_config + .supported_versions(C::supported_quic_versions(), C::default_quic_version())?; let socket = UdpSocket::bind(addr)?; let port = socket.local_addr()?.port(); @@ -204,14 +195,19 @@ impl EndpointConfig { socket, endpoint, port, - client_config, + crypto_config, capabilities, }) } - pub fn spawn(self) -> TransportChannel { + pub fn spawn(self) -> TransportChannel + where + ::ClientConfig: Send + Unpin, + ::HeaderKey: Unpin, + ::PacketKey: Unpin, + { let (tx1, rx1) = mpsc::unbounded(); - let (tx2, rx2) = mpsc::unbounded(); + let (tx2, rx2) = mpsc::channel(1); let transport = TransportChannel { tx: tx1, rx: rx2, @@ -230,18 +226,20 @@ impl EndpointConfig { } } -struct Endpoint { - channel: EndpointChannel, - endpoint: QuinnEndpoint, +struct Endpoint { + channel: EndpointChannel, + endpoint: QuinnEndpoint, socket: UdpSocket, - client_config: ClientConfig, - connections: FnvHashMap>, + crypto_config: Arc>, + connections: FnvHashMap>, outgoing: VecDeque, recv_buf: Box<[u8]>, + incoming_slot: Option>, + event_slot: Option<(ConnectionHandle, ConnectionEvent)>, } -impl Endpoint { - pub fn new(channel: EndpointChannel, config: EndpointConfig) -> Self { +impl Endpoint { + pub fn new(channel: EndpointChannel, config: EndpointConfig) -> Self { let max_udp_payload_size = config .endpoint .config() @@ -252,10 +250,12 @@ impl Endpoint { channel, endpoint: config.endpoint, socket: config.socket, - client_config: config.client_config, + crypto_config: config.crypto_config, connections: Default::default(), outgoing: Default::default(), recv_buf, + incoming_slot: None, + event_slot: None, } } @@ -273,63 +273,120 @@ impl Endpoint { }; self.outgoing.push_back(transmit); } + + fn send_incoming(&mut self, muxer: QuicMuxer, cx: &mut Context) -> bool { + assert!(self.incoming_slot.is_none()); + match self.channel.tx.poll_ready(cx) { + Poll::Pending => { + self.incoming_slot = Some(muxer); + true + } + Poll::Ready(Ok(())) => { + self.channel.tx.try_send(Ok(muxer)).ok(); + false + } + Poll::Ready(_err) => false, + } + } + + fn send_event( + &mut self, + id: ConnectionHandle, + event: ConnectionEvent, + cx: &mut Context, + ) -> bool { + assert!(self.event_slot.is_none()); + let conn = self.connections.get_mut(&id).unwrap(); + match conn.poll_ready(cx) { + Poll::Pending => { + self.event_slot = Some((id, event)); + true + } + Poll::Ready(Ok(())) => { + conn.try_send(event).ok(); + false + } + Poll::Ready(_err) => false, + } + } } -impl Future for Endpoint { +impl Future for Endpoint +where + ::ClientConfig: Unpin, + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ type Output = (); fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { let me = Pin::into_inner(self); + if let Some(muxer) = me.incoming_slot.take() { + if !me.send_incoming(muxer, cx) { + tracing::info!("cleared incoming slot"); + } + } + + if let Some((id, event)) = me.event_slot.take() { + if !me.send_event(id, event, cx) { + tracing::info!("cleared event slot"); + } + } + while let Some(transmit) = me.endpoint.poll_transmit() { me.transmit(transmit); } - while let Poll::Ready(event) = me.channel.poll_next_event(cx) { - match event { - Some(ToEndpoint::Dial { - addr, - public_key, - tx, - }) => { - let mut client_config = me.client_config.clone(); - client_config.crypto.remote_public_key = Some(public_key); - let (id, connection) = - match me.endpoint.connect(client_config, addr, "server_name") { - Ok(c) => c, - Err(err) => { - tracing::error!("dial failure: {}", err); - let _ = tx.send(Err(err.into())); - continue; - } + if me.event_slot.is_none() { + while let Poll::Ready(event) = me.channel.poll_next_event(cx) { + match event { + Some(ToEndpoint::Dial { + addr, + public_key, + tx, + }) => { + let crypto = C::new_client_config(&me.crypto_config, public_key); + let client_config = ClientConfig { + transport: me.crypto_config.transport.clone(), + crypto, }; - let (channel, conn) = me.channel.create_connection(id); - me.connections.insert(id, conn); - let muxer = QuicMuxer::new(channel, connection); - tx.send(Ok(muxer)).ok(); - } - Some(ToEndpoint::ConnectionEvent { - connection_id, - event, - }) => { - let is_drained_event = event.is_drained(); - if is_drained_event { - me.connections.remove(&connection_id); + let (id, connection) = + match me.endpoint.connect(client_config, addr, "server_name") { + Ok(c) => c, + Err(err) => { + tracing::error!("dial failure: {}", err); + let _ = tx.send(Err(err.into())); + continue; + } + }; + let (channel, conn) = me.channel.create_connection(id); + me.connections.insert(id, conn); + let muxer = QuicMuxer::new(channel, connection); + tx.send(Ok(muxer)).ok(); } - if let Some(event) = me.endpoint.handle_event(connection_id, event) { - me.connections - .get_mut(&connection_id) - .unwrap() - .unbounded_send(event) - .ok(); + Some(ToEndpoint::ConnectionEvent { + connection_id, + event, + }) => { + let is_drained_event = event.is_drained(); + if is_drained_event { + me.connections.remove(&connection_id); + } + if let Some(event) = me.endpoint.handle_event(connection_id, event) { + if me.send_event(connection_id, event, cx) { + tracing::info!("filled event slot"); + break; + } + } + } + Some(ToEndpoint::Transmit(transmit)) => { + me.transmit(transmit); + } + None => { + me.endpoint.reject_new_connections(); + return Poll::Ready(()); } - } - Some(ToEndpoint::Transmit(transmit)) => { - me.transmit(transmit); - } - None => { - me.endpoint.reject_new_connections(); - return Poll::Ready(()); } } } @@ -345,50 +402,54 @@ impl Future for Endpoint { } } - let mut metas = [RecvMeta::default(); BATCH_SIZE]; - let mut iovs = MaybeUninit::<[IoSliceMut; BATCH_SIZE]>::uninit(); - me.recv_buf - .chunks_mut(me.recv_buf.len() / BATCH_SIZE) - .enumerate() - .for_each(|(i, buf)| unsafe { - iovs.as_mut_ptr() - .cast::() - .add(i) - .write(IoSliceMut::new(buf)); - }); - let mut iovs = unsafe { iovs.assume_init() }; - while let Poll::Ready(result) = me.socket.poll_recv(cx, &mut iovs, &mut metas) { - let n = match result { - Ok(n) => n, - Err(err) => { - tracing::error!("recv_from: {}", err); - continue; - } - }; - for i in 0..n { - let meta = &metas[i]; - let packet = From::from(&iovs[i][..meta.len]); - let ecn = meta - .ecn - .map(|ecn| EcnCodepoint::from_bits(ecn as u8)) - .unwrap_or_default(); - match me - .endpoint - .handle(Instant::now(), meta.source, meta.dst_ip, ecn, packet) - { - None => {} - Some((id, DatagramEvent::ConnectionEvent(event))) => { - me.connections - .get_mut(&id) - .unwrap() - .unbounded_send(event) - .ok(); + if me.event_slot.is_none() && me.incoming_slot.is_none() { + let mut metas = [RecvMeta::default(); BATCH_SIZE]; + let mut iovs = MaybeUninit::<[IoSliceMut; BATCH_SIZE]>::uninit(); + me.recv_buf + .chunks_mut(me.recv_buf.len() / BATCH_SIZE) + .enumerate() + .for_each(|(i, buf)| unsafe { + iovs.as_mut_ptr() + .cast::() + .add(i) + .write(IoSliceMut::new(buf)); + }); + let mut iovs = unsafe { iovs.assume_init() }; + while let Poll::Ready(result) = me.socket.poll_recv(cx, &mut iovs, &mut metas) { + let n = match result { + Ok(n) => n, + Err(err) => { + tracing::error!("recv_from: {}", err); + continue; } - Some((id, DatagramEvent::NewConnection(connection))) => { - let (channel, tx) = me.channel.create_connection(id); - me.connections.insert(id, tx); - let muxer = QuicMuxer::new(channel, connection); - let _ = me.channel.send_incoming(muxer); + }; + for i in 0..n { + let meta = &metas[i]; + let packet = From::from(&iovs[i][..meta.len]); + let ecn = meta + .ecn + .map(|ecn| EcnCodepoint::from_bits(ecn as u8)) + .unwrap_or_default(); + match me + .endpoint + .handle(Instant::now(), meta.source, meta.dst_ip, ecn, packet) + { + None => {} + Some((id, DatagramEvent::ConnectionEvent(event))) => { + if me.send_event(id, event, cx) { + tracing::info!("filled event slot"); + break; + } + } + Some((id, DatagramEvent::NewConnection(connection))) => { + let (channel, tx) = me.channel.create_connection(id); + me.connections.insert(id, tx); + let muxer = QuicMuxer::new(channel, connection); + if me.send_incoming(muxer, cx) { + tracing::info!("filled incoming slot"); + break; + } + } } } } diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 254d78631d2..622e25ac2b6 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -1,16 +1,25 @@ +mod crypto; mod endpoint; mod muxer; +#[cfg(feature = "tls")] +mod tls; mod transport; +pub use crate::crypto::Crypto; +#[cfg(feature = "noise")] +pub use crate::crypto::NoiseCrypto; +#[cfg(feature = "tls")] +pub use crate::crypto::TlsCrypto; pub use crate::muxer::{QuicMuxer, QuicMuxerError}; pub use crate::transport::{QuicDial, QuicTransport}; -pub use quinn_noise::{KeyLog, KeyLogFile, Keypair, PublicKey, SecretKey}; +pub use ed25519_dalek::{Keypair, PublicKey, SecretKey}; +#[cfg(feature = "noise")] +pub use quinn_noise::{KeyLog, KeyLogFile}; pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; -use libp2p_core::identity; use libp2p_core::transport::TransportError; -use libp2p_core::{Multiaddr, PeerId}; -use std::sync::Arc; +use libp2p_core::{identity, Multiaddr, PeerId}; +use quinn_proto::crypto::Session; use thiserror::Error; pub fn generate_keypair() -> Keypair { @@ -18,14 +27,14 @@ pub fn generate_keypair() -> Keypair { } /// Quic configuration. -pub struct QuicConfig { +pub struct QuicConfig { pub keypair: Keypair, pub psk: Option<[u8; 32]>, pub transport: TransportConfig, - pub keylogger: Option>, + pub keylogger: Option, } -impl Default for QuicConfig { +impl Default for QuicConfig { fn default() -> Self { Self { keypair: Keypair::generate(&mut rand_core::OsRng {}), @@ -36,7 +45,7 @@ impl Default for QuicConfig { } } -impl std::fmt::Debug for QuicConfig { +impl std::fmt::Debug for QuicConfig { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { f.debug_struct("QuicConfig") .field("keypair", &self.keypair.public) @@ -46,7 +55,12 @@ impl std::fmt::Debug for QuicConfig { } } -impl QuicConfig { +impl QuicConfig +where + ::ClientConfig: Send + Unpin, + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ /// Creates a new config from a keypair. pub fn new(keypair: Keypair) -> Self { Self { @@ -57,7 +71,7 @@ impl QuicConfig { /// Enable keylogging. pub fn enable_keylogger(&mut self) -> &mut Self { - self.keylogger = Some(Arc::new(KeyLogFile::new())); + self.keylogger = Some(C::keylogger()); self } @@ -65,7 +79,7 @@ impl QuicConfig { pub async fn listen_on( self, addr: Multiaddr, - ) -> Result> { + ) -> Result, TransportError> { QuicTransport::new(self, addr).await } } @@ -112,6 +126,6 @@ impl ToLibp2p for PublicKey { fn to_public(&self) -> identity::PublicKey { let public_key = self.to_bytes(); let public_key = identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); - identity::PublicKey::Ed25519(public_key.into()) + identity::PublicKey::Ed25519(public_key) } } diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index e2d0f2b6d55..0851b3d0de5 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -1,13 +1,11 @@ +use crate::crypto::Crypto; use crate::endpoint::ConnectionChannel; -use crate::ToLibp2p; use async_io::Timer; use fnv::FnvHashMap; use futures::prelude::*; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use libp2p_core::{Multiaddr, PeerId}; use parking_lot::Mutex; -use quinn_noise::NoiseSession; -use quinn_proto::crypto::Session; use quinn_proto::generic::Connection; use quinn_proto::{ ConnectionError, Dir, Event, FinishError, ReadError, ReadableError, StreamEvent, StreamId, @@ -22,18 +20,24 @@ use std::time::Instant; use thiserror::Error; /// State for a single opened QUIC connection. -#[derive(Debug)] -pub struct QuicMuxer { - inner: Mutex, +pub struct QuicMuxer { + inner: Mutex>, +} + +impl std::fmt::Debug for QuicMuxer { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "QuicMuxer") + } } /// Mutex protected fields of [`QuicMuxer`]. -#[derive(Debug)] -struct QuicMuxerInner { +struct QuicMuxerInner { + /// Accept incoming streams. + accept_incoming: bool, /// Endpoint channel. - endpoint: ConnectionChannel, + endpoint: ConnectionChannel, /// Inner connection object that yields events. - connection: Connection, + connection: Connection, /// Connection waker. waker: Option, /// Connection timer. @@ -55,10 +59,11 @@ struct SubstreamState { write_waker: Option, } -impl QuicMuxer { - pub fn new(endpoint: ConnectionChannel, connection: Connection) -> Self { +impl QuicMuxer { + pub fn new(endpoint: ConnectionChannel, connection: Connection) -> Self { Self { inner: Mutex::new(QuicMuxerInner { + accept_incoming: false, endpoint, connection, waker: None, @@ -74,14 +79,10 @@ impl QuicMuxer { self.inner.lock().connection.is_handshaking() } - pub fn peer_id(&self) -> PeerId { - self.inner - .lock() - .connection - .crypto_session() - .peer_identity() - .expect("In an IK handshake the PeerId is always available") - .to_peer_id() + pub fn peer_id(&self) -> Option { + let inner = self.inner.lock(); + let session = inner.connection.crypto_session(); + C::peer_id(&session) } pub fn local_addr(&self) -> Multiaddr { @@ -99,9 +100,14 @@ impl QuicMuxer { let addr = inner.connection.remote_address(); crate::transport::socketaddr_to_multiaddr(&addr) } + + pub(crate) fn set_accept_incoming(&self, accept: bool) { + let mut inner = self.inner.lock(); + inner.accept_incoming = accept; + } } -impl StreamMuxer for QuicMuxer { +impl StreamMuxer for QuicMuxer { type Substream = StreamId; type OutboundSubstream = (); type Error = QuicMuxerError; @@ -213,10 +219,12 @@ impl StreamMuxer for QuicMuxer { // TODO quinn doesn't support `StreamMuxerEvent::AddressChange`. - if let Some(id) = inner.connection.streams().accept(Dir::Bi) { - inner.substreams.insert(id, Default::default()); - tracing::trace!("opened incoming substream {}", id); - return Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(id))); + if inner.accept_incoming { + if let Some(id) = inner.connection.streams().accept(Dir::Bi) { + inner.substreams.insert(id, Default::default()); + tracing::trace!("opened incoming substream {}", id); + return Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(id))); + } } if inner.substreams.is_empty() { diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs new file mode 100644 index 00000000000..38e02808998 --- /dev/null +++ b/transports/quic/src/tls/certificate.rs @@ -0,0 +1,85 @@ +// Copyright 2020 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. + +//! Certificate handling for libp2p +//! +//! This module handles generation, signing, and verification of certificates. + +use super::LIBP2P_SIGNING_PREFIX_LENGTH; +use libp2p_core::identity::Keypair; + +const LIBP2P_OID: &[u64] = &[1, 3, 6, 1, 4, 1, 53594, 1, 1]; // Based on libp2p TLS 1.3 specs +const LIBP2P_SIGNATURE_ALGORITHM_PUBLIC_KEY_LENGTH: usize = 65; +static LIBP2P_SIGNATURE_ALGORITHM: &rcgen::SignatureAlgorithm = &rcgen::PKCS_ECDSA_P256_SHA256; + +/// Generates a self-signed TLS certificate that includes a libp2p-specific +/// certificate extension containing the public key of the given keypair. +pub(crate) fn make_cert(keypair: &Keypair) -> Result { + // Keypair used to sign the certificate. + let certif_keypair = rcgen::KeyPair::generate(&LIBP2P_SIGNATURE_ALGORITHM)?; + + // The libp2p-specific extension to the certificate contains a signature of the public key + // of the certificate using the libp2p private key. + let libp2p_ext_signature = { + let certif_pubkey = certif_keypair.public_key_raw(); + assert_eq!( + certif_pubkey.len(), + LIBP2P_SIGNATURE_ALGORITHM_PUBLIC_KEY_LENGTH, + ); + + let mut buf = + [0u8; LIBP2P_SIGNING_PREFIX_LENGTH + LIBP2P_SIGNATURE_ALGORITHM_PUBLIC_KEY_LENGTH]; + buf[..LIBP2P_SIGNING_PREFIX_LENGTH].copy_from_slice(&super::LIBP2P_SIGNING_PREFIX[..]); + buf[LIBP2P_SIGNING_PREFIX_LENGTH..].copy_from_slice(certif_pubkey); + keypair.sign(&buf)? + }; + + // Generate the libp2p-specific extension. + let libp2p_extension: rcgen::CustomExtension = { + let extension_content = { + let serialized_pubkey = keypair.public().into_protobuf_encoding(); + yasna::construct_der(|writer| { + writer.write_sequence(|writer| { + writer + .next() + .write_bitvec_bytes(&serialized_pubkey, serialized_pubkey.len() * 8); + writer + .next() + .write_bitvec_bytes(&libp2p_ext_signature, libp2p_ext_signature.len() * 8); + }) + }) + }; + + let mut ext = rcgen::CustomExtension::from_oid_content(LIBP2P_OID, extension_content); + ext.set_criticality(true); + ext + }; + + let certificate = { + let mut params = rcgen::CertificateParams::new(vec![]); + params.distinguished_name = rcgen::DistinguishedName::new(); + params.custom_extensions.push(libp2p_extension); + params.alg = &LIBP2P_SIGNATURE_ALGORITHM; + params.key_pair = Some(certif_keypair); + rcgen::Certificate::from_params(params)? + }; + + Ok(certificate) +} diff --git a/transports/quic/src/tls/mod.rs b/transports/quic/src/tls/mod.rs new file mode 100644 index 00000000000..91248529f4a --- /dev/null +++ b/transports/quic/src/tls/mod.rs @@ -0,0 +1,89 @@ +// Copyright 2020 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. + +//! TLS configuration for `libp2p-quic`. + +mod certificate; +mod verifier; + +use std::sync::Arc; +use thiserror::Error; + +pub use verifier::extract_peerid_or_panic; + +const LIBP2P_SIGNING_PREFIX: [u8; 21] = *b"libp2p-tls-handshake:"; +const LIBP2P_SIGNING_PREFIX_LENGTH: usize = LIBP2P_SIGNING_PREFIX.len(); +const LIBP2P_OID_BYTES: &[u8] = &[43, 6, 1, 4, 1, 131, 162, 90, 1, 1]; // Based on libp2p TLS 1.3 specs + +/// Error creating a configuration +// TODO: remove this; what is the user supposed to do with this error? +#[derive(Debug, Error)] +pub enum ConfigError { + /// TLS private key or certificate rejected + #[error("TLS private or certificate key rejected: {0}")] + TLSError(#[from] rustls::TLSError), + /// Signing failed + #[error("Signing failed: {0}")] + SigningError(#[from] libp2p_core::identity::error::SigningError), + /// Certificate generation error + #[error("Certificate generation error: {0}")] + RcgenError(#[from] rcgen::RcgenError), +} + +fn make_client_config( + certificate: rustls::Certificate, + key: rustls::PrivateKey, + verifier: Arc, +) -> Result { + let mut crypto = rustls::ClientConfig::new(); + crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; + crypto.alpn_protocols = vec![b"libp2p".to_vec()]; + crypto.enable_early_data = false; + crypto.set_single_client_cert(vec![certificate], key)?; + crypto.dangerous().set_certificate_verifier(verifier); + Ok(crypto) +} + +fn make_server_config( + certificate: rustls::Certificate, + key: rustls::PrivateKey, + verifier: Arc, +) -> Result { + let mut crypto = rustls::ServerConfig::new(verifier); + crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; + crypto.alpn_protocols = vec![b"libp2p".to_vec()]; + crypto.set_single_cert(vec![certificate], key)?; + Ok(crypto) +} + +/// Create TLS client and server configurations for libp2p. +pub fn make_tls_config( + keypair: &libp2p_core::identity::Keypair, +) -> Result<(rustls::ClientConfig, rustls::ServerConfig), ConfigError> { + let cert = certificate::make_cert(&keypair)?; + let private_key = cert.serialize_private_key_der(); + let verifier = Arc::new(verifier::Libp2pCertificateVerifier); + let cert = rustls::Certificate(cert.serialize_der()?); + let key = rustls::PrivateKey(private_key); + Ok(( + make_client_config(cert.clone(), key.clone(), verifier.clone())?, + make_server_config(cert, key, verifier)?, + )) +} diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs new file mode 100644 index 00000000000..364833ed7ca --- /dev/null +++ b/transports/quic/src/tls/verifier.rs @@ -0,0 +1,238 @@ +// Copyright 2020 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 libp2p_core::identity::PublicKey; +use libp2p_core::PeerId; +use ring::io::der; +use rustls::{ + internal::msgs::handshake::DigitallySignedStruct, Certificate, ClientCertVerified, + HandshakeSignatureValid, ServerCertVerified, TLSError, +}; +use untrusted::{Input, Reader}; +use webpki::Error; + +/// Implementation of the `rustls` certificate verification traits for libp2p. +/// +/// Only TLS 1.3 is supported. TLS 1.2 should be disabled in the configuration of `rustls`. +pub(crate) struct Libp2pCertificateVerifier; + +/// libp2p requires the following of X.509 server certificate chains: +/// +/// - Exactly one certificate must be presented. +/// - The certificate must be self-signed. +/// - The certificate must have a valid libp2p extension that includes a +/// signature of its public key. +/// +/// The check that the [`PeerId`] matches the expected `PeerId` must be done by +/// the caller. +impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { + fn verify_server_cert( + &self, + _roots: &rustls::RootCertStore, + presented_certs: &[rustls::Certificate], + _dns_name: webpki::DNSNameRef<'_>, + _ocsp_response: &[u8], + ) -> Result { + verify_presented_certs(presented_certs).map(|()| ServerCertVerified::assertion()) + } + + fn verify_tls12_signature( + &self, + _message: &[u8], + _cert: &Certificate, + _dss: &DigitallySignedStruct, + ) -> Result { + Err(TLSError::PeerIncompatibleError( + "Only TLS 1.3 certificates are supported".to_string(), + )) + } + + fn verify_tls13_signature( + &self, + message: &[u8], + cert: &Certificate, + dss: &DigitallySignedStruct, + ) -> Result { + verify_tls13_signature(message, cert, dss) + } +} + +/// libp2p requires the following of X.509 client certificate chains: +/// +/// - Exactly one certificate must be presented. In particular, client +/// authentication is mandatory in libp2p. +/// - The certificate must be self-signed. +/// - The certificate must have a valid libp2p extension that includes a +/// signature of its public key. +/// +/// The check that the [`PeerId`] matches the expected `PeerId` must be done by +/// the caller. +/// +/// [`PeerId`]: libp2p_core::PeerId +impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { + fn offer_client_auth(&self) -> bool { + true + } + + fn client_auth_root_subjects( + &self, + _dns_name: Option<&webpki::DNSName>, + ) -> Option { + Some(vec![]) + } + + fn verify_client_cert( + &self, + presented_certs: &[Certificate], + _dns_name: Option<&webpki::DNSName>, + ) -> Result { + verify_presented_certs(presented_certs).map(|()| ClientCertVerified::assertion()) + } + + fn verify_tls12_signature( + &self, + _message: &[u8], + _cert: &Certificate, + _dss: &DigitallySignedStruct, + ) -> Result { + Err(TLSError::PeerIncompatibleError( + "Only TLS 1.3 certificates are supported".to_string(), + )) + } + + fn verify_tls13_signature( + &self, + message: &[u8], + cert: &Certificate, + dss: &DigitallySignedStruct, + ) -> Result { + barebones_x509::parse_certificate(cert.as_ref()) + .map_err(rustls::TLSError::WebPKIError)? + .check_tls13_signature(dss.scheme, message, dss.sig.0.as_ref()) + .map_err(rustls::TLSError::WebPKIError) + .map(|()| rustls::HandshakeSignatureValid::assertion()) + } +} + +fn verify_tls13_signature( + message: &[u8], + cert: &Certificate, + dss: &DigitallySignedStruct, +) -> Result { + barebones_x509::parse_certificate(cert.as_ref()) + .map_err(rustls::TLSError::WebPKIError)? + .check_tls13_signature(dss.scheme, message, dss.sig.0.as_ref()) + .map_err(rustls::TLSError::WebPKIError) + .map(|()| rustls::HandshakeSignatureValid::assertion()) +} + +fn verify_libp2p_signature( + libp2p_extension: &Libp2pExtension<'_>, + x509_pkey_bytes: &[u8], +) -> Result<(), Error> { + let mut v = Vec::with_capacity(super::LIBP2P_SIGNING_PREFIX_LENGTH + x509_pkey_bytes.len()); + v.extend_from_slice(&super::LIBP2P_SIGNING_PREFIX[..]); + v.extend_from_slice(x509_pkey_bytes); + if libp2p_extension + .peer_key + .verify(&v, libp2p_extension.signature) + { + Ok(()) + } else { + Err(Error::UnknownIssuer) + } +} + +fn parse_certificate( + certificate: &[u8], +) -> Result<(barebones_x509::X509Certificate<'_>, Libp2pExtension<'_>), Error> { + let parsed = barebones_x509::parse_certificate(certificate)?; + let mut libp2p_extension = None; + + parsed + .extensions() + .iterate(&mut |oid, critical, extension| { + match oid { + super::LIBP2P_OID_BYTES if libp2p_extension.is_some() => return Err(Error::BadDER), + super::LIBP2P_OID_BYTES => { + libp2p_extension = Some(parse_libp2p_extension(extension)?) + } + _ if critical => return Err(Error::UnsupportedCriticalExtension), + _ => {} + }; + Ok(()) + })?; + let libp2p_extension = libp2p_extension.ok_or(Error::UnknownIssuer)?; + Ok((parsed, libp2p_extension)) +} + +fn verify_presented_certs(presented_certs: &[Certificate]) -> Result<(), TLSError> { + if presented_certs.len() != 1 { + return Err(TLSError::NoCertificatesPresented); + } + let (certificate, extension) = + parse_certificate(presented_certs[0].as_ref()).map_err(TLSError::WebPKIError)?; + certificate.valid().map_err(TLSError::WebPKIError)?; + certificate + .check_self_issued() + .map_err(TLSError::WebPKIError)?; + verify_libp2p_signature(&extension, certificate.subject_public_key_info().key()) + .map_err(TLSError::WebPKIError) +} + +struct Libp2pExtension<'a> { + peer_key: PublicKey, + signature: &'a [u8], +} + +fn parse_libp2p_extension(extension: Input<'_>) -> Result, Error> { + fn read_bit_string<'a>(input: &mut Reader<'a>, e: Error) -> Result, Error> { + der::bit_string_with_no_unused_bits(input).map_err(|_| e) + } + + let e = Error::ExtensionValueInvalid; + Input::read_all(&extension, e, |input| { + der::nested(input, der::Tag::Sequence, e, |input| { + let public_key = read_bit_string(input, e)?.as_slice_less_safe(); + let signature = read_bit_string(input, e)?.as_slice_less_safe(); + // We deliberately discard the error information because this is + // either a broken peer or an attack. + let peer_key = PublicKey::from_protobuf_encoding(public_key).map_err(|_| e)?; + Ok(Libp2pExtension { + peer_key, + signature, + }) + }) + }) +} + +/// Extracts the `PeerId` from a certificate’s libp2p extension. It is erroneous +/// to call this unless the certificate is known to be a well-formed X.509 +/// certificate with a valid libp2p extension. The certificate verifier in this +/// module check this. +/// +/// # Panics +/// +/// Panics if called on an invalid certificate. +pub fn extract_peerid_or_panic(certificate: &[u8]) -> PeerId { + let r = parse_certificate(certificate) + .expect("we already checked that the certificate was valid during the handshake; qed"); + PeerId::from_public_key(r.1.peer_key) +} diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index ffcee2b780e..9cf4b984abe 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -1,15 +1,17 @@ +use crate::crypto::Crypto; use crate::endpoint::{EndpointConfig, TransportChannel}; use crate::muxer::QuicMuxer; -use crate::{PublicKey, QuicConfig, QuicError}; +use crate::{QuicConfig, QuicError}; +use ed25519_dalek::PublicKey; use futures::channel::oneshot; -use futures::future::{ready, Ready}; use futures::prelude::*; use if_watch::{IfEvent, IfWatcher}; use libp2p_core::multiaddr::{Multiaddr, Protocol}; -use libp2p_core::muxing::StreamMuxerBox; +use libp2p_core::muxing::{StreamMuxer, StreamMuxerBox}; use libp2p_core::transport::{Boxed, ListenerEvent, Transport, TransportError}; use libp2p_core::PeerId; use parking_lot::Mutex; +use quinn_proto::crypto::Session; use std::net::{IpAddr, SocketAddr}; use std::pin::Pin; use std::sync::Arc; @@ -17,14 +19,19 @@ use std::task::{Context, Poll}; use udp_socket::SocketType; #[derive(Clone)] -pub struct QuicTransport { - inner: Arc>, +pub struct QuicTransport { + inner: Arc>>, } -impl QuicTransport { +impl QuicTransport +where + ::ClientConfig: Send + Unpin, + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ /// Creates a new quic transport. pub async fn new( - config: QuicConfig, + config: QuicConfig, addr: Multiaddr, ) -> Result> { let socket_addr = multiaddr_to_socketaddr(&addr) @@ -56,14 +63,14 @@ impl QuicTransport { } } -impl std::fmt::Debug for QuicTransport { +impl std::fmt::Debug for QuicTransport { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { f.debug_struct("QuicTransport").finish() } } -struct QuicTransportInner { - channel: TransportChannel, +struct QuicTransportInner { + channel: TransportChannel, addresses: Addresses, } @@ -72,12 +79,16 @@ enum Addresses { Ip(Option), } -impl Transport for QuicTransport { - type Output = (PeerId, QuicMuxer); +impl Transport for QuicTransport +where + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ + type Output = (PeerId, QuicMuxer); type Error = QuicError; type Listener = Self; - type ListenerUpgrade = Ready>; - type Dial = QuicDial; + type ListenerUpgrade = QuicUpgrade; + type Dial = QuicDial; fn listen_on(self, addr: Multiaddr) -> Result> { multiaddr_to_socketaddr(&addr).map_err(|_| TransportError::MultiaddrNotSupported(addr))?; @@ -98,7 +109,7 @@ impl Transport for QuicTransport { } tracing::debug!("dialing {}", socket_addr); let rx = self.inner.lock().channel.dial(socket_addr, public_key); - Ok(QuicDial { rx }) + Ok(QuicDial::Dialing(rx)) } fn address_translation(&self, _listen: &Multiaddr, observed: &Multiaddr) -> Option { @@ -106,9 +117,8 @@ impl Transport for QuicTransport { } } -impl Stream for QuicTransport { - type Item = - Result>, QuicError>, QuicError>; +impl Stream for QuicTransport { + type Item = Result, QuicError>, QuicError>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { let mut inner = self.inner.lock(); @@ -150,7 +160,7 @@ impl Stream for QuicTransport { Poll::Ready(Some(Ok(muxer))) => Poll::Ready(Some(Ok(ListenerEvent::Upgrade { local_addr: muxer.local_addr(), remote_addr: muxer.remote_addr(), - upgrade: ready(Ok((muxer.peer_id(), muxer))), + upgrade: QuicUpgrade::new(muxer), }))), Poll::Ready(Some(Err(err))) => Poll::Ready(Some(Err(err))), Poll::Ready(None) => Poll::Ready(None), @@ -160,22 +170,70 @@ impl Stream for QuicTransport { } #[allow(clippy::large_enum_variant)] -pub struct QuicDial { - rx: oneshot::Receiver>, +pub enum QuicDial { + Dialing(oneshot::Receiver, QuicError>>), + Upgrade(QuicUpgrade), } -impl Future for QuicDial { - type Output = Result<(PeerId, QuicMuxer), QuicError>; +impl Future for QuicDial +where + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ + type Output = Result<(PeerId, QuicMuxer), QuicError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { loop { - match Pin::new(&mut self.rx).poll(cx) { - Poll::Ready(Ok(Ok(muxer))) => { - return Poll::Ready(Ok((muxer.peer_id(), muxer))); + match &mut *self { + Self::Dialing(rx) => match Pin::new(rx).poll(cx) { + Poll::Ready(Ok(Ok(muxer))) => { + *self = Self::Upgrade(QuicUpgrade::new(muxer)); + } + Poll::Ready(Ok(Err(err))) => return Poll::Ready(Err(err)), + Poll::Ready(Err(_)) => panic!("endpoint crashed"), + Poll::Pending => return Poll::Pending, + }, + Self::Upgrade(upgrade) => return Pin::new(upgrade).poll(cx), + } + } + } +} + +pub struct QuicUpgrade { + muxer: Option>, +} + +impl QuicUpgrade { + fn new(muxer: QuicMuxer) -> Self { + Self { muxer: Some(muxer) } + } +} + +impl Future for QuicUpgrade +where + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ + type Output = Result<(PeerId, QuicMuxer), QuicError>; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let inner = Pin::into_inner(self); + let muxer = inner.muxer.as_mut().expect("future polled after ready"); + match muxer.poll_event(cx) { + Poll::Pending => { + if let Some(peer_id) = muxer.peer_id() { + muxer.set_accept_incoming(true); + Poll::Ready(Ok(( + peer_id, + inner.muxer.take().expect("future polled after ready"), + ))) + } else { + Poll::Pending } - Poll::Ready(Ok(Err(err))) => return Poll::Ready(Err(err)), - Poll::Ready(Err(_)) => panic!("endpoint crashed"), - Poll::Pending => return Poll::Pending, + } + Poll::Ready(Err(err)) => Poll::Ready(Err(err.into())), + Poll::Ready(Ok(_)) => { + panic!("muxer.incoming is set to false so no events can be produced"); } } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 26d9933d7fa..7209e881273 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -5,19 +5,37 @@ use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; use futures::stream::StreamExt; use libp2p::core::upgrade; use libp2p::multiaddr::Protocol; -use libp2p::quic::{Keypair, QuicConfig, ToLibp2p}; use libp2p::request_response::{ ProtocolName, ProtocolSupport, RequestResponse, RequestResponseCodec, RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, }; use libp2p::swarm::{Swarm, SwarmEvent}; +use libp2p_quic::{Crypto, Keypair, QuicConfig, ToLibp2p}; +use quinn_proto::crypto::Session; use rand::RngCore; use std::{io, iter}; -async fn create_swarm(keylog: bool) -> Result>> { +#[cfg(feature = "noise")] +#[async_std::test] +async fn smoke_noise() -> Result<()> { + smoke::().await +} + +#[cfg(feature = "tls")] +#[async_std::test] +async fn smoke_tls() -> Result<()> { + smoke::().await +} + +async fn create_swarm(keylog: bool) -> Result>> +where + ::ClientConfig: Send + Unpin, + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ let keypair = Keypair::generate(&mut rand_core::OsRng {}); let peer_id = keypair.to_peer_id(); - let mut transport = QuicConfig::new(keypair); + let mut transport = QuicConfig::::new(keypair); if keylog { transport.enable_keylogger(); } @@ -33,8 +51,12 @@ async fn create_swarm(keylog: bool) -> Result>> Ok(Swarm::new(transport, behaviour, peer_id)) } -#[async_std::test] -async fn smoke() -> Result<()> { +async fn smoke() -> Result<()> +where + ::ClientConfig: Send + Unpin, + ::HeaderKey: Unpin, + ::PacketKey: Unpin, +{ tracing_subscriber::fmt() .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) .try_init() @@ -42,8 +64,8 @@ async fn smoke() -> Result<()> { log_panics::init(); let mut rng = rand::thread_rng(); - let mut a = create_swarm(true).await?; - let mut b = create_swarm(false).await?; + let mut a = create_swarm::(true).await?; + let mut b = create_swarm::(false).await?; Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; From a762d1469552d846d2c8b1008700ef3b56000a36 Mon Sep 17 00:00:00 2001 From: David Craven Date: Mon, 9 Aug 2021 12:01:52 +0200 Subject: [PATCH 04/14] Update to latest libp2p-quic. --- transports/quic/src/crypto.rs | 15 ++-- transports/quic/src/lib.rs | 26 +++---- transports/quic/src/muxer.rs | 4 +- transports/quic/src/tls/certificate.rs | 23 ++---- transports/quic/src/tls/mod.rs | 57 +++++++-------- transports/quic/src/tls/verifier.rs | 46 ++++++------ transports/quic/src/transport.rs | 12 ++-- transports/quic/tests/smoke.rs | 99 ++++++++++++++++++++++++++ 8 files changed, 189 insertions(+), 93 deletions(-) diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index 2d95e7c8eb6..5aca6b6fe78 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -1,5 +1,5 @@ use ed25519_dalek::{Keypair, PublicKey}; -use libp2p_core::PeerId; +use libp2p::PeerId; use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; use std::sync::Arc; @@ -104,8 +104,8 @@ impl Crypto for TlsCrypto { ) -> ::ServerConfig { assert!(config.psk.is_none(), "invalid config"); use crate::ToLibp2p; - let (_, mut server) = - crate::tls::make_tls_config(&config.keypair.to_keypair()).expect("invalid config"); + let mut server = + crate::tls::make_server_config(&config.keypair.to_keypair()).expect("invalid config"); if let Some(key_log) = config.keylogger.clone() { server.key_log = key_log; } @@ -114,12 +114,15 @@ impl Crypto for TlsCrypto { fn new_client_config( config: &Arc>, - _remote_public: PublicKey, + remote_public: PublicKey, ) -> ::ClientConfig { assert!(config.psk.is_none(), "invalid config"); use crate::ToLibp2p; - let (mut client, _) = - crate::tls::make_tls_config(&config.keypair.to_keypair()).expect("invalid config"); + let mut client = crate::tls::make_client_config( + &config.keypair.to_keypair(), + remote_public.to_peer_id(), + ) + .expect("invalid config"); if let Some(key_log) = config.keylogger.clone() { client.key_log = key_log; } diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 086ec07dd15..e748e5ad5f1 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -17,8 +17,8 @@ pub use ed25519_dalek::{Keypair, PublicKey, SecretKey}; pub use quinn_noise::{KeyLog, KeyLogFile}; pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; -use libp2p_core::transport::TransportError; -use libp2p_core::{identity, Multiaddr, PeerId}; +use libp2p::core::transport::TransportError; +use libp2p::{Multiaddr, PeerId}; use quinn_proto::crypto::Session; use thiserror::Error; @@ -99,33 +99,33 @@ pub enum QuicError { } pub trait ToLibp2p { - fn to_keypair(&self) -> identity::Keypair; - fn to_public(&self) -> identity::PublicKey; + fn to_keypair(&self) -> libp2p::identity::Keypair; + fn to_public(&self) -> libp2p::identity::PublicKey; fn to_peer_id(&self) -> PeerId { - self.to_public().to_peer_id() + self.to_public().into_peer_id() } } impl ToLibp2p for Keypair { - fn to_keypair(&self) -> identity::Keypair { + fn to_keypair(&self) -> libp2p::identity::Keypair { let mut secret_key = self.secret.to_bytes(); - let secret_key = identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); - identity::Keypair::Ed25519(secret_key.into()) + let secret_key = libp2p::identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); + libp2p::identity::Keypair::Ed25519(secret_key.into()) } - fn to_public(&self) -> identity::PublicKey { + fn to_public(&self) -> libp2p::identity::PublicKey { self.public.to_public() } } impl ToLibp2p for PublicKey { - fn to_keypair(&self) -> identity::Keypair { + fn to_keypair(&self) -> libp2p::identity::Keypair { panic!("wtf?"); } - fn to_public(&self) -> identity::PublicKey { + fn to_public(&self) -> libp2p::identity::PublicKey { let public_key = self.to_bytes(); - let public_key = identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); - identity::PublicKey::Ed25519(public_key) + let public_key = libp2p::identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); + libp2p::identity::PublicKey::Ed25519(public_key) } } diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 0851b3d0de5..e3837c7175f 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -3,8 +3,8 @@ use crate::endpoint::ConnectionChannel; use async_io::Timer; use fnv::FnvHashMap; use futures::prelude::*; -use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; -use libp2p_core::{Multiaddr, PeerId}; +use libp2p::core::muxing::{StreamMuxer, StreamMuxerEvent}; +use libp2p::{Multiaddr, PeerId}; use parking_lot::Mutex; use quinn_proto::generic::Connection; use quinn_proto::{ diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index df071cc8376..4851802f850 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -23,10 +23,10 @@ //! This module handles generation, signing, and verification of certificates. use super::LIBP2P_SIGNING_PREFIX_LENGTH; -use libp2p_core::identity::Keypair; +use libp2p::identity::Keypair; const LIBP2P_OID: &[u64] = &[1, 3, 6, 1, 4, 1, 53594, 1, 1]; // Based on libp2p TLS 1.3 specs -const LIBP2P_SIGNATURE_ALGORITHM_PUBLIC_KEY_LENGTH: usize = 65; +const LIBP2P_SIGNATURE_ALGORITHM_PUBLIC_KEY_LENGTH: usize = 91; static LIBP2P_SIGNATURE_ALGORITHM: &rcgen::SignatureAlgorithm = &rcgen::PKCS_ECDSA_P256_SHA256; /// Generates a self-signed TLS certificate that includes a libp2p-specific @@ -38,7 +38,7 @@ pub(crate) fn make_cert(keypair: &Keypair) -> Result Result, -) -> Result { +pub fn make_client_config( + keypair: &libp2p::identity::Keypair, + remote_peer_id: PeerId, +) -> Result { + let cert = certificate::make_cert(&keypair)?; + let private_key = cert.serialize_private_key_der(); + let cert = rustls::Certificate(cert.serialize_der()?); + let key = rustls::PrivateKey(private_key); + let verifier = verifier::Libp2pServerCertificateVerifier(remote_peer_id); + let mut crypto = rustls::ClientConfig::new(); crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; crypto.alpn_protocols = vec![b"libp2p".to_vec()]; crypto.enable_early_data = false; - crypto.set_single_client_cert(vec![certificate], key)?; - crypto.dangerous().set_certificate_verifier(verifier); + crypto.set_single_client_cert(vec![cert], key)?; + crypto + .dangerous() + .set_certificate_verifier(Arc::new(verifier)); Ok(crypto) } -fn make_server_config( - certificate: rustls::Certificate, - key: rustls::PrivateKey, - verifier: Arc, -) -> Result { - let mut crypto = rustls::ServerConfig::new(verifier); - crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; - crypto.alpn_protocols = vec![b"libp2p".to_vec()]; - crypto.set_single_cert(vec![certificate], key)?; - Ok(crypto) -} - -/// Create TLS client and server configurations for libp2p. -pub fn make_tls_config( - keypair: &libp2p_core::identity::Keypair, -) -> Result<(rustls::ClientConfig, rustls::ServerConfig), ConfigError> { - let cert = certificate::make_cert(&keypair)?; +pub fn make_server_config( + keypair: &libp2p::identity::Keypair, +) -> Result { + let cert = certificate::make_cert(keypair)?; let private_key = cert.serialize_private_key_der(); - let verifier = Arc::new(verifier::Libp2pCertificateVerifier); let cert = rustls::Certificate(cert.serialize_der()?); let key = rustls::PrivateKey(private_key); - Ok(( - make_client_config(cert.clone(), key.clone(), verifier.clone())?, - make_server_config(cert, key, verifier)?, - )) + let verifier = verifier::Libp2pClientCertificateVerifier; + + let mut crypto = rustls::ServerConfig::new(Arc::new(verifier)); + crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; + crypto.alpn_protocols = vec![b"libp2p".to_vec()]; + crypto.set_single_cert(vec![cert], key)?; + Ok(crypto) } diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 9ee221f06f0..fada6b3dd17 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -18,8 +18,8 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use libp2p_core::identity::PublicKey; -use libp2p_core::PeerId; +use libp2p::identity::PublicKey; +use libp2p::PeerId; use ring::io::der; use rustls::{ internal::msgs::handshake::DigitallySignedStruct, Certificate, ClientCertVerified, @@ -31,7 +31,7 @@ use webpki::Error; /// Implementation of the `rustls` certificate verification traits for libp2p. /// /// Only TLS 1.3 is supported. TLS 1.2 should be disabled in the configuration of `rustls`. -pub(crate) struct Libp2pCertificateVerifier; +pub(crate) struct Libp2pServerCertificateVerifier(pub(crate) PeerId); /// libp2p requires the following of X.509 server certificate chains: /// @@ -39,10 +39,7 @@ pub(crate) struct Libp2pCertificateVerifier; /// - The certificate must be self-signed. /// - The certificate must have a valid libp2p extension that includes a /// signature of its public key. -/// -/// The check that the [`PeerId`] matches the expected `PeerId` must be done by -/// the caller. -impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { +impl rustls::ServerCertVerifier for Libp2pServerCertificateVerifier { fn verify_server_cert( &self, _roots: &rustls::RootCertStore, @@ -50,7 +47,13 @@ impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { _dns_name: webpki::DNSNameRef<'_>, _ocsp_response: &[u8], ) -> Result { - verify_presented_certs(presented_certs).map(|()| ServerCertVerified::assertion()) + let peer_id = verify_presented_certs(presented_certs)?; + if peer_id != self.0 { + return Err(TLSError::PeerIncompatibleError( + "Unexpected peer id".to_string(), + )); + } + Ok(ServerCertVerified::assertion()) } fn verify_tls12_signature( @@ -74,6 +77,11 @@ impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { } } +/// Implementation of the `rustls` certificate verification traits for libp2p. +/// +/// Only TLS 1.3 is supported. TLS 1.2 should be disabled in the configuration of `rustls`. +pub(crate) struct Libp2pClientCertificateVerifier; + /// libp2p requires the following of X.509 client certificate chains: /// /// - Exactly one certificate must be presented. In particular, client @@ -81,12 +89,7 @@ impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { /// - The certificate must be self-signed. /// - The certificate must have a valid libp2p extension that includes a /// signature of its public key. -/// -/// The check that the [`PeerId`] matches the expected `PeerId` must be done by -/// the caller. -/// -/// [`PeerId`]: libp2p_core::PeerId -impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { +impl rustls::ClientCertVerifier for Libp2pClientCertificateVerifier { fn offer_client_auth(&self) -> bool { true } @@ -103,7 +106,7 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { presented_certs: &[Certificate], _dns_name: Option<&webpki::DNSName>, ) -> Result { - verify_presented_certs(presented_certs).map(|()| ClientCertVerified::assertion()) + verify_presented_certs(presented_certs).map(|_| ClientCertVerified::assertion()) } fn verify_tls12_signature( @@ -183,7 +186,7 @@ fn parse_certificate( Ok((parsed, libp2p_extension)) } -fn verify_presented_certs(presented_certs: &[Certificate]) -> Result<(), TLSError> { +fn verify_presented_certs(presented_certs: &[Certificate]) -> Result { if presented_certs.len() != 1 { return Err(TLSError::NoCertificatesPresented); } @@ -193,8 +196,9 @@ fn verify_presented_certs(presented_certs: &[Certificate]) -> Result<(), TLSErro certificate .check_self_issued() .map_err(TLSError::WebPKIError)?; - verify_libp2p_signature(&extension, certificate.subject_public_key_info().key()) - .map_err(TLSError::WebPKIError) + verify_libp2p_signature(&extension, certificate.subject_public_key_info().spki()) + .map_err(TLSError::WebPKIError)?; + Ok(PeerId::from_public_key(extension.peer_key)) } struct Libp2pExtension<'a> { @@ -204,7 +208,9 @@ struct Libp2pExtension<'a> { fn parse_libp2p_extension(extension: Input<'_>) -> Result, Error> { fn read_bit_string<'a>(input: &mut Reader<'a>, e: Error) -> Result, Error> { - der::bit_string_with_no_unused_bits(input).map_err(|_| e) + // The specification states that this is a BIT STRING, but the Go implementation + // uses an OCTET STRING. OCTET STRING is superior in this context, so use it. + der::expect_tag_and_get_value(input, der::Tag::OctetString).map_err(|_| e) } let e = Error::ExtensionValueInvalid; @@ -234,5 +240,5 @@ fn parse_libp2p_extension(extension: Input<'_>) -> Result, E pub fn extract_peerid_or_panic(certificate: &[u8]) -> PeerId { let r = parse_certificate(certificate) .expect("we already checked that the certificate was valid during the handshake; qed"); - PeerId::from_public_key(&r.1.peer_key) + PeerId::from_public_key(r.1.peer_key) } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 9cf4b984abe..3c555eed93a 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -6,10 +6,10 @@ use ed25519_dalek::PublicKey; use futures::channel::oneshot; use futures::prelude::*; use if_watch::{IfEvent, IfWatcher}; -use libp2p_core::multiaddr::{Multiaddr, Protocol}; -use libp2p_core::muxing::{StreamMuxer, StreamMuxerBox}; -use libp2p_core::transport::{Boxed, ListenerEvent, Transport, TransportError}; -use libp2p_core::PeerId; +use libp2p::core::muxing::{StreamMuxer, StreamMuxerBox}; +use libp2p::core::transport::{Boxed, ListenerEvent, Transport, TransportError}; +use libp2p::multiaddr::{Multiaddr, Protocol}; +use libp2p::PeerId; use parking_lot::Mutex; use quinn_proto::crypto::Session; use std::net::{IpAddr, SocketAddr}; @@ -252,8 +252,8 @@ fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result<(SocketAddr, Option Result<()> { + tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .try_init() + .ok(); + log_panics::init(); + + let mut a = create_swarm::(true).await?; + let mut b = create_swarm::(false).await?; + + Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; + + let keypair = Keypair::generate(&mut rand_core::OsRng {}); + let fake_peer_id = keypair.to_peer_id(); + + let mut addr = match a.next().await { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + }; + addr.push(Protocol::P2p(fake_peer_id.into())); + + b.behaviour_mut().add_address(&fake_peer_id, addr); + b.behaviour_mut() + .send_request(&fake_peer_id, Ping(b"hello world".to_vec())); + + match b.next().await { + Some(SwarmEvent::Dialing(_)) => {} + e => panic!("{:?}", e), + } + + match b.next().await { + Some(SwarmEvent::ConnectionEstablished { .. }) => {} + e => panic!("{:?}", e), + }; + + match b.next().await { + Some(SwarmEvent::ConnectionClosed { .. }) => {} + e => panic!("{:?}", e), + }; + + assert!(a.next().now_or_never().is_none()); + + Ok(()) +} + +#[cfg(feature = "tls")] +#[async_std::test] +async fn dial_failure_tls() -> Result<()> { + tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .try_init() + .ok(); + log_panics::init(); + + let mut a = create_swarm::(true).await?; + let mut b = create_swarm::(false).await?; + + Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; + + let keypair = Keypair::generate(&mut rand_core::OsRng {}); + let fake_peer_id = keypair.to_peer_id(); + + let mut addr = match a.next().await { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + }; + addr.push(Protocol::P2p(fake_peer_id.into())); + + b.behaviour_mut().add_address(&fake_peer_id, addr); + b.behaviour_mut() + .send_request(&fake_peer_id, Ping(b"hello world".to_vec())); + + match b.next().await { + Some(SwarmEvent::Dialing(_)) => {} + e => panic!("{:?}", e), + } + + match a.next().await { + Some(SwarmEvent::IncomingConnection { .. }) => {} + e => panic!("{:?}", e), + } + + match b.next().await { + Some(SwarmEvent::UnreachableAddr { .. }) => {} + e => panic!("{:?}", e), + }; + + match b.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::OutboundFailure { .. })) => {} + e => panic!("{:?}", e), + }; + + assert!(a.next().now_or_never().is_none()); + + Ok(()) +} From 4a69421afa7ecc74e52ae39668ab68aff15077d2 Mon Sep 17 00:00:00 2001 From: David Craven Date: Mon, 9 Aug 2021 12:05:55 +0200 Subject: [PATCH 05/14] Update imports. --- transports/quic/src/crypto.rs | 2 +- transports/quic/src/lib.rs | 26 +++++++++++++------------- transports/quic/src/muxer.rs | 4 ++-- transports/quic/src/tls/certificate.rs | 4 ++-- transports/quic/src/tls/mod.rs | 8 ++++---- transports/quic/src/tls/verifier.rs | 8 ++++---- transports/quic/src/transport.rs | 12 ++++++------ 7 files changed, 32 insertions(+), 32 deletions(-) diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index 5aca6b6fe78..cca26153446 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -1,5 +1,5 @@ use ed25519_dalek::{Keypair, PublicKey}; -use libp2p::PeerId; +use libp2p_core::PeerId; use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; use std::sync::Arc; diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index e748e5ad5f1..086ec07dd15 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -17,8 +17,8 @@ pub use ed25519_dalek::{Keypair, PublicKey, SecretKey}; pub use quinn_noise::{KeyLog, KeyLogFile}; pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; -use libp2p::core::transport::TransportError; -use libp2p::{Multiaddr, PeerId}; +use libp2p_core::transport::TransportError; +use libp2p_core::{identity, Multiaddr, PeerId}; use quinn_proto::crypto::Session; use thiserror::Error; @@ -99,33 +99,33 @@ pub enum QuicError { } pub trait ToLibp2p { - fn to_keypair(&self) -> libp2p::identity::Keypair; - fn to_public(&self) -> libp2p::identity::PublicKey; + fn to_keypair(&self) -> identity::Keypair; + fn to_public(&self) -> identity::PublicKey; fn to_peer_id(&self) -> PeerId { - self.to_public().into_peer_id() + self.to_public().to_peer_id() } } impl ToLibp2p for Keypair { - fn to_keypair(&self) -> libp2p::identity::Keypair { + fn to_keypair(&self) -> identity::Keypair { let mut secret_key = self.secret.to_bytes(); - let secret_key = libp2p::identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); - libp2p::identity::Keypair::Ed25519(secret_key.into()) + let secret_key = identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); + identity::Keypair::Ed25519(secret_key.into()) } - fn to_public(&self) -> libp2p::identity::PublicKey { + fn to_public(&self) -> identity::PublicKey { self.public.to_public() } } impl ToLibp2p for PublicKey { - fn to_keypair(&self) -> libp2p::identity::Keypair { + fn to_keypair(&self) -> identity::Keypair { panic!("wtf?"); } - fn to_public(&self) -> libp2p::identity::PublicKey { + fn to_public(&self) -> identity::PublicKey { let public_key = self.to_bytes(); - let public_key = libp2p::identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); - libp2p::identity::PublicKey::Ed25519(public_key) + let public_key = identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); + identity::PublicKey::Ed25519(public_key) } } diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index e3837c7175f..0851b3d0de5 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -3,8 +3,8 @@ use crate::endpoint::ConnectionChannel; use async_io::Timer; use fnv::FnvHashMap; use futures::prelude::*; -use libp2p::core::muxing::{StreamMuxer, StreamMuxerEvent}; -use libp2p::{Multiaddr, PeerId}; +use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; +use libp2p_core::{Multiaddr, PeerId}; use parking_lot::Mutex; use quinn_proto::generic::Connection; use quinn_proto::{ diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index 4851802f850..2de3d0f0aba 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -23,7 +23,7 @@ //! This module handles generation, signing, and verification of certificates. use super::LIBP2P_SIGNING_PREFIX_LENGTH; -use libp2p::identity::Keypair; +use libp2p_core::identity::Keypair; const LIBP2P_OID: &[u64] = &[1, 3, 6, 1, 4, 1, 53594, 1, 1]; // Based on libp2p TLS 1.3 specs const LIBP2P_SIGNATURE_ALGORITHM_PUBLIC_KEY_LENGTH: usize = 91; @@ -54,7 +54,7 @@ pub(crate) fn make_cert(keypair: &Keypair) -> Result Result { let cert = certificate::make_cert(&keypair)?; @@ -70,7 +70,7 @@ pub fn make_client_config( } pub fn make_server_config( - keypair: &libp2p::identity::Keypair, + keypair: &libp2p_core::identity::Keypair, ) -> Result { let cert = certificate::make_cert(keypair)?; let private_key = cert.serialize_private_key_der(); diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index fada6b3dd17..9dcf2ed2298 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -18,8 +18,8 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use libp2p::identity::PublicKey; -use libp2p::PeerId; +use libp2p_core::identity::PublicKey; +use libp2p_core::PeerId; use ring::io::der; use rustls::{ internal::msgs::handshake::DigitallySignedStruct, Certificate, ClientCertVerified, @@ -198,7 +198,7 @@ fn verify_presented_certs(presented_certs: &[Certificate]) -> Result { @@ -240,5 +240,5 @@ fn parse_libp2p_extension(extension: Input<'_>) -> Result, E pub fn extract_peerid_or_panic(certificate: &[u8]) -> PeerId { let r = parse_certificate(certificate) .expect("we already checked that the certificate was valid during the handshake; qed"); - PeerId::from_public_key(r.1.peer_key) + PeerId::from_public_key(&r.1.peer_key) } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 3c555eed93a..9cf4b984abe 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -6,10 +6,10 @@ use ed25519_dalek::PublicKey; use futures::channel::oneshot; use futures::prelude::*; use if_watch::{IfEvent, IfWatcher}; -use libp2p::core::muxing::{StreamMuxer, StreamMuxerBox}; -use libp2p::core::transport::{Boxed, ListenerEvent, Transport, TransportError}; -use libp2p::multiaddr::{Multiaddr, Protocol}; -use libp2p::PeerId; +use libp2p_core::multiaddr::{Multiaddr, Protocol}; +use libp2p_core::muxing::{StreamMuxer, StreamMuxerBox}; +use libp2p_core::transport::{Boxed, ListenerEvent, Transport, TransportError}; +use libp2p_core::PeerId; use parking_lot::Mutex; use quinn_proto::crypto::Session; use std::net::{IpAddr, SocketAddr}; @@ -252,8 +252,8 @@ fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result<(SocketAddr, Option Date: Thu, 2 Sep 2021 18:00:48 +0300 Subject: [PATCH 06/14] Review fixes --- transports/quic/Cargo.toml | 14 +-- transports/quic/src/crypto.rs | 121 +++++++++++++++++++------ transports/quic/src/endpoint.rs | 9 +- transports/quic/src/lib.rs | 61 ++----------- transports/quic/src/muxer.rs | 4 +- transports/quic/src/tls/certificate.rs | 6 +- transports/quic/src/tls/mod.rs | 2 +- transports/quic/src/tls/verifier.rs | 2 +- transports/quic/src/transport.rs | 79 +++++++++++----- transports/quic/tests/smoke.rs | 42 +++++++-- 10 files changed, 212 insertions(+), 128 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 397d3f63fe7..999297639fb 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -3,21 +3,20 @@ name = "libp2p-quic" version = "0.6.0" authors = ["David Craven ", "Parity Technologies "] edition = "2018" -description = "libp2p-quic is a noise based quic implementation for rust-libp2p." -repository = "https://github.com/ipfs-rust/libp2p-quic" -license = "ISC" +description = "TLS and Noise based QUIC transport implementation for libp2p" +repository = "https://github.com/libp2p/rust-libp2p" +license = "MIT" [features] -noise = ["quinn-noise"] +noise = ["quinn-noise", "ed25519-dalek"] tls = ["barebones-x509", "quinn-proto/tls-rustls", "rcgen", "ring", "rustls", "untrusted", "webpki", "yasna"] [dependencies] -anyhow = "1.0.41" async-global-executor = "2.0.2" async-io = "1.6.0" barebones-x509 = { version = "0.5.0", optional = true, features = ["webpki", "rustls", "std"] } bytes = "1.0.1" -ed25519-dalek = "1.0.1" +ed25519-dalek = { version = "1.0.1", optional = true } fnv = "1.0.7" futures = "0.3.15" if-watch = "0.2.2" @@ -26,7 +25,6 @@ multihash = { version = "0.14.0", default-features = false } parking_lot = "0.11.1" quinn-noise = { version = "0.3.0", optional = true } quinn-proto = { version = "0.7.3", default-features = false } -rand_core = "0.5.1" rcgen = { version = "0.8.11", optional = true } ring = { version = "0.16.20", optional = true } rustls = { version = "0.19.1", optional = true, features = ["dangerous_configuration"] } @@ -38,9 +36,11 @@ webpki = { version = "0.21.4", optional = true, features = ["std"] } yasna = { version = "0.4.0", optional = true } [dev-dependencies] +anyhow = "1.0.41" async-std = { version = "1.9.0", features = ["attributes"] } async-trait = "0.1.50" libp2p = { version = "0.40.0", default-features = false, features = ["request-response"], path = "../.." } log-panics = "2.0.0" rand = "0.8.4" +rand_core = "0.5.1" tracing-subscriber = "0.2.19" diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index cca26153446..ee1ec0321f5 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -1,37 +1,91 @@ -use ed25519_dalek::{Keypair, PublicKey}; use libp2p_core::PeerId; use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; use std::sync::Arc; -pub struct CryptoConfig { - pub keypair: Keypair, +pub struct CryptoConfig { + pub keypair: C::Keypair, pub psk: Option<[u8; 32]>, - pub keylogger: Option, + pub keylogger: Option, pub transport: Arc, } #[cfg(feature = "noise")] -impl CryptoConfig { - fn clone_keypair(&self) -> Keypair { - Keypair::from_bytes(&self.keypair.to_bytes()).expect("serde works") +trait CloneKeypair { + fn clone_keypair(&self) -> Self; +} + +#[cfg(feature = "noise")] +impl CloneKeypair for ed25519_dalek::Keypair { + fn clone_keypair(&self) -> Self { + ed25519_dalek::Keypair::from_bytes(&self.to_bytes()).expect("serde works") + } +} + +pub trait ToLibp2p { + fn to_keypair(&self) -> libp2p_core::identity::Keypair; + fn to_public(&self) -> libp2p_core::identity::PublicKey; + fn to_peer_id(&self) -> PeerId { + self.to_public().to_peer_id() + } +} + +#[cfg(feature = "noise")] +impl ToLibp2p for ed25519_dalek::Keypair { + fn to_keypair(&self) -> libp2p_core::identity::Keypair { + let mut secret_key = self.secret.to_bytes(); + let secret_key = + libp2p_core::identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); + libp2p_core::identity::Keypair::Ed25519(secret_key.into()) + } + + fn to_public(&self) -> libp2p_core::identity::PublicKey { + self.public.to_public() + } +} + +#[cfg(feature = "noise")] +impl ToLibp2p for ed25519_dalek::PublicKey { + fn to_keypair(&self) -> libp2p_core::identity::Keypair { + panic!("wtf?"); + } + + fn to_public(&self) -> libp2p_core::identity::PublicKey { + let public_key = self.to_bytes(); + let public_key = + libp2p_core::identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); + libp2p_core::identity::PublicKey::Ed25519(public_key) + } +} + +#[cfg(feature = "tls")] +impl ToLibp2p for libp2p_core::identity::Keypair { + fn to_keypair(&self) -> libp2p_core::identity::Keypair { + self.clone() + } + + fn to_public(&self) -> libp2p_core::identity::PublicKey { + self.public() } } pub trait Crypto: std::fmt::Debug + Clone + 'static { type Session: Session + Unpin; type Keylogger: Send + Sync; + type Keypair: Send + Sync + ToLibp2p; + type PublicKey: Send + std::fmt::Debug + PartialEq; fn new_server_config( - config: &Arc>, + config: &Arc>, ) -> ::ServerConfig; fn new_client_config( - config: &Arc>, - remote_public: PublicKey, + config: &Arc>, + remote_public: Self::PublicKey, ) -> ::ClientConfig; fn supported_quic_versions() -> Vec; fn default_quic_version() -> u32; fn peer_id(session: &Self::Session) -> Option; + fn extract_public_key(generic_key: libp2p_core::PublicKey) -> Option; fn keylogger() -> Self::Keylogger; } @@ -43,13 +97,15 @@ pub struct NoiseCrypto; impl Crypto for NoiseCrypto { type Session = quinn_noise::NoiseSession; type Keylogger = Arc; + type Keypair = ed25519_dalek::Keypair; + type PublicKey = ed25519_dalek::PublicKey; fn new_server_config( - config: &Arc>, + config: &Arc>, ) -> ::ServerConfig { Arc::new( quinn_noise::NoiseServerConfig { - keypair: config.clone_keypair(), + keypair: config.keypair.clone_keypair(), psk: config.psk, keylogger: config.keylogger.clone(), supported_protocols: vec![b"libp2p".to_vec()], @@ -59,11 +115,11 @@ impl Crypto for NoiseCrypto { } fn new_client_config( - config: &Arc>, - remote_public_key: PublicKey, + config: &Arc>, + remote_public_key: Self::PublicKey, ) -> ::ClientConfig { quinn_noise::NoiseClientConfig { - keypair: config.clone_keypair(), + keypair: config.keypair.clone_keypair(), psk: config.psk, alpn: b"libp2p".to_vec(), remote_public_key, @@ -81,10 +137,18 @@ impl Crypto for NoiseCrypto { } fn peer_id(session: &Self::Session) -> Option { - use crate::ToLibp2p; Some(session.peer_identity()?.to_peer_id()) } + fn extract_public_key(generic_key: libp2p_core::PublicKey) -> Option { + let public_key = if let libp2p_core::PublicKey::Ed25519(public_key) = generic_key { + public_key.encode() + } else { + return None; + }; + Self::PublicKey::from_bytes(&public_key).ok() + } + fn keylogger() -> Self::Keylogger { Arc::new(quinn_noise::KeyLogFile::new()) } @@ -98,14 +162,14 @@ pub struct TlsCrypto; impl Crypto for TlsCrypto { type Session = quinn_proto::crypto::rustls::TlsSession; type Keylogger = Arc; + type Keypair = libp2p_core::identity::Keypair; + type PublicKey = libp2p_core::identity::PublicKey; fn new_server_config( - config: &Arc>, + config: &Arc>, ) -> ::ServerConfig { assert!(config.psk.is_none(), "invalid config"); - use crate::ToLibp2p; - let mut server = - crate::tls::make_server_config(&config.keypair.to_keypair()).expect("invalid config"); + let mut server = crate::tls::make_server_config(&config.keypair).expect("invalid config"); if let Some(key_log) = config.keylogger.clone() { server.key_log = key_log; } @@ -113,16 +177,13 @@ impl Crypto for TlsCrypto { } fn new_client_config( - config: &Arc>, - remote_public: PublicKey, + config: &Arc>, + remote_public: Self::PublicKey, ) -> ::ClientConfig { assert!(config.psk.is_none(), "invalid config"); - use crate::ToLibp2p; - let mut client = crate::tls::make_client_config( - &config.keypair.to_keypair(), - remote_public.to_peer_id(), - ) - .expect("invalid config"); + let mut client = + crate::tls::make_client_config(&config.keypair, remote_public.to_peer_id()) + .expect("invalid config"); if let Some(key_log) = config.keylogger.clone() { client.key_log = key_log; } @@ -144,6 +205,10 @@ impl Crypto for TlsCrypto { )) } + fn extract_public_key(generic_key: libp2p_core::PublicKey) -> Option { + Some(generic_key) + } + fn keylogger() -> Self::Keylogger { Arc::new(rustls::KeyLogFile::new()) } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 9d44cb21606..ba38771dc7c 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -1,7 +1,6 @@ use crate::crypto::{Crypto, CryptoConfig}; use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; -use ed25519_dalek::PublicKey; use fnv::FnvHashMap; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; @@ -28,7 +27,7 @@ enum ToEndpoint { /// UDP address to connect to. addr: SocketAddr, /// The remotes public key. - public_key: PublicKey, + public_key: C::PublicKey, /// Channel to return the result of the dialing to. tx: oneshot::Sender, QuicError>>, }, @@ -54,7 +53,7 @@ impl TransportChannel { pub fn dial( &mut self, addr: SocketAddr, - public_key: PublicKey, + public_key: C::PublicKey, ) -> oneshot::Receiver, QuicError>> { let (tx, rx) = oneshot::channel(); let msg = ToEndpoint::Dial { @@ -159,7 +158,7 @@ pub struct EndpointConfig { socket: UdpSocket, endpoint: QuinnEndpoint, port: u16, - crypto_config: Arc>, + crypto_config: Arc>, capabilities: UdpCapabilities, } @@ -230,7 +229,7 @@ struct Endpoint { channel: EndpointChannel, endpoint: QuinnEndpoint, socket: UdpSocket, - crypto_config: Arc>, + crypto_config: Arc>, connections: FnvHashMap>, outgoing: VecDeque, recv_buf: Box<[u8]>, diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 086ec07dd15..9185922c6cb 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -10,45 +10,30 @@ pub use crate::crypto::Crypto; pub use crate::crypto::NoiseCrypto; #[cfg(feature = "tls")] pub use crate::crypto::TlsCrypto; +pub use crate::crypto::ToLibp2p; pub use crate::muxer::{QuicMuxer, QuicMuxerError}; pub use crate::transport::{QuicDial, QuicTransport}; -pub use ed25519_dalek::{Keypair, PublicKey, SecretKey}; #[cfg(feature = "noise")] pub use quinn_noise::{KeyLog, KeyLogFile}; pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; use libp2p_core::transport::TransportError; -use libp2p_core::{identity, Multiaddr, PeerId}; +use libp2p_core::Multiaddr; use quinn_proto::crypto::Session; use thiserror::Error; -pub fn generate_keypair() -> Keypair { - Keypair::generate(&mut rand_core::OsRng {}) -} - /// Quic configuration. pub struct QuicConfig { - pub keypair: Keypair, + pub keypair: C::Keypair, pub psk: Option<[u8; 32]>, pub transport: TransportConfig, pub keylogger: Option, } -impl Default for QuicConfig { - fn default() -> Self { - Self { - keypair: Keypair::generate(&mut rand_core::OsRng {}), - psk: None, - transport: TransportConfig::default(), - keylogger: None, - } - } -} - impl std::fmt::Debug for QuicConfig { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { f.debug_struct("QuicConfig") - .field("keypair", &self.keypair.public) + .field("keypair", &self.keypair.to_public()) .field("psk", &self.psk) .field("transport", &self.transport) .finish() @@ -62,10 +47,12 @@ where ::PacketKey: Unpin, { /// Creates a new config from a keypair. - pub fn new(keypair: Keypair) -> Self { + pub fn new(keypair: C::Keypair) -> Self { Self { keypair, - ..Default::default() + psk: None, + transport: TransportConfig::default(), + keylogger: None, } } @@ -97,35 +84,3 @@ pub enum QuicError { #[error("a `StreamMuxerEvent` was generated before the handshake was complete.")] UpgradeError, } - -pub trait ToLibp2p { - fn to_keypair(&self) -> identity::Keypair; - fn to_public(&self) -> identity::PublicKey; - fn to_peer_id(&self) -> PeerId { - self.to_public().to_peer_id() - } -} - -impl ToLibp2p for Keypair { - fn to_keypair(&self) -> identity::Keypair { - let mut secret_key = self.secret.to_bytes(); - let secret_key = identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); - identity::Keypair::Ed25519(secret_key.into()) - } - - fn to_public(&self) -> identity::PublicKey { - self.public.to_public() - } -} - -impl ToLibp2p for PublicKey { - fn to_keypair(&self) -> identity::Keypair { - panic!("wtf?"); - } - - fn to_public(&self) -> identity::PublicKey { - let public_key = self.to_bytes(); - let public_key = identity::ed25519::PublicKey::decode(&public_key[..]).unwrap(); - identity::PublicKey::Ed25519(public_key) - } -} diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 0851b3d0de5..794f7394a76 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -82,7 +82,7 @@ impl QuicMuxer { pub fn peer_id(&self) -> Option { let inner = self.inner.lock(); let session = inner.connection.crypto_session(); - C::peer_id(&session) + C::peer_id(session) } pub fn local_addr(&self) -> Multiaddr { @@ -305,7 +305,7 @@ impl StreamMuxer for QuicMuxer { waker.wake(); } } - let substream = inner.substreams.get_mut(&id).unwrap(); + let substream = inner.substreams.get_mut(id).unwrap(); if pending && bytes == 0 { substream.read_waker = Some(cx.waker().clone()); Poll::Pending diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index 2de3d0f0aba..59b6c2a4e77 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -33,7 +33,7 @@ static LIBP2P_SIGNATURE_ALGORITHM: &rcgen::SignatureAlgorithm = &rcgen::PKCS_ECD /// certificate extension containing the public key of the given keypair. pub(crate) fn make_cert(keypair: &Keypair) -> Result { // Keypair used to sign the certificate. - let certif_keypair = rcgen::KeyPair::generate(&LIBP2P_SIGNATURE_ALGORITHM)?; + let certif_keypair = rcgen::KeyPair::generate(LIBP2P_SIGNATURE_ALGORITHM)?; // The libp2p-specific extension to the certificate contains a signature of the public key // of the certificate using the libp2p private key. @@ -59,7 +59,7 @@ pub(crate) fn make_cert(keypair: &Keypair) -> Result Result Result { - let cert = certificate::make_cert(&keypair)?; + let cert = certificate::make_cert(keypair)?; let private_key = cert.serialize_private_key_der(); let cert = rustls::Certificate(cert.serialize_der()?); let key = rustls::PrivateKey(private_key); diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 9dcf2ed2298..a82bc39575e 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -174,7 +174,7 @@ fn parse_certificate( .iterate(&mut |oid, critical, extension| { match oid { super::LIBP2P_OID_BYTES if libp2p_extension.is_some() => return Err(Error::BadDER), - super::LIBP2P_OID_BYTES => { + super::LIBP2P_OID_BYTES if critical == true => { libp2p_extension = Some(parse_libp2p_extension(extension)?) } _ if critical => return Err(Error::UnsupportedCriticalExtension), diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 9cf4b984abe..ae2ca5c4ca6 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -2,7 +2,6 @@ use crate::crypto::Crypto; use crate::endpoint::{EndpointConfig, TransportChannel}; use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; -use ed25519_dalek::PublicKey; use futures::channel::oneshot; use futures::prelude::*; use if_watch::{IfEvent, IfWatcher}; @@ -34,7 +33,7 @@ where config: QuicConfig, addr: Multiaddr, ) -> Result> { - let socket_addr = multiaddr_to_socketaddr(&addr) + let socket_addr = multiaddr_to_socketaddr::(&addr) .map_err(|_| TransportError::MultiaddrNotSupported(addr.clone()))? .0; let addresses = if socket_addr.ip().is_unspecified() { @@ -91,13 +90,14 @@ where type Dial = QuicDial; fn listen_on(self, addr: Multiaddr) -> Result> { - multiaddr_to_socketaddr(&addr).map_err(|_| TransportError::MultiaddrNotSupported(addr))?; + multiaddr_to_socketaddr::(&addr) + .map_err(|_| TransportError::MultiaddrNotSupported(addr))?; Ok(self) } fn dial(self, addr: Multiaddr) -> Result> { let (socket_addr, public_key) = - if let Ok((socket_addr, Some(public_key))) = multiaddr_to_socketaddr(&addr) { + if let Ok((socket_addr, Some(public_key))) = multiaddr_to_socketaddr::(&addr) { (socket_addr, public_key) } else { tracing::debug!("invalid multiaddr"); @@ -241,7 +241,9 @@ where /// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns an error if the format /// of the multiaddr is wrong. -fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result<(SocketAddr, Option), ()> { +fn multiaddr_to_socketaddr( + addr: &Multiaddr, +) -> Result<(SocketAddr, Option), ()> { let mut iter = addr.iter().peekable(); let proto1 = iter.next().ok_or(())?; let proto2 = iter.next().ok_or(())?; @@ -253,12 +255,7 @@ fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result<(SocketAddr, Option Multiaddr { mod tests { use super::*; - #[test] - fn multiaddr_to_udp_conversion() { - use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; + fn multiaddr_to_udp_conversion() { + use std::net::{Ipv4Addr, Ipv6Addr}; - assert!( - multiaddr_to_socketaddr(&"/ip4/127.0.0.1/udp/1234".parse::().unwrap()) - .is_err() - ); + assert!(multiaddr_to_socketaddr::( + &"/ip4/127.0.0.1/udp/1234".parse::().unwrap() + ) + .is_err()); assert_eq!( - multiaddr_to_socketaddr( + multiaddr_to_socketaddr::( &"/ip4/127.0.0.1/udp/12345/quic" .parse::() .unwrap() @@ -313,7 +309,7 @@ mod tests { )) ); assert_eq!( - multiaddr_to_socketaddr( + multiaddr_to_socketaddr::( &"/ip4/255.255.255.255/udp/8080/quic" .parse::() .unwrap() @@ -324,14 +320,14 @@ mod tests { )) ); assert_eq!( - multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), + multiaddr_to_socketaddr::(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), Ok(( SocketAddr::new(IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), 12345,), None )) ); assert_eq!( - multiaddr_to_socketaddr( + multiaddr_to_socketaddr::( &"/ip6/ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/udp/8080/quic" .parse::() .unwrap() @@ -347,4 +343,43 @@ mod tests { )) ); } + + #[cfg(feature = "noise")] + #[test] + fn multiaddr_to_udp_noise() { + multiaddr_to_udp_conversion::(); + } + #[cfg(feature = "tls")] + #[test] + fn multiaddr_to_udp_tls() { + multiaddr_to_udp_conversion::(); + } + + fn multiaddr_to_pk_conversion(keypair: C::Keypair) { + use crate::crypto::ToLibp2p; + use std::net::Ipv4Addr; + + let peer_id = keypair.to_public().to_peer_id(); + let addr = String::from("/ip4/127.0.0.1/udp/12345/quic/p2p/") + &peer_id.to_base58(); + assert_eq!( + multiaddr_to_socketaddr::(&addr.parse::().unwrap()), + Ok(( + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 12345,), + C::extract_public_key(keypair.to_public()) + )) + ); + } + + #[cfg(feature = "tls")] + #[test] + fn multiaddr_to_pk_tls() { + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); + multiaddr_to_pk_conversion::(keypair); + } + #[cfg(feature = "noise")] + #[test] + fn multiaddr_to_pk_noise() { + let keypair = ed25519_dalek::Keypair::generate(&mut rand_core::OsRng {}); + multiaddr_to_pk_conversion::(keypair); + } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 84a6a647d37..3ee821e956a 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -10,11 +10,20 @@ use libp2p::request_response::{ RequestResponseEvent, RequestResponseMessage, }; use libp2p::swarm::{Swarm, SwarmEvent}; -use libp2p_quic::{Crypto, Keypair, QuicConfig, ToLibp2p}; +use libp2p_quic::{Crypto, QuicConfig, ToLibp2p}; use quinn_proto::crypto::Session; use rand::RngCore; use std::{io, iter}; +#[cfg(feature = "noise")] +fn generate_noise_keypair() -> ed25519_dalek::Keypair { + ed25519_dalek::Keypair::generate(&mut rand_core::OsRng {}) +} +#[cfg(feature = "tls")] +fn generate_tls_keypair() -> libp2p::identity::Keypair { + libp2p::identity::Keypair::generate_ed25519() +} + #[cfg(feature = "noise")] #[async_std::test] async fn smoke_noise() -> Result<()> { @@ -27,13 +36,33 @@ async fn smoke_tls() -> Result<()> { smoke::().await } -async fn create_swarm(keylog: bool) -> Result>> +trait GenerateKeypair: Crypto { + fn generate_keypair() -> Self::Keypair; +} + +#[cfg(feature = "noise")] +impl GenerateKeypair for libp2p_quic::NoiseCrypto { + fn generate_keypair() -> Self::Keypair { + generate_noise_keypair() + } +} + +#[cfg(feature = "tls")] +impl GenerateKeypair for libp2p_quic::TlsCrypto { + fn generate_keypair() -> Self::Keypair { + generate_tls_keypair() + } +} + +async fn create_swarm( + keylog: bool, +) -> Result>> where ::ClientConfig: Send + Unpin, ::HeaderKey: Unpin, ::PacketKey: Unpin, { - let keypair = Keypair::generate(&mut rand_core::OsRng {}); + let keypair = C::generate_keypair(); let peer_id = keypair.to_peer_id(); let mut transport = QuicConfig::::new(keypair); if keylog { @@ -51,7 +80,7 @@ where Ok(Swarm::new(transport, behaviour, peer_id)) } -async fn smoke() -> Result<()> +async fn smoke() -> Result<()> where ::ClientConfig: Send + Unpin, ::HeaderKey: Unpin, @@ -276,7 +305,7 @@ async fn dial_failure_noise() -> Result<()> { Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; - let keypair = Keypair::generate(&mut rand_core::OsRng {}); + let keypair = libp2p_quic::NoiseCrypto::generate_keypair(); let fake_peer_id = keypair.to_peer_id(); let mut addr = match a.next().await { @@ -323,7 +352,7 @@ async fn dial_failure_tls() -> Result<()> { Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; - let keypair = Keypair::generate(&mut rand_core::OsRng {}); + let keypair = libp2p_quic::TlsCrypto::generate_keypair(); let fake_peer_id = keypair.to_peer_id(); let mut addr = match a.next().await { @@ -356,6 +385,7 @@ async fn dial_failure_tls() -> Result<()> { e => panic!("{:?}", e), }; + assert!(a.next().await.is_some()); // ConnectionClosed assert!(a.next().now_or_never().is_none()); Ok(()) From 1eface67e1529e608a731348912f46d91752f53c Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Mon, 6 Sep 2021 13:12:40 +0300 Subject: [PATCH 07/14] Don't require critical libp2p ext --- transports/quic/src/tls/verifier.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index a82bc39575e..9dcf2ed2298 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -174,7 +174,7 @@ fn parse_certificate( .iterate(&mut |oid, critical, extension| { match oid { super::LIBP2P_OID_BYTES if libp2p_extension.is_some() => return Err(Error::BadDER), - super::LIBP2P_OID_BYTES if critical == true => { + super::LIBP2P_OID_BYTES => { libp2p_extension = Some(parse_libp2p_extension(extension)?) } _ if critical => return Err(Error::UnsupportedCriticalExtension), From f760494d38db18da4fee2c8cabfb8a89128b0447 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 8 Sep 2021 14:34:29 +0300 Subject: [PATCH 08/14] Use std HashMap instead of fnv's --- transports/quic/Cargo.toml | 1 - transports/quic/src/endpoint.rs | 9 ++++----- transports/quic/src/muxer.rs | 5 ++--- 3 files changed, 6 insertions(+), 9 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 999297639fb..f02545e13dd 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -17,7 +17,6 @@ async-io = "1.6.0" barebones-x509 = { version = "0.5.0", optional = true, features = ["webpki", "rustls", "std"] } bytes = "1.0.1" ed25519-dalek = { version = "1.0.1", optional = true } -fnv = "1.0.7" futures = "0.3.15" if-watch = "0.2.2" libp2p-core = { version = "0.30.0", path = "../../core" } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index ba38771dc7c..5876beb48b7 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -1,7 +1,6 @@ use crate::crypto::{Crypto, CryptoConfig}; use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; -use fnv::FnvHashMap; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; use quinn_proto::crypto::Session; @@ -9,7 +8,7 @@ use quinn_proto::generic::{ClientConfig, ServerConfig}; use quinn_proto::{ ConnectionEvent, ConnectionHandle, DatagramEvent, EcnCodepoint, EndpointEvent, Transmit, }; -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::io::IoSliceMut; use std::mem::MaybeUninit; use std::net::SocketAddr; @@ -230,7 +229,7 @@ struct Endpoint { endpoint: QuinnEndpoint, socket: UdpSocket, crypto_config: Arc>, - connections: FnvHashMap>, + connections: HashMap>, outgoing: VecDeque, recv_buf: Box<[u8]>, incoming_slot: Option>, @@ -391,8 +390,8 @@ where } while !me.outgoing.is_empty() { - me.outgoing.make_contiguous(); - match me.socket.poll_send(cx, me.outgoing.as_slices().0) { + let transmits: &[_] = me.outgoing.make_contiguous(); + match me.socket.poll_send(cx, transmits) { Poll::Ready(Ok(n)) => { me.outgoing.drain(..n); } diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 794f7394a76..29f6b2a3ced 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -1,7 +1,6 @@ use crate::crypto::Crypto; use crate::endpoint::ConnectionChannel; use async_io::Timer; -use fnv::FnvHashMap; use futures::prelude::*; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use libp2p_core::{Multiaddr, PeerId}; @@ -11,7 +10,7 @@ use quinn_proto::{ ConnectionError, Dir, Event, FinishError, ReadError, ReadableError, StreamEvent, StreamId, VarInt, WriteError, }; -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::io::Write; use std::net::{IpAddr, Ipv4Addr, SocketAddr}; use std::pin::Pin; @@ -43,7 +42,7 @@ struct QuicMuxerInner { /// Connection timer. timer: Option, /// State of all open substreams. - substreams: FnvHashMap, + substreams: HashMap, /// Pending substreams. pending_substreams: VecDeque, /// Close waker. From 9a88cf280846bc06ecd850d9a96b0b7787c9cc0a Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 8 Sep 2021 15:16:02 +0300 Subject: [PATCH 09/14] Remove ToLibp2p to_keypair --- transports/quic/src/crypto.rs | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index ee1ec0321f5..81d1c8193b3 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -23,7 +23,6 @@ impl CloneKeypair for ed25519_dalek::Keypair { } pub trait ToLibp2p { - fn to_keypair(&self) -> libp2p_core::identity::Keypair; fn to_public(&self) -> libp2p_core::identity::PublicKey; fn to_peer_id(&self) -> PeerId { self.to_public().to_peer_id() @@ -32,13 +31,6 @@ pub trait ToLibp2p { #[cfg(feature = "noise")] impl ToLibp2p for ed25519_dalek::Keypair { - fn to_keypair(&self) -> libp2p_core::identity::Keypair { - let mut secret_key = self.secret.to_bytes(); - let secret_key = - libp2p_core::identity::ed25519::SecretKey::from_bytes(&mut secret_key).unwrap(); - libp2p_core::identity::Keypair::Ed25519(secret_key.into()) - } - fn to_public(&self) -> libp2p_core::identity::PublicKey { self.public.to_public() } @@ -46,10 +38,6 @@ impl ToLibp2p for ed25519_dalek::Keypair { #[cfg(feature = "noise")] impl ToLibp2p for ed25519_dalek::PublicKey { - fn to_keypair(&self) -> libp2p_core::identity::Keypair { - panic!("wtf?"); - } - fn to_public(&self) -> libp2p_core::identity::PublicKey { let public_key = self.to_bytes(); let public_key = @@ -60,10 +48,6 @@ impl ToLibp2p for ed25519_dalek::PublicKey { #[cfg(feature = "tls")] impl ToLibp2p for libp2p_core::identity::Keypair { - fn to_keypair(&self) -> libp2p_core::identity::Keypair { - self.clone() - } - fn to_public(&self) -> libp2p_core::identity::PublicKey { self.public() } From 35ac0fb946204fd7a19ab438e302477c14cec95d Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 9 Sep 2021 15:51:49 +0300 Subject: [PATCH 10/14] Move iovs init into a separate function --- transports/quic/src/endpoint.rs | 23 +++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 5876beb48b7..ca56cf39d30 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -403,17 +403,28 @@ where if me.event_slot.is_none() && me.incoming_slot.is_none() { let mut metas = [RecvMeta::default(); BATCH_SIZE]; let mut iovs = MaybeUninit::<[IoSliceMut; BATCH_SIZE]>::uninit(); - me.recv_buf - .chunks_mut(me.recv_buf.len() / BATCH_SIZE) - .enumerate() - .for_each(|(i, buf)| unsafe { + // SAFETY: safe as long as iovs do not outlive recv_buf + // and there exist only one iovs initialized with the given recv_buf. + // Also recv_buf should not be moved while there exist initialized iovs. + unsafe fn init_iovs<'a, 'b>( + iovs: &'b mut MaybeUninit<[IoSliceMut<'b>; BATCH_SIZE]>, + recv_buf: &'a mut [u8], + ) -> &'b mut [IoSliceMut<'b>] { + let chunk_size = recv_buf.len() / BATCH_SIZE; + let chunks = recv_buf.chunks_mut(chunk_size); + // every iovs elem must be initialized with an according elem from buf chunks + assert_eq!(chunks.len(), BATCH_SIZE); + chunks.enumerate().for_each(|(i, buf)| unsafe { iovs.as_mut_ptr() .cast::() .add(i) .write(IoSliceMut::new(buf)); }); - let mut iovs = unsafe { iovs.assume_init() }; - while let Poll::Ready(result) = me.socket.poll_recv(cx, &mut iovs, &mut metas) { + + iovs.assume_init_mut() + } + let iovs = unsafe { init_iovs(&mut iovs, &mut me.recv_buf) }; + while let Poll::Ready(result) = me.socket.poll_recv(cx, iovs, &mut metas) { let n = match result { Ok(n) => n, Err(err) => { From d665212060ecc92dfccf2731ac3bae8fba8d4f1a Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Sat, 11 Sep 2021 12:11:47 +0300 Subject: [PATCH 11/14] Use mem::replace to unbind recv_buf --- transports/quic/src/endpoint.rs | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index ca56cf39d30..6056d1e255a 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -403,13 +403,10 @@ where if me.event_slot.is_none() && me.incoming_slot.is_none() { let mut metas = [RecvMeta::default(); BATCH_SIZE]; let mut iovs = MaybeUninit::<[IoSliceMut; BATCH_SIZE]>::uninit(); - // SAFETY: safe as long as iovs do not outlive recv_buf - // and there exist only one iovs initialized with the given recv_buf. - // Also recv_buf should not be moved while there exist initialized iovs. - unsafe fn init_iovs<'a, 'b>( - iovs: &'b mut MaybeUninit<[IoSliceMut<'b>; BATCH_SIZE]>, - recv_buf: &'a mut [u8], - ) -> &'b mut [IoSliceMut<'b>] { + fn init_iovs<'a>( + iovs: &'a mut MaybeUninit<[IoSliceMut<'a>; BATCH_SIZE]>, + recv_buf: &mut [u8], + ) -> &'a mut [IoSliceMut<'a>] { let chunk_size = recv_buf.len() / BATCH_SIZE; let chunks = recv_buf.chunks_mut(chunk_size); // every iovs elem must be initialized with an according elem from buf chunks @@ -421,9 +418,13 @@ where .write(IoSliceMut::new(buf)); }); - iovs.assume_init_mut() + unsafe { + // SAFETY: all elements are initialized + iovs.assume_init_mut() + } } - let iovs = unsafe { init_iovs(&mut iovs, &mut me.recv_buf) }; + let mut recv_buf = core::mem::replace(&mut me.recv_buf, Vec::new().into_boxed_slice()); + let iovs = init_iovs(&mut iovs, &mut recv_buf); while let Poll::Ready(result) = me.socket.poll_recv(cx, iovs, &mut metas) { let n = match result { Ok(n) => n, @@ -462,6 +463,7 @@ where } } } + me.recv_buf = recv_buf; } Poll::Pending From ee7a7f599aa4a584c5a6919db1ba23633a657667 Mon Sep 17 00:00:00 2001 From: David Craven Date: Sat, 11 Sep 2021 11:45:04 +0200 Subject: [PATCH 12/14] Update transports/quic/src/endpoint.rs --- transports/quic/src/endpoint.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 6056d1e255a..3295b5bbf58 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -405,7 +405,7 @@ where let mut iovs = MaybeUninit::<[IoSliceMut; BATCH_SIZE]>::uninit(); fn init_iovs<'a>( iovs: &'a mut MaybeUninit<[IoSliceMut<'a>; BATCH_SIZE]>, - recv_buf: &mut [u8], + recv_buf: &'a mut [u8], ) -> &'a mut [IoSliceMut<'a>] { let chunk_size = recv_buf.len() / BATCH_SIZE; let chunks = recv_buf.chunks_mut(chunk_size); From aee258ced57f2720ae62eadf2390fbf60b709fc7 Mon Sep 17 00:00:00 2001 From: Roman Date: Tue, 14 Sep 2021 18:11:14 +0300 Subject: [PATCH 13/14] Merge master (#18) * protocols/gossipsub: Fix inconsistency in mesh peer tracking (#2189) Co-authored-by: Age Manning * misc/metrics: Add auxiliary crate to record events as OpenMetrics (#2063) This commit adds an auxiliary crate recording protocol and Swarm events and exposing them as metrics in the OpenMetrics format. * README: Mention security@ipfs.io * examples/: Add file sharing example (#2186) Basic file sharing application with peers either providing or locating and getting files by name. While obviously showcasing how to build a basic file sharing application, the actual goal of this example is **to show how to integrate rust-libp2p into a larger application**. Architectural properties - Clean clonable async/await interface ([`Client`]) to interact with the network layer. - Single task driving the network layer, no locks required. * examples/README: Give an overview over the many examples (#2194) * protocols/kad: Enable filtering of (provider) records (#2163) Introduce `KademliaStoreInserts` option, which allows to filter records. Co-authored-by: Max Inden * swarm/src/protocols_handler: Impl ProtocolsHandler on either::Either (#2192) Implement ProtocolsHandler on either::Either representing either of two ProtocolsHandler implementations. Co-authored-by: Thomas Eizinger * *: Make libp2p-core default features optional (#2181) Co-authored-by: Max Inden * core/: Remove DisconnectedPeer::set_connected and Pool::add (#2195) This logic seems to be a leftover of https://github.com/libp2p/rust-libp2p/pull/889 and unused today. * protocols/gossipsub: Use ed25519 in tests (#2197) With f2905c07f1246c3c3fdc1cde95f7e9c5c1c9b01a the secp256k1 feature is disabled by default. Instead of enabling it in the dev-dependency, simply use ed25519. * build(deps): Update minicbor requirement from 0.10 to 0.11 (#2200) Updates the requirements on [minicbor](https://gitlab.com/twittner/minicbor) to permit the latest version. - [Release notes](https://gitlab.com/twittner/minicbor/tags) - [Changelog](https://gitlab.com/twittner/minicbor/blob/master/CHANGELOG.md) - [Commits](https://gitlab.com/twittner/minicbor/compare/minicbor-v0.10.0...minicbor-v0.11.0) --- updated-dependencies: - dependency-name: minicbor dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> * build(deps): Update salsa20 requirement from 0.8 to 0.9 (#2206) * build(deps): Update salsa20 requirement from 0.8 to 0.9 Updates the requirements on [salsa20](https://github.com/RustCrypto/stream-ciphers) to permit the latest version. - [Release notes](https://github.com/RustCrypto/stream-ciphers/releases) - [Commits](https://github.com/RustCrypto/stream-ciphers/compare/ctr-v0.8.0...salsa20-v0.9.0) --- updated-dependencies: - dependency-name: salsa20 dependency-type: direct:production ... Signed-off-by: dependabot[bot] * *: Bump pnet to v0.22 Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Max Inden * *: 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 * core/: Remove deprecated read/write functions (#2213) Co-authored-by: Max Inden * protocols/ping: Revise naming of symbols (#2215) Co-authored-by: Max Inden * protocols/rendezvous: Implement protocol (#2107) Implement the libp2p rendezvous protocol. > A lightweight mechanism for generalized peer discovery. It can be used for bootstrap purposes, real time peer discovery, application specific routing, and so on. Co-authored-by: rishflab Co-authored-by: Daniel Karzel * core/src/network/event.rs: Fix typo (#2218) * protocols/mdns: Do not fire all timers at the same time. (#2212) Co-authored-by: Max Inden * misc/metrics/src/kad: Set query_duration lowest bucket to 0.1 sec (#2219) Probability for a Kademlia query to return in less than 100 milliseconds is low, thus increasing the lower bucket to improve accuracy within the higher ranges. * misc/metrics/src/swarm: Expose role on connections_closed (#2220) Expose whether closed connection was a Dialer or Listener. * .github/workflows/ci.yml: Use clang 11 (#2233) * protocols/rendezvous: Update prost (#2226) Co-authored-by: Max Inden * *: Fix clippy warnings (#2227) * swarm-derive/: Make event_process = false the default (#2214) Co-authored-by: Max Inden Co-authored-by: Max Inden Co-authored-by: Age Manning Co-authored-by: Ruben De Smet Co-authored-by: Thomas Eizinger Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: rishflab Co-authored-by: Daniel Karzel Co-authored-by: David Craven --- .github/workflows/ci.yml | 4 +- CHANGELOG.md | 6 +- Cargo.toml | 19 +- README.md | 5 +- core/CHANGELOG.md | 22 + core/build.rs | 10 +- core/src/connection.rs | 8 +- core/src/connection/error.rs | 21 +- core/src/connection/manager.rs | 55 +- core/src/connection/manager/task.rs | 73 +- core/src/connection/pool.rs | 144 +--- core/src/envelope.proto | 30 + core/src/identity.rs | 57 +- core/src/identity/rsa.rs | 10 +- core/src/lib.rs | 12 + core/src/network.rs | 103 ++- core/src/network/event.rs | 23 +- core/src/network/peer.rs | 53 +- core/src/peer_id.rs | 2 +- core/src/peer_record.proto | 27 + core/src/peer_record.rs | 199 +++++ core/src/signed_envelope.rs | 203 +++++ core/src/upgrade.rs | 2 - core/src/upgrade/transfer.rs | 110 +-- core/tests/connection_limits.rs | 12 +- core/tests/network_dial_error.rs | 7 +- examples/README.md | 51 ++ examples/chat-tokio.rs | 1 + examples/chat.rs | 78 +- examples/distributed-key-value-store.rs | 1 + examples/file-sharing.rs | 733 +++++++++++++++++ examples/ipfs-private.rs | 35 +- examples/ping.rs | 5 +- misc/metrics/CHANGELOG.md | 3 + misc/metrics/Cargo.toml | 29 + misc/metrics/examples/metrics.rs | 115 +++ misc/metrics/src/identify.rs | 122 +++ misc/metrics/src/kad.rs | 427 ++++++++++ misc/metrics/src/lib.rs | 76 ++ misc/metrics/src/ping.rs | 109 +++ misc/metrics/src/swarm.rs | 267 ++++++ misc/multistream-select/Cargo.toml | 2 +- .../src/length_delimited.rs | 2 +- misc/peer-id-generator/Cargo.toml | 2 +- muxers/mplex/CHANGELOG.md | 2 + muxers/mplex/Cargo.toml | 2 +- muxers/mplex/src/io.rs | 1 - muxers/yamux/CHANGELOG.md | 3 + muxers/yamux/Cargo.toml | 2 +- protocols/floodsub/CHANGELOG.md | 3 + protocols/floodsub/Cargo.toml | 2 +- protocols/floodsub/src/layer.rs | 20 +- protocols/gossipsub/CHANGELOG.md | 6 + protocols/gossipsub/Cargo.toml | 2 +- protocols/gossipsub/src/behaviour.rs | 186 ++--- protocols/gossipsub/src/behaviour/tests.rs | 42 +- protocols/gossipsub/src/protocol.rs | 8 +- protocols/identify/CHANGELOG.md | 3 + protocols/identify/Cargo.toml | 2 +- protocols/identify/src/identify.rs | 19 +- protocols/identify/src/protocol.rs | 2 +- protocols/kad/CHANGELOG.md | 8 + protocols/kad/Cargo.toml | 2 +- protocols/kad/src/behaviour.rs | 150 +++- protocols/kad/src/behaviour/test.rs | 64 +- protocols/kad/src/handler.rs | 48 +- protocols/kad/src/kbucket/entry.rs | 2 +- protocols/kad/src/lib.rs | 2 +- protocols/kad/src/query/peers/fixed.rs | 1 + protocols/mdns/CHANGELOG.md | 8 +- protocols/mdns/Cargo.toml | 2 +- protocols/mdns/src/behaviour.rs | 40 +- protocols/mdns/src/dns.rs | 4 +- protocols/ping/CHANGELOG.md | 11 + protocols/ping/Cargo.toml | 2 +- protocols/ping/src/handler.rs | 71 +- protocols/ping/src/lib.rs | 65 +- protocols/ping/tests/ping.rs | 44 +- protocols/relay/CHANGELOG.md | 3 + protocols/relay/Cargo.toml | 2 +- protocols/relay/examples/relay.rs | 19 +- protocols/relay/src/behaviour.rs | 31 +- .../relay/src/protocol/incoming_dst_req.rs | 5 +- protocols/relay/tests/lib.rs | 34 +- protocols/rendezvous/CHANGELOG.md | 3 + protocols/rendezvous/Cargo.toml | 36 + protocols/rendezvous/build.rs | 3 + protocols/rendezvous/examples/discover.rs | 140 ++++ protocols/rendezvous/examples/register.rs | 139 ++++ .../examples/register_with_identify.rs | 145 ++++ .../rendezvous/examples/rendezvous_point.rs | 112 +++ protocols/rendezvous/src/client.rs | 337 ++++++++ protocols/rendezvous/src/codec.rs | 622 ++++++++++++++ protocols/rendezvous/src/handler.rs | 48 ++ protocols/rendezvous/src/handler/inbound.rs | 189 +++++ protocols/rendezvous/src/handler/outbound.rs | 132 +++ protocols/rendezvous/src/lib.rs | 43 + protocols/rendezvous/src/rpc.proto | 61 ++ protocols/rendezvous/src/server.rs | 764 ++++++++++++++++++ protocols/rendezvous/src/substream_handler.rs | 551 +++++++++++++ protocols/rendezvous/tests/harness/mod.rs | 221 +++++ protocols/rendezvous/tests/rendezvous.rs | 384 +++++++++ protocols/request-response/CHANGELOG.md | 3 + protocols/request-response/Cargo.toml | 5 +- protocols/request-response/src/lib.rs | 21 +- protocols/request-response/src/throttled.rs | 46 +- protocols/request-response/tests/ping.rs | 1 + src/lib.rs | 8 + swarm-derive/CHANGELOG.md | 8 + swarm-derive/Cargo.toml | 2 +- swarm-derive/src/lib.rs | 192 ++++- swarm-derive/tests/test.rs | 35 +- swarm/CHANGELOG.md | 33 +- swarm/Cargo.toml | 5 +- swarm/src/behaviour.rs | 362 ++++++++- swarm/src/lib.rs | 265 +++--- swarm/src/protocols_handler.rs | 8 +- swarm/src/protocols_handler/either.rs | 305 +++++++ swarm/src/protocols_handler/node_handler.rs | 10 + swarm/src/protocols_handler/select.rs | 8 + swarm/src/registry.rs | 2 +- swarm/src/test.rs | 36 +- swarm/src/toggle.rs | 44 +- transports/deflate/CHANGELOG.md | 3 + transports/deflate/Cargo.toml | 2 +- transports/dns/CHANGELOG.md | 3 + transports/dns/Cargo.toml | 2 +- transports/dns/src/lib.rs | 2 +- transports/noise/CHANGELOG.md | 3 + transports/noise/Cargo.toml | 2 +- transports/noise/src/io.rs | 4 +- transports/noise/src/protocol/x25519.rs | 2 +- transports/noise/src/protocol/x25519_spec.rs | 2 +- transports/plaintext/CHANGELOG.md | 3 + transports/plaintext/Cargo.toml | 2 +- transports/pnet/CHANGELOG.md | 4 + transports/pnet/Cargo.toml | 4 +- transports/tcp/CHANGELOG.md | 3 + transports/tcp/Cargo.toml | 2 +- transports/uds/CHANGELOG.md | 3 + transports/uds/Cargo.toml | 2 +- transports/wasm-ext/CHANGELOG.md | 3 + transports/wasm-ext/Cargo.toml | 2 +- transports/websocket/CHANGELOG.md | 3 + transports/websocket/Cargo.toml | 2 +- transports/websocket/src/framed.rs | 18 +- 146 files changed, 8475 insertions(+), 1128 deletions(-) create mode 100644 core/src/envelope.proto create mode 100644 core/src/peer_record.proto create mode 100644 core/src/peer_record.rs create mode 100644 core/src/signed_envelope.rs create mode 100644 examples/README.md create mode 100644 examples/file-sharing.rs create mode 100644 misc/metrics/CHANGELOG.md create mode 100644 misc/metrics/Cargo.toml create mode 100644 misc/metrics/examples/metrics.rs create mode 100644 misc/metrics/src/identify.rs create mode 100644 misc/metrics/src/kad.rs create mode 100644 misc/metrics/src/lib.rs create mode 100644 misc/metrics/src/ping.rs create mode 100644 misc/metrics/src/swarm.rs create mode 100644 protocols/rendezvous/CHANGELOG.md create mode 100644 protocols/rendezvous/Cargo.toml create mode 100644 protocols/rendezvous/build.rs create mode 100644 protocols/rendezvous/examples/discover.rs create mode 100644 protocols/rendezvous/examples/register.rs create mode 100644 protocols/rendezvous/examples/register_with_identify.rs create mode 100644 protocols/rendezvous/examples/rendezvous_point.rs create mode 100644 protocols/rendezvous/src/client.rs create mode 100644 protocols/rendezvous/src/codec.rs create mode 100644 protocols/rendezvous/src/handler.rs create mode 100644 protocols/rendezvous/src/handler/inbound.rs create mode 100644 protocols/rendezvous/src/handler/outbound.rs create mode 100644 protocols/rendezvous/src/lib.rs create mode 100644 protocols/rendezvous/src/rpc.proto create mode 100644 protocols/rendezvous/src/server.rs create mode 100644 protocols/rendezvous/src/substream_handler.rs create mode 100644 protocols/rendezvous/tests/harness/mod.rs create mode 100644 protocols/rendezvous/tests/rendezvous.rs create mode 100644 swarm/src/protocols_handler/either.rs diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 9dff2d46700..544f844ea63 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -46,7 +46,7 @@ jobs: container: image: rust env: - CC: clang-10 + CC: clang-11 steps: - name: Cancel Previous Runs @@ -75,7 +75,7 @@ jobs: wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add - echo "deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-10 main" >> /etc/apt/sources.list apt-get update - apt-get install -y clang-10 + apt-get install -y clang-11 - name: Install CMake run: apt-get install -y cmake diff --git a/CHANGELOG.md b/CHANGELOG.md index b5e3e828d81..c28e4afe1e8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,7 @@ - [`libp2p-ping` CHANGELOG](protocols/ping/CHANGELOG.md) - [`libp2p-relay` CHANGELOG](protocols/relay/CHANGELOG.md) - [`libp2p-request-response` CHANGELOG](protocols/request-response/CHANGELOG.md) +- [`libp2p-rendezvous` CHANGELOG](protocols/rendezvous/CHANGELOG.md) ## Transport Protocols & Upgrades @@ -36,6 +37,7 @@ ## Utilities +- [`libp2p-metrics` CHANGELOG](misc/metrics/CHANGELOG.md) - [`multistream-select` CHANGELOG](misc/multistream-select/CHANGELOG.md) # `libp2p` facade crate @@ -66,7 +68,9 @@ - Re-export the `wasm-bindgen` feature from `parking_lot`, so `libp2p` users can opt-in to that crate's Wasm support. See [PR 2180]. - + +- Add `libp2p-metrics`. + [PR 2180]: https://github.com/libp2p/rust-libp2p/pull/2180/ ## Version 0.39.1 [2021-07-12] diff --git a/Cargo.toml b/Cargo.toml index e02f558c22c..eac6e34607a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,6 +26,7 @@ default = [ "quic", "relay", "request-response", + "rendezvous", "secp256k1", "tcp-async-io", "uds", @@ -38,17 +39,19 @@ dns-async-std = ["libp2p-dns", "libp2p-dns/async-std"] dns-tokio = ["libp2p-dns", "libp2p-dns/tokio"] floodsub = ["libp2p-floodsub"] gossipsub = ["libp2p-gossipsub"] -identify = ["libp2p-identify"] -kad = ["libp2p-kad"] +identify = ["libp2p-identify", "libp2p-metrics/identify"] +kad = ["libp2p-kad", "libp2p-metrics/kad"] +metrics = ["libp2p-metrics"] mdns = ["libp2p-mdns"] mplex = ["libp2p-mplex"] noise = ["libp2p-noise"] -ping = ["libp2p-ping"] +ping = ["libp2p-ping", "libp2p-metrics/ping"] plaintext = ["libp2p-plaintext"] pnet = ["libp2p-pnet"] quic = ["libp2p-quic"] relay = ["libp2p-relay"] request-response = ["libp2p-request-response"] +rendezvous = ["libp2p-rendezvous"] tcp-async-io = ["libp2p-tcp", "libp2p-tcp/async-io"] tcp-tokio = ["libp2p-tcp", "libp2p-tcp/tokio"] uds = ["libp2p-uds"] @@ -72,15 +75,17 @@ libp2p-floodsub = { version = "0.31.0", path = "protocols/floodsub", optional = libp2p-gossipsub = { version = "0.33.0", path = "./protocols/gossipsub", optional = true } libp2p-identify = { version = "0.31.0", path = "protocols/identify", optional = true } libp2p-kad = { version = "0.32.0", path = "protocols/kad", optional = true } +libp2p-metrics = { version = "0.1.0", path = "misc/metrics", optional = true } libp2p-mplex = { version = "0.30.0", path = "muxers/mplex", optional = true } libp2p-noise = { version = "0.33.0", path = "transports/noise", optional = true } libp2p-ping = { version = "0.31.0", path = "protocols/ping", optional = true } libp2p-plaintext = { version = "0.30.0", path = "transports/plaintext", optional = true } -libp2p-pnet = { version = "0.21.0", path = "transports/pnet", optional = true } +libp2p-pnet = { version = "0.22.0", path = "transports/pnet", optional = true } libp2p-relay = { version = "0.4.0", path = "protocols/relay", optional = true } +libp2p-rendezvous = { version = "0.1.0", path = "protocols/rendezvous", optional = true } libp2p-request-response = { version = "0.13.0", path = "protocols/request-response", optional = true } libp2p-swarm = { version = "0.31.0", path = "swarm" } -libp2p-swarm-derive = { version = "0.24.0", path = "swarm-derive" } +libp2p-swarm-derive = { version = "0.25.0", path = "swarm-derive" } libp2p-uds = { version = "0.30.0", path = "transports/uds", optional = true } libp2p-wasm-ext = { version = "0.30.0", path = "transports/wasm-ext", default-features = false, optional = true } libp2p-yamux = { version = "0.34.0", path = "muxers/yamux", optional = true } @@ -100,19 +105,23 @@ libp2p-websocket = { version = "0.31.0", path = "transports/websocket", optional [dev-dependencies] async-std = { version = "1.6.2", features = ["attributes"] } +async-trait = "0.1" env_logger = "0.9.0" +structopt = "0.3.21" tokio = { version = "1.0.1", features = ["io-util", "io-std", "macros", "rt", "rt-multi-thread"] } [workspace] resolver = "2" members = [ "core", + "misc/metrics", "misc/multistream-select", "misc/peer-id-generator", "muxers/mplex", "muxers/yamux", "protocols/floodsub", "protocols/gossipsub", + "protocols/rendezvous", "protocols/identify", "protocols/kad", "protocols/mdns", diff --git a/README.md b/README.md index 7c0fd2e9200..e4ca3fc36e8 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,10 @@ This repository is the central place for Rust development of the [libp2p](https: - The **[examples](examples)** folder contains small binaries showcasing the many protocols in this repository. -- To **report bugs, suggest improvements or request new features** please open +- For **security related issues** please reach out to security@ipfs.io. Please + do not file a public issue on GitHub. + +- To **report bugs, suggest improvements or request new features** please open a GitHub issue on this repository. - For **rust-libp2p specific questions** please use the GitHub _Discussions_ diff --git a/core/CHANGELOG.md b/core/CHANGELOG.md index 695d1522ea9..1c4ace6835f 100644 --- a/core/CHANGELOG.md +++ b/core/CHANGELOG.md @@ -20,10 +20,32 @@ - Require `ConnectionHandler::{InEvent,OutEvent,Error}` to implement `Debug` (see [PR 2183]). +- Remove `DisconnectedPeer::set_connected` and `Pool::add` (see [PR 2195]). + +- Report `ConnectionLimit` error through `ConnectionError` and thus through + `NetworkEvent::ConnectionClosed` instead of previously through + `PendingConnectionError` and thus `NetworkEvent::{IncomingConnectionError, + DialError}` (see [PR 2191]). + +- Report abortion of pending connection through `DialError`, + `UnknownPeerDialError` or `IncomingConnectionError` (see [PR 2191]). + +- Remove deprecated functions `upgrade::write_one`, `upgrade::write_with_len_prefix` + and `upgrade::read_one` (see [PR 2213]). + +- Add `SignedEnvelope` and `PeerRecord` according to [RFC0002] and [RFC0003] + (see [PR 2107]). + [PR 2145]: https://github.com/libp2p/rust-libp2p/pull/2145 +[PR 2213]: https://github.com/libp2p/rust-libp2p/pull/2213 [PR 2142]: https://github.com/libp2p/rust-libp2p/pull/2142 [PR 2137]: https://github.com/libp2p/rust-libp2p/pull/2137 [PR 2183]: https://github.com/libp2p/rust-libp2p/pull/2183 +[PR 2191]: https://github.com/libp2p/rust-libp2p/pull/2191 +[PR 2195]: https://github.com/libp2p/rust-libp2p/pull/2195 +[PR 2107]: https://github.com/libp2p/rust-libp2p/pull/2107 +[RFC0002]: https://github.com/libp2p/specs/blob/master/RFC/0002-signed-envelopes.md +[RFC0003]: https://github.com/libp2p/specs/blob/master/RFC/0003-routing-records.md # 0.29.0 [2021-07-12] diff --git a/core/build.rs b/core/build.rs index 9692abd9c81..f0c09f93abf 100644 --- a/core/build.rs +++ b/core/build.rs @@ -19,5 +19,13 @@ // DEALINGS IN THE SOFTWARE. fn main() { - prost_build::compile_protos(&["src/keys.proto"], &["src"]).unwrap(); + prost_build::compile_protos( + &[ + "src/keys.proto", + "src/envelope.proto", + "src/peer_record.proto", + ], + &["src"], + ) + .unwrap(); } diff --git a/core/src/connection.rs b/core/src/connection.rs index 335e2046c2d..9e39ae21807 100644 --- a/core/src/connection.rs +++ b/core/src/connection.rs @@ -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 { - 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) { + (self.handler, self.muxing.close().0) } /// Polls the connection for events produced by the associated handler diff --git a/core/src/connection/error.rs b/core/src/connection/error.rs index 66da0670c98..ec4f7ff6e61 100644 --- a/core/src/connection/error.rs +++ b/core/src/connection/error.rs @@ -29,6 +29,10 @@ pub enum ConnectionError { // 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 { /// An error occurred while negotiating the transport protocol(s). Transport(TransportError), + /// 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, } } } diff --git a/core/src/connection/manager.rs b/core/src/connection/manager.rs index 1d7acb92e69..51c08024202 100644 --- a/core/src/connection/manager.rs +++ b/core/src/connection/manager.rs @@ -20,7 +20,7 @@ use super::{ handler::{THandlerError, THandlerInEvent, THandlerOutEvent}, - Connected, ConnectedPoint, Connection, ConnectionError, ConnectionHandler, + Connected, ConnectedPoint, ConnectionError, ConnectionHandler, ConnectionLimit, IntoConnectionHandler, PendingConnectionError, Substream, }; use crate::{muxing::StreamMuxer, Executor}; @@ -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>>, + handler: H::Handler, }, /// A connection has been established. @@ -276,40 +277,6 @@ impl Manager { ConnectionId(task_id) } - /// Adds an existing connection to the manager. - pub fn add(&mut self, conn: Connection, info: Connected) -> ConnectionId - where - H: IntoConnectionHandler + Send + 'static, - H::Handler: ConnectionHandler> + Send + 'static, - ::OutboundOpenInfo: Send + 'static, - TE: error::Error + Send + 'static, - M: StreamMuxer + Send + Sync + 'static, - M::OutboundSubstream: Send + 'static, - { - let task_id = self.next_task_id; - self.next_task_id.0 += 1; - - let (tx, rx) = mpsc::channel(self.task_command_buffer_size); - self.tasks.insert( - task_id, - TaskInfo { - sender: tx, - state: TaskState::Established(info), - }, - ); - - let task: Pin>>, _, _, _>>> = - Box::pin(Task::established(task_id, self.events_tx.clone(), rx, conn)); - - if let Some(executor) = &mut self.executor { - executor.exec(task); - } else { - self.local_spawns.push(task); - } - - ConnectionId(task_id) - } - /// Gets an entry for a managed connection, if it exists. pub fn entry(&mut self, id: ConnectionId) -> Option>> { if let hash_map::Entry::Occupied(task) = self.tasks.entry(id.0) { @@ -384,7 +351,7 @@ impl Manager { 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 { @@ -392,6 +359,7 @@ impl Manager { id, connected, error, + handler, }, TaskState::Pending => unreachable!( "`Event::Closed` implies (2) occurred on that task and thus (3)." @@ -471,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) { // Clone the sender so that we are guaranteed to have // capacity for the close command (every sender gets a slot). match self @@ -479,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."), @@ -494,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()) diff --git a/core/src/connection/manager/task.rs b/core/src/connection/manager/task.rs index db8fb43adb6..5860bcc6c9d 100644 --- a/core/src/connection/manager/task.rs +++ b/core/src/connection/manager/task.rs @@ -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 { NotifyHandler(T), /// Gracefully close the connection (active close) before /// terminating the task. - Close, + Close(Option), } /// Events that a task can emit to its manager. @@ -71,6 +71,7 @@ pub enum Event { Closed { id: TaskId, error: Option>>, + handler: H::Handler, }, } @@ -130,24 +131,6 @@ where }, } } - - /// Create a task for an existing node we are already connected to. - pub fn established( - id: TaskId, - events: mpsc::Sender>, - commands: mpsc::Receiver>>, - connection: Connection, - ) -> Self { - Task { - id, - events, - commands: commands.fuse(), - state: State::Established { - connection, - event: None, - }, - } - } } /// The state associated with the `Task` of a connection. @@ -177,7 +160,11 @@ where }, /// The connection is closing (active close). - Closing(Close), + Closing { + closing_muxer: Close, + handler: H::Handler, + error: Option, + }, /// The task is terminating with a final event for the `Manager`. Terminating(Event), @@ -222,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.") @@ -261,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(()); } } @@ -324,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); } @@ -335,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(ConnectionError::ConnectionLimit), + handler, }; this.state = State::Terminating(event); } @@ -349,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; } } diff --git a/core/src/connection/pool.rs b/core/src/connection/pool.rs index 9925dd526c0..f147f766d4d 100644 --- a/core/src/connection/pool.rs +++ b/core/src/connection/pool.rs @@ -20,13 +20,13 @@ use crate::{ connection::{ - self, handler::{THandlerError, THandlerInEvent, THandlerOutEvent}, manager::{self, Manager, ManagerConfig}, - Connected, Connection, ConnectionError, ConnectionHandler, ConnectionId, ConnectionLimit, - IncomingInfo, IntoConnectionHandler, OutgoingInfo, PendingConnectionError, Substream, + Connected, ConnectionError, ConnectionHandler, ConnectionId, ConnectionLimit, IncomingInfo, + IntoConnectionHandler, OutgoingInfo, PendingConnectionError, Substream, }, muxing::StreamMuxer, + network::DialError, ConnectedPoint, PeerId, }; use either::Either; @@ -54,12 +54,6 @@ pub struct Pool { /// The pending connections that are currently being negotiated. pending: FnvHashMap)>, - - /// 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, } impl fmt::Debug for Pool { @@ -102,6 +96,7 @@ pub enum PoolEvent<'a, THandler: IntoConnectionHandler, TTransErr> { pool: &'a mut Pool, /// The remaining number of established connections to the same peer. num_established: u32, + handler: THandler::Handler, }, /// A connection attempt failed. @@ -114,7 +109,7 @@ pub enum PoolEvent<'a, THandler: IntoConnectionHandler, TTransErr> { error: PendingConnectionError, /// The handler that was supposed to handle the connection, /// if the connection failed before the handler was consumed. - handler: Option, + handler: THandler, /// The (expected) peer of the failed connection. peer: Option, /// A reference to the pool that managed the connection. @@ -200,7 +195,6 @@ impl Pool { manager: Manager::new(manager_config), established: Default::default(), pending: Default::default(), - disconnected: Vec::new(), } } @@ -246,7 +240,7 @@ impl Pool { future: TFut, handler: THandler, info: OutgoingInfo<'_>, - ) -> Result + ) -> Result> where TFut: Future>> + Send @@ -258,7 +252,9 @@ impl Pool { 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())) } @@ -313,37 +309,6 @@ impl Pool { id } - /// Adds an existing established connection to the pool. - /// - /// Returns the assigned connection ID on success. An error is returned - /// if the configured maximum number of established connections for the - /// connected peer has been reached. - pub fn add( - &mut self, - c: Connection, - i: Connected, - ) -> Result - where - THandler: IntoConnectionHandler + Send + 'static, - THandler::Handler: - ConnectionHandler> + Send + 'static, - ::OutboundOpenInfo: Send + 'static, - TTransErr: error::Error + Send + 'static, - TMuxer: StreamMuxer + Send + Sync + 'static, - TMuxer::OutboundSubstream: Send + 'static, - { - self.counters.check_max_established(&i.endpoint)?; - self.counters - .check_max_established_per_peer(self.num_peer_established(&i.peer_id))?; - let id = self.manager.add(c, i.clone()); - self.counters.inc_established(&i.endpoint); - self.established - .entry(i.peer_id) - .or_default() - .insert(id, i.endpoint); - Ok(id) - } - /// Gets an entry representing a connection in the pool. /// /// Returns `None` if the pool has no connection with the given ID. @@ -411,45 +376,24 @@ impl Pool { /// (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. @@ -490,7 +434,7 @@ impl Pool { /// Returns an iterator for information on all pending outgoing connections. pub fn iter_pending_outgoing(&self) -> impl Iterator> { self.iter_pending_info() - .filter_map(|(_, ref endpoint, ref peer_id)| match endpoint { + .filter_map(|(_, ref endpoint, peer_id)| match endpoint { ConnectedPoint::Listener { .. } => None, ConnectedPoint::Dialer { address } => Some(OutgoingInfo { address, @@ -535,28 +479,6 @@ impl Pool { &'a mut self, cx: &mut Context<'_>, ) -> Poll> { - // 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) { @@ -572,7 +494,7 @@ impl Pool { id, endpoint, error, - handler: Some(handler), + handler, peer, pool: self, }); @@ -582,6 +504,7 @@ impl Pool { id, connected, error, + handler, } => { let num_established = if let Some(conns) = self.established.get_mut(&connected.peer_id) { @@ -601,6 +524,7 @@ impl Pool { error, num_established, pool: self, + handler, }); } manager::Event::ConnectionEstablished { entry } => { @@ -610,30 +534,16 @@ impl Pool { // 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`. @@ -822,7 +732,7 @@ impl 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) } } @@ -1096,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, -} diff --git a/core/src/envelope.proto b/core/src/envelope.proto new file mode 100644 index 00000000000..9ab3e6fd256 --- /dev/null +++ b/core/src/envelope.proto @@ -0,0 +1,30 @@ +syntax = "proto3"; + +package envelope_proto; + +import "keys.proto"; + +// Envelope encloses a signed payload produced by a peer, along with the public +// key of the keypair it was signed with so that it can be statelessly validated +// by the receiver. +// +// The payload is prefixed with a byte string that determines the type, so it +// can be deserialized deterministically. Often, this byte string is a +// multicodec. +message Envelope { + // public_key is the public key of the keypair the enclosed payload was + // signed with. + keys_proto.PublicKey public_key = 1; + + // payload_type encodes the type of payload, so that it can be deserialized + // deterministically. + bytes payload_type = 2; + + // payload is the actual payload carried inside this envelope. + bytes payload = 3; + + // signature is the signature produced by the private key corresponding to + // the enclosed public key, over the payload, prefixing a domain string for + // additional security. + bytes signature = 5; +} diff --git a/core/src/identity.rs b/core/src/identity.rs index 8c3c83db16e..76ed4c39fd4 100644 --- a/core/src/identity.rs +++ b/core/src/identity.rs @@ -42,6 +42,7 @@ pub mod error; use self::error::*; use crate::{keys_proto, PeerId}; +use std::convert::{TryFrom, TryInto}; /// Identity keypair of a node. /// @@ -205,6 +206,7 @@ impl PublicKey { /// that the signature has been produced by the corresponding /// private key (authenticity), and that the message has not been /// tampered with (integrity). + #[must_use] pub fn verify(&self, msg: &[u8], sig: &[u8]) -> bool { use PublicKey::*; match self { @@ -221,7 +223,35 @@ impl PublicKey { pub fn to_protobuf_encoding(&self) -> Vec { use prost::Message; - let public_key = match self { + let public_key = keys_proto::PublicKey::from(self); + + let mut buf = Vec::with_capacity(public_key.encoded_len()); + public_key + .encode(&mut buf) + .expect("Vec provides capacity as needed"); + buf + } + + /// Decode a public key from a protobuf structure, e.g. read from storage + /// or received from another node. + pub fn from_protobuf_encoding(bytes: &[u8]) -> Result { + use prost::Message; + + let pubkey = keys_proto::PublicKey::decode(bytes) + .map_err(|e| DecodingError::new("Protobuf").source(e))?; + + pubkey.try_into() + } + + /// Convert the `PublicKey` into the corresponding `PeerId`. + pub fn to_peer_id(&self) -> PeerId { + self.into() + } +} + +impl From<&PublicKey> for keys_proto::PublicKey { + fn from(key: &PublicKey) -> Self { + match key { PublicKey::Ed25519(key) => keys_proto::PublicKey { r#type: keys_proto::KeyType::Ed25519 as i32, data: key.encode().to_vec(), @@ -236,24 +266,14 @@ impl PublicKey { r#type: keys_proto::KeyType::Secp256k1 as i32, data: key.encode().to_vec(), }, - }; - - let mut buf = Vec::with_capacity(public_key.encoded_len()); - public_key - .encode(&mut buf) - .expect("Vec provides capacity as needed"); - buf + } } +} - /// Decode a public key from a protobuf structure, e.g. read from storage - /// or received from another node. - pub fn from_protobuf_encoding(bytes: &[u8]) -> Result { - use prost::Message; - - #[allow(unused_mut)] // Due to conditional compilation. - let mut pubkey = keys_proto::PublicKey::decode(bytes) - .map_err(|e| DecodingError::new("Protobuf").source(e))?; +impl TryFrom for PublicKey { + type Error = DecodingError; + fn try_from(pubkey: keys_proto::PublicKey) -> Result { let key_type = keys_proto::KeyType::from_i32(pubkey.r#type) .ok_or_else(|| DecodingError::new(format!("unknown key type: {}", pubkey.r#type)))?; @@ -281,11 +301,6 @@ impl PublicKey { } } } - - /// Convert the `PublicKey` into the corresponding `PeerId`. - pub fn to_peer_id(&self) -> PeerId { - self.into() - } } #[cfg(test)] diff --git a/core/src/identity/rsa.rs b/core/src/identity/rsa.rs index 752bb156764..cd18f6a848c 100644 --- a/core/src/identity/rsa.rs +++ b/core/src/identity/rsa.rs @@ -39,7 +39,7 @@ impl Keypair { /// /// [RFC5208]: https://tools.ietf.org/html/rfc5208#section-5 pub fn from_pkcs8(der: &mut [u8]) -> Result { - let kp = RsaKeyPair::from_pkcs8(&der) + let kp = RsaKeyPair::from_pkcs8(der) .map_err(|e| DecodingError::new("RSA PKCS#8 PrivateKeyInfo").source(e))?; der.zeroize(); Ok(Keypair(Arc::new(kp))) @@ -54,7 +54,7 @@ impl Keypair { pub fn sign(&self, data: &[u8]) -> Result, SigningError> { let mut signature = vec![0; self.0.public_modulus_len()]; let rng = SystemRandom::new(); - match self.0.sign(&RSA_PKCS1_SHA256, &rng, &data, &mut signature) { + match self.0.sign(&RSA_PKCS1_SHA256, &rng, data, &mut signature) { Ok(()) => Ok(signature), Err(e) => Err(SigningError::new("RSA").source(e)), } @@ -94,11 +94,9 @@ impl PublicKey { subjectPublicKey: Asn1SubjectPublicKey(self.clone()), }; let mut buf = Vec::new(); - let buf = spki - .encode(&mut buf) + spki.encode(&mut buf) .map(|_| buf) - .expect("RSA X.509 public key encoding failed."); - buf + .expect("RSA X.509 public key encoding failed.") } /// Decode an RSA public key from a DER-encoded X.509 SubjectPublicKeyInfo diff --git a/core/src/lib.rs b/core/src/lib.rs index 60727c52062..1a7d841e342 100644 --- a/core/src/lib.rs +++ b/core/src/lib.rs @@ -39,6 +39,14 @@ mod keys_proto { include!(concat!(env!("OUT_DIR"), "/keys_proto.rs")); } +mod envelope_proto { + include!(concat!(env!("OUT_DIR"), "/envelope_proto.rs")); +} + +mod peer_record_proto { + include!(concat!(env!("OUT_DIR"), "/peer_record_proto.rs")); +} + /// Multi-address re-export. pub use multiaddr; pub type Negotiated = multistream_select::Negotiated; @@ -51,6 +59,8 @@ pub mod either; pub mod identity; pub mod muxing; pub mod network; +pub mod peer_record; +pub mod signed_envelope; pub mod transport; pub mod upgrade; @@ -61,6 +71,8 @@ pub use multihash; pub use muxing::StreamMuxer; pub use network::Network; pub use peer_id::PeerId; +pub use peer_record::PeerRecord; +pub use signed_envelope::SignedEnvelope; pub use translation::address_translation; pub use transport::Transport; pub use upgrade::{InboundUpgrade, OutboundUpgrade, ProtocolName, UpgradeError, UpgradeInfo}; diff --git a/core/src/network.rs b/core/src/network.rs index 784c1e01ca7..831a99c4b01 100644 --- a/core/src/network.rs +++ b/core/src/network.rs @@ -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 + ) -> Result> where TTrans: Transport, 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) -> Result + fn dial_peer( + &mut self, + opts: DialingOpts, + ) -> Result> where TTrans: Transport, TTrans::Dial: Send + 'static, @@ -502,7 +503,7 @@ fn dial_peer_impl( pool: &mut Pool, dialing: &mut FnvHashMap>, opts: DialingOpts, -) -> Result +) -> Result> where THandler: IntoConnectionHandler + Send + 'static, ::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, - handler: Option, + handler: THandler, ) -> ( Option>, NetworkEvent<'a, TTrans, THandlerInEvent, THandlerOutEvent, 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 { } /// Possible (synchronous) errors when dialing a peer. -#[derive(Clone, Debug)] -pub enum DialError { +#[derive(Clone)] +pub enum DialError { /// 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 fmt::Debug for DialError { + 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)] diff --git a/core/src/network/event.rs b/core/src/network/event.rs index 7b4158265d9..cea5bbddc21 100644 --- a/core/src/network/event.rs +++ b/core/src/network/event.rs @@ -92,6 +92,7 @@ where send_back_addr: Multiaddr, /// The error that happened. error: PendingConnectionError, + handler: THandler, }, /// A new connection to a peer has been established. @@ -124,12 +125,13 @@ where error: Option::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, /// Id of the peer we were trying to dial. peer_id: PeerId, @@ -148,6 +150,8 @@ where /// The error that happened. error: PendingConnectionError, + + handler: THandler, }, /// An established connection produced an event. @@ -169,6 +173,20 @@ where }, } +pub enum DialAttemptsRemaining { + Some(NonZeroU32), + None(THandler), +} + +impl DialAttemptsRemaining { + pub fn get_attempts(&self) -> u32 { + match self { + DialAttemptsRemaining::Some(attempts) => (*attempts).into(), + DialAttemptsRemaining::None(_) => 0, + } + } +} + impl 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) diff --git a/core/src/network/peer.rs b/core/src/network/peer.rs index ca1b9be7502..1eda5dde9e0 100644 --- a/core/src/network/peer.rs +++ b/core/src/network/peer.rs @@ -21,9 +21,9 @@ use super::{DialError, DialingOpts, Network}; use crate::{ connection::{ - handler::THandlerInEvent, pool::Pool, Connected, ConnectedPoint, Connection, - ConnectionHandler, ConnectionId, ConnectionLimit, EstablishedConnection, - EstablishedConnectionIter, IntoConnectionHandler, PendingConnection, Substream, + handler::THandlerInEvent, pool::Pool, ConnectedPoint, ConnectionHandler, ConnectionId, + 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> where I: IntoIterator, { @@ -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 { @@ -472,44 +467,6 @@ where pub fn into_peer(self) -> Peer<'a, TTrans, THandler> { Peer::Disconnected(self) } - - /// Moves the peer into a connected state by supplying an existing - /// established connection. - /// - /// No event is generated for this action. - /// - /// # Panics - /// - /// Panics if `connected.peer_id` does not identify the current peer. - pub fn set_connected( - self, - connected: Connected, - connection: Connection, - ) -> Result, ConnectionLimit> - where - THandler: Send + 'static, - TTrans::Error: Send + 'static, - THandler::Handler: ConnectionHandler> + Send, - ::OutboundOpenInfo: Send, - ::Error: error::Error + Send + 'static, - TMuxer: StreamMuxer + Send + Sync + 'static, - TMuxer::OutboundSubstream: Send, - { - if connected.peer_id != self.peer_id { - panic!( - "Invalid peer ID given: {:?}. Expected: {:?}", - connected.peer_id, self.peer_id - ) - } - - self.network - .pool - .add(connection, connected) - .map(move |_id| ConnectedPeer { - network: self.network, - peer_id: self.peer_id, - }) - } } /// The (internal) state of a `DialingAttempt`, tracking the diff --git a/core/src/peer_id.rs b/core/src/peer_id.rs index 5a9ae8b0341..6e3823e64ed 100644 --- a/core/src/peer_id.rs +++ b/core/src/peer_id.rs @@ -66,7 +66,7 @@ impl PeerId { /// Parses a `PeerId` from bytes. pub fn from_bytes(data: &[u8]) -> Result { - PeerId::from_multihash(Multihash::from_bytes(&data)?) + PeerId::from_multihash(Multihash::from_bytes(data)?) .map_err(|mh| Error::UnsupportedCode(mh.code())) } diff --git a/core/src/peer_record.proto b/core/src/peer_record.proto new file mode 100644 index 00000000000..69bb345e02f --- /dev/null +++ b/core/src/peer_record.proto @@ -0,0 +1,27 @@ +syntax = "proto3"; + +package peer_record_proto; + +// PeerRecord messages contain information that is useful to share with other peers. +// Currently, a PeerRecord contains the public listen addresses for a peer, but this +// is expected to expand to include other information in the future. +// +// PeerRecords are designed to be serialized to bytes and placed inside of +// SignedEnvelopes before sharing with other peers. +message PeerRecord { + + // AddressInfo is a wrapper around a binary multiaddr. It is defined as a + // separate message to allow us to add per-address metadata in the future. + message AddressInfo { + bytes multiaddr = 1; + } + + // peer_id contains a libp2p peer id in its binary representation. + bytes peer_id = 1; + + // seq contains a monotonically-increasing sequence counter to order PeerRecords in time. + uint64 seq = 2; + + // addresses is a list of public listen addresses for the peer. + repeated AddressInfo addresses = 3; +} diff --git a/core/src/peer_record.rs b/core/src/peer_record.rs new file mode 100644 index 00000000000..6b7759213c9 --- /dev/null +++ b/core/src/peer_record.rs @@ -0,0 +1,199 @@ +use crate::identity::error::SigningError; +use crate::identity::Keypair; +use crate::signed_envelope::SignedEnvelope; +use crate::{peer_record_proto, signed_envelope, Multiaddr, PeerId}; +use std::convert::TryInto; +use std::fmt; +use std::time::SystemTime; + +const PAYLOAD_TYPE: &str = "/libp2p/routing-state-record"; +const DOMAIN_SEP: &str = "libp2p-routing-state"; + +/// Represents a peer routing record. +/// +/// Peer records are designed to be distributable and carry a signature by being wrapped in a signed envelope. +/// For more information see RFC0003 of the libp2p specifications: +#[derive(Debug, PartialEq, Clone)] +pub struct PeerRecord { + peer_id: PeerId, + seq: u64, + addresses: Vec, + + /// A signed envelope representing this [`PeerRecord`]. + /// + /// If this [`PeerRecord`] was constructed from a [`SignedEnvelope`], this is the original instance. + envelope: SignedEnvelope, +} + +impl PeerRecord { + /// Attempt to re-construct a [`PeerRecord`] from a [`SignedEnvelope`]. + /// + /// If this function succeeds, the [`SignedEnvelope`] contained a peer record with a valid signature and can hence be considered authenticated. + pub fn from_signed_envelope(envelope: SignedEnvelope) -> Result { + use prost::Message; + + let payload = envelope.payload(String::from(DOMAIN_SEP), PAYLOAD_TYPE.as_bytes())?; + let record = peer_record_proto::PeerRecord::decode(payload)?; + + let peer_id = PeerId::from_bytes(&record.peer_id)?; + let seq = record.seq; + let addresses = record + .addresses + .into_iter() + .map(|a| a.multiaddr.try_into()) + .collect::, _>>()?; + + Ok(Self { + peer_id, + seq, + addresses, + envelope, + }) + } + + /// Construct a new [`PeerRecord`] by authenticating the provided addresses with the given key. + /// + /// This is the same key that is used for authenticating every libp2p connection of your application, i.e. what you use when setting up your [`crate::transport::Transport`]. + pub fn new(key: Keypair, addresses: Vec) -> Result { + use prost::Message; + + let seq = SystemTime::now() + .duration_since(SystemTime::UNIX_EPOCH) + .expect("now() is never before UNIX_EPOCH") + .as_secs(); + let peer_id = key.public().to_peer_id(); + + let payload = { + let record = peer_record_proto::PeerRecord { + peer_id: peer_id.to_bytes(), + seq, + addresses: addresses + .iter() + .map(|m| peer_record_proto::peer_record::AddressInfo { + multiaddr: m.to_vec(), + }) + .collect(), + }; + + let mut buf = Vec::with_capacity(record.encoded_len()); + record + .encode(&mut buf) + .expect("Vec provides capacity as needed"); + buf + }; + + let envelope = SignedEnvelope::new( + key, + String::from(DOMAIN_SEP), + PAYLOAD_TYPE.as_bytes().to_vec(), + payload, + )?; + + Ok(Self { + peer_id, + seq, + addresses, + envelope, + }) + } + + pub fn to_signed_envelope(&self) -> SignedEnvelope { + self.envelope.clone() + } + + pub fn into_signed_envelope(self) -> SignedEnvelope { + self.envelope + } + + pub fn peer_id(&self) -> PeerId { + self.peer_id + } + + pub fn seq(&self) -> u64 { + self.seq + } + + pub fn addresses(&self) -> &[Multiaddr] { + self.addresses.as_slice() + } +} + +#[derive(Debug)] +pub enum FromEnvelopeError { + /// Failed to extract the payload from the envelope. + BadPayload(signed_envelope::ReadPayloadError), + /// Failed to decode the provided bytes as a [`PeerRecord`]. + InvalidPeerRecord(prost::DecodeError), + /// Failed to decode the peer ID. + InvalidPeerId(multihash::Error), + /// Failed to decode a multi-address. + InvalidMultiaddr(multiaddr::Error), +} + +impl From for FromEnvelopeError { + fn from(e: signed_envelope::ReadPayloadError) -> Self { + Self::BadPayload(e) + } +} + +impl From for FromEnvelopeError { + fn from(e: prost::DecodeError) -> Self { + Self::InvalidPeerRecord(e) + } +} + +impl From for FromEnvelopeError { + fn from(e: multihash::Error) -> Self { + Self::InvalidPeerId(e) + } +} + +impl From for FromEnvelopeError { + fn from(e: multiaddr::Error) -> Self { + Self::InvalidMultiaddr(e) + } +} + +impl fmt::Display for FromEnvelopeError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::BadPayload(_) => write!(f, "Failed to extract payload from envelope"), + Self::InvalidPeerRecord(_) => { + write!(f, "Failed to decode bytes as PeerRecord") + } + Self::InvalidPeerId(_) => write!(f, "Failed to decode bytes as PeerId"), + Self::InvalidMultiaddr(_) => { + write!(f, "Failed to decode bytes as MultiAddress") + } + } + } +} + +impl std::error::Error for FromEnvelopeError { + fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { + match self { + Self::InvalidPeerRecord(inner) => Some(inner), + Self::InvalidPeerId(inner) => Some(inner), + Self::InvalidMultiaddr(inner) => Some(inner), + Self::BadPayload(inner) => Some(inner), + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + const HOME: &str = "/ip4/127.0.0.1/tcp/1337"; + + #[test] + fn roundtrip_envelope() { + let record = + PeerRecord::new(Keypair::generate_ed25519(), vec![HOME.parse().unwrap()]).unwrap(); + + let envelope = record.to_signed_envelope(); + let reconstructed = PeerRecord::from_signed_envelope(envelope).unwrap(); + + assert_eq!(reconstructed, record) + } +} diff --git a/core/src/signed_envelope.rs b/core/src/signed_envelope.rs new file mode 100644 index 00000000000..a528cb0811f --- /dev/null +++ b/core/src/signed_envelope.rs @@ -0,0 +1,203 @@ +use crate::identity::error::SigningError; +use crate::identity::Keypair; +use crate::{identity, PublicKey}; +use std::convert::TryInto; +use std::fmt; +use unsigned_varint::encode::usize_buffer; + +/// A signed envelope contains an arbitrary byte string payload, a signature of the payload, and the public key that can be used to verify the signature. +/// +/// For more details see libp2p RFC0002: +#[derive(Debug, Clone, PartialEq)] +pub struct SignedEnvelope { + key: PublicKey, + payload_type: Vec, + payload: Vec, + signature: Vec, +} + +impl SignedEnvelope { + /// Constructs a new [`SignedEnvelope`]. + pub fn new( + key: Keypair, + domain_separation: String, + payload_type: Vec, + payload: Vec, + ) -> Result { + let buffer = signature_payload(domain_separation, &payload_type, &payload); + + let signature = key.sign(&buffer)?; + + Ok(Self { + key: key.public(), + payload_type, + payload, + signature, + }) + } + + /// Verify this [`SignedEnvelope`] against the provided domain-separation string. + #[must_use] + pub fn verify(&self, domain_separation: String) -> bool { + let buffer = signature_payload(domain_separation, &self.payload_type, &self.payload); + + self.key.verify(&buffer, &self.signature) + } + + /// Extract the payload of this [`SignedEnvelope`]. + /// + /// You must provide the correct domain-separation string and expected payload type in order to get the payload. + /// This guards against accidental mis-use of the payload where the signature was created for a different purpose or payload type. + pub fn payload( + &self, + domain_separation: String, + expected_payload_type: &[u8], + ) -> Result<&[u8], ReadPayloadError> { + if self.payload_type != expected_payload_type { + return Err(ReadPayloadError::UnexpectedPayloadType { + expected: expected_payload_type.to_vec(), + got: self.payload_type.clone(), + }); + } + + if !self.verify(domain_separation) { + return Err(ReadPayloadError::InvalidSignature); + } + + Ok(&self.payload) + } + + /// Encode this [`SignedEnvelope`] using the protobuf encoding specified in the RFC. + pub fn into_protobuf_encoding(self) -> Vec { + use prost::Message; + + let envelope = crate::envelope_proto::Envelope { + public_key: Some((&self.key).into()), + payload_type: self.payload_type, + payload: self.payload, + signature: self.signature, + }; + + let mut buf = Vec::with_capacity(envelope.encoded_len()); + envelope + .encode(&mut buf) + .expect("Vec provides capacity as needed"); + + buf + } + + /// Decode a [`SignedEnvelope`] using the protobuf encoding specified in the RFC. + pub fn from_protobuf_encoding(bytes: &[u8]) -> Result { + use prost::Message; + + let envelope = crate::envelope_proto::Envelope::decode(bytes)?; + + Ok(Self { + key: envelope + .public_key + .ok_or(DecodingError::MissingPublicKey)? + .try_into()?, + payload_type: envelope.payload_type, + payload: envelope.payload, + signature: envelope.signature, + }) + } +} + +fn signature_payload(domain_separation: String, payload_type: &[u8], payload: &[u8]) -> Vec { + let mut domain_sep_length_buffer = usize_buffer(); + let domain_sep_length = + unsigned_varint::encode::usize(domain_separation.len(), &mut domain_sep_length_buffer); + + let mut payload_type_length_buffer = usize_buffer(); + let payload_type_length = + unsigned_varint::encode::usize(payload_type.len(), &mut payload_type_length_buffer); + + let mut payload_length_buffer = usize_buffer(); + let payload_length = unsigned_varint::encode::usize(payload.len(), &mut payload_length_buffer); + + let mut buffer = Vec::with_capacity( + domain_sep_length.len() + + domain_separation.len() + + payload_type_length.len() + + payload_type.len() + + payload_length.len() + + payload.len(), + ); + + buffer.extend_from_slice(domain_sep_length); + buffer.extend_from_slice(domain_separation.as_bytes()); + buffer.extend_from_slice(payload_type_length); + buffer.extend_from_slice(payload_type); + buffer.extend_from_slice(payload_length); + buffer.extend_from_slice(payload); + + buffer +} + +/// Errors that occur whilst decoding a [`SignedEnvelope`] from its byte representation. +#[derive(Debug)] +pub enum DecodingError { + /// Decoding the provided bytes as a signed envelope failed. + InvalidEnvelope(prost::DecodeError), + /// The public key in the envelope could not be converted to our internal public key type. + InvalidPublicKey(identity::error::DecodingError), + /// The public key in the envelope could not be converted to our internal public key type. + MissingPublicKey, +} + +impl From for DecodingError { + fn from(e: prost::DecodeError) -> Self { + Self::InvalidEnvelope(e) + } +} + +impl From for DecodingError { + fn from(e: identity::error::DecodingError) -> Self { + Self::InvalidPublicKey(e) + } +} + +impl fmt::Display for DecodingError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::InvalidEnvelope(_) => write!(f, "Failed to decode envelope"), + Self::InvalidPublicKey(_) => write!(f, "Failed to convert public key"), + Self::MissingPublicKey => write!(f, "Public key is missing from protobuf struct"), + } + } +} + +impl std::error::Error for DecodingError { + fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { + match self { + Self::InvalidEnvelope(inner) => Some(inner), + Self::InvalidPublicKey(inner) => Some(inner), + Self::MissingPublicKey => None, + } + } +} + +/// Errors that occur whilst extracting the payload of a [`SignedEnvelope`]. +#[derive(Debug)] +pub enum ReadPayloadError { + /// The signature on the signed envelope does not verify with the provided domain separation string. + InvalidSignature, + /// The payload contained in the envelope is not of the expected type. + UnexpectedPayloadType { expected: Vec, got: Vec }, +} + +impl fmt::Display for ReadPayloadError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Self::InvalidSignature => write!(f, "Invalid signature"), + Self::UnexpectedPayloadType { expected, got } => write!( + f, + "Unexpected payload type, expected {:?} but got {:?}", + expected, got + ), + } + } +} + +impl std::error::Error for ReadPayloadError {} diff --git a/core/src/upgrade.rs b/core/src/upgrade.rs index fbee321a83d..d9edd6492bc 100644 --- a/core/src/upgrade.rs +++ b/core/src/upgrade.rs @@ -69,8 +69,6 @@ mod transfer; use futures::future::Future; -#[allow(deprecated)] -pub use self::transfer::ReadOneError; pub use self::{ apply::{apply, apply_inbound, apply_outbound, InboundUpgradeApply, OutboundUpgradeApply}, denied::DeniedUpgrade, diff --git a/core/src/upgrade/transfer.rs b/core/src/upgrade/transfer.rs index fd8127758f1..baea400adfd 100644 --- a/core/src/upgrade/transfer.rs +++ b/core/src/upgrade/transfer.rs @@ -21,7 +21,7 @@ //! Contains some helper futures for creating upgrades. use futures::prelude::*; -use std::{error, fmt, io}; +use std::io; // TODO: these methods could be on an Ext trait to AsyncWrite @@ -40,42 +40,6 @@ pub async fn write_length_prefixed( Ok(()) } -/// Send a message to the given socket, then shuts down the writing side. -/// -/// > **Note**: Prepends a variable-length prefix indicate the length of the message. This is -/// > compatible with what `read_one` expects. -/// -#[deprecated( - since = "0.29.0", - note = "Use `write_length_prefixed` instead. You will need to manually close the stream using `socket.close().await`." -)] -#[allow(dead_code)] -pub async fn write_one( - socket: &mut (impl AsyncWrite + Unpin), - data: impl AsRef<[u8]>, -) -> Result<(), io::Error> { - write_varint(socket, data.as_ref().len()).await?; - socket.write_all(data.as_ref()).await?; - socket.close().await?; - Ok(()) -} - -/// Send a message to the given socket with a length prefix appended to it. Also flushes the socket. -/// -/// > **Note**: Prepends a variable-length prefix indicate the length of the message. This is -/// > compatible with what `read_one` expects. -#[deprecated(since = "0.29.0", note = "Use `write_length_prefixed` instead.")] -#[allow(dead_code)] -pub async fn write_with_len_prefix( - socket: &mut (impl AsyncWrite + Unpin), - data: impl AsRef<[u8]>, -) -> Result<(), io::Error> { - write_varint(socket, data.as_ref().len()).await?; - socket.write_all(data.as_ref()).await?; - socket.flush().await?; - Ok(()) -} - /// Writes a variable-length integer to the `socket`. /// /// > **Note**: Does **NOT** flush the socket. @@ -162,78 +126,6 @@ pub async fn read_length_prefixed( Ok(buf) } -/// Reads a length-prefixed message from the given socket. -/// -/// The `max_size` parameter is the maximum size in bytes of the message that we accept. This is -/// necessary in order to avoid DoS attacks where the remote sends us a message of several -/// gigabytes. -/// -/// > **Note**: Assumes that a variable-length prefix indicates the length of the message. This is -/// > compatible with what `write_one` does. -#[deprecated(since = "0.29.0", note = "Use `read_length_prefixed` instead.")] -#[allow(dead_code, deprecated)] -pub async fn read_one( - socket: &mut (impl AsyncRead + Unpin), - max_size: usize, -) -> Result, ReadOneError> { - let len = read_varint(socket).await?; - if len > max_size { - return Err(ReadOneError::TooLarge { - requested: len, - max: max_size, - }); - } - - let mut buf = vec![0; len]; - socket.read_exact(&mut buf).await?; - Ok(buf) -} - -/// Error while reading one message. -#[derive(Debug)] -#[deprecated( - since = "0.29.0", - note = "Use `read_length_prefixed` instead of `read_one` to avoid depending on this type." -)] -pub enum ReadOneError { - /// Error on the socket. - Io(std::io::Error), - /// Requested data is over the maximum allowed size. - TooLarge { - /// Size requested by the remote. - requested: usize, - /// Maximum allowed. - max: usize, - }, -} - -#[allow(deprecated)] -impl From for ReadOneError { - fn from(err: std::io::Error) -> ReadOneError { - ReadOneError::Io(err) - } -} - -#[allow(deprecated)] -impl fmt::Display for ReadOneError { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match *self { - ReadOneError::Io(ref err) => write!(f, "{}", err), - ReadOneError::TooLarge { .. } => write!(f, "Received data size over maximum"), - } - } -} - -#[allow(deprecated)] -impl error::Error for ReadOneError { - fn source(&self) -> Option<&(dyn error::Error + 'static)> { - match *self { - ReadOneError::Io(ref err) => Some(err), - ReadOneError::TooLarge { .. } => None, - } - } -} - #[cfg(test)] mod tests { use super::*; diff --git a/core/tests/connection_limits.rs b/core/tests/connection_limits.rs index 65e61c4b3c4..d5156664ffb 100644 --- a/core/tests/connection_limits.rs +++ b/core/tests/connection_limits.rs @@ -23,7 +23,7 @@ mod util; use futures::{future::poll_fn, ready}; use libp2p_core::multiaddr::{multiaddr, Multiaddr}; use libp2p_core::{ - connection::PendingConnectionError, + connection::ConnectionError, network::{ConnectionLimits, DialError, NetworkConfig, NetworkEvent}, PeerId, }; @@ -53,9 +53,9 @@ fn max_outgoing() { .dial(Multiaddr::empty(), Vec::new(), TestHandler()) .expect_err("Unexpected dialing success.") { - DialError::ConnectionLimit(err) => { - assert_eq!(err.current, outgoing_limit); - assert_eq!(err.limit, outgoing_limit); + DialError::ConnectionLimit { limit, handler: _ } => { + assert_eq!(limit.current, outgoing_limit); + assert_eq!(limit.limit, outgoing_limit); } e => panic!("Unexpected error: {:?}", e), } @@ -111,8 +111,8 @@ fn max_established_incoming() { network1.accept(connection, TestHandler()).unwrap(); } NetworkEvent::ConnectionEstablished { .. } => {} - NetworkEvent::IncomingConnectionError { - error: PendingConnectionError::ConnectionLimit(err), + NetworkEvent::ConnectionClosed { + error: Some(ConnectionError::ConnectionLimit(err)), .. } => { assert_eq!(err.limit, limit); diff --git a/core/tests/network_dial_error.rs b/core/tests/network_dial_error.rs index 224d7950eac..bed4c06e023 100644 --- a/core/tests/network_dial_error.rs +++ b/core/tests/network_dial_error.rs @@ -65,11 +65,12 @@ fn deny_incoming_connec() { match swarm2.poll(cx) { Poll::Ready(NetworkEvent::DialError { - attempts_remaining: 0, + attempts_remaining, peer_id, multiaddr, error: PendingConnectionError::Transport(_), }) => { + assert_eq!(0u32, attempts_remaining.get_attempts()); assert_eq!(&peer_id, swarm1.local_peer_id()); assert_eq!( multiaddr, @@ -201,10 +202,10 @@ fn multiple_addresses_err() { .with(Protocol::P2p(target.clone().into())); assert_eq!(multiaddr, expected); if addresses.is_empty() { - assert_eq!(attempts_remaining, 0); + assert_eq!(attempts_remaining.get_attempts(), 0); return Poll::Ready(Ok(())); } else { - assert_eq!(attempts_remaining, addresses.len() as u32); + assert_eq!(attempts_remaining.get_attempts(), addresses.len() as u32); } } Poll::Ready(_) => unreachable!(), diff --git a/examples/README.md b/examples/README.md new file mode 100644 index 00000000000..98dbfbecf1c --- /dev/null +++ b/examples/README.md @@ -0,0 +1,51 @@ +# Examples + +A set of examples showcasing how to use rust-libp2p. + +## Getting started + +- [Ping](ping.rs) + + Small `ping` clone, sending a ping to a peer, expecting a pong as a response. See + [tutorial](../src/tutorial.rs) for a step-by-step guide building the example. + +## Individual libp2p protocols + +- [Chat](./chat.rs) + + A basic chat application demonstrating libp2p and the mDNS and floodsub protocols. + + - [Gossipsub chat](./gossipsub-chat.rs) + + Same as the chat example but using the Gossipsub protocol. + + - [Tokio based chat](./chat-tokio.rs) + + Same as the chat example but using tokio for all asynchronous tasks and I/O. + +- [Distributed key-value store](./distributed-key-value-store.rs) + + A basic key value store demonstrating libp2p and the mDNS and Kademlia protocol. + +- [IPFS Kademlia](ipfs-kad.rs) + + Demonstrates how to perform Kademlia queries on the IPFS network. + +- [IPFS Private](ipfs-private.rs) + + Implementation using the gossipsub, ping and identify protocols to implement the ipfs private + swarms feature. + +- [Passive Discovery via MDNS](mdns-passive-discovery.rs) + + Discover peers on the same network via the MDNS protocol. + +## Integration into a larger application + +- [File sharing application](./file-sharing.rs) + + Basic file sharing application with peers either providing or locating and getting files by name. + + While obviously showcasing how to build a basic file sharing application with the Kademlia and + Request-Response protocol, the actual goal of this example is **to show how to integrate + rust-libp2p into a larger application**. diff --git a/examples/chat-tokio.rs b/examples/chat-tokio.rs index 202b5b39156..e5238e785e8 100644 --- a/examples/chat-tokio.rs +++ b/examples/chat-tokio.rs @@ -87,6 +87,7 @@ async fn main() -> Result<(), Box> { // requires the implementations of `NetworkBehaviourEventProcess` for // the events of each behaviour. #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct MyBehaviour { floodsub: Floodsub, mdns: Mdns, diff --git a/examples/chat.rs b/examples/chat.rs index 18ef72b96b8..7343732f8c4 100644 --- a/examples/chat.rs +++ b/examples/chat.rs @@ -55,7 +55,7 @@ use libp2p::{ floodsub::{self, Floodsub, FloodsubEvent}, identity, mdns::{Mdns, MdnsConfig, MdnsEvent}, - swarm::{NetworkBehaviourEventProcess, SwarmEvent}, + swarm::SwarmEvent, Multiaddr, NetworkBehaviour, PeerId, Swarm, }; use std::{ @@ -83,6 +83,7 @@ async fn main() -> Result<(), Box> { // Use the derive to generate delegating NetworkBehaviour impl and require the // NetworkBehaviourEventProcess implementations below. #[derive(NetworkBehaviour)] + #[behaviour(out_event = "OutEvent")] struct MyBehaviour { floodsub: Floodsub, mdns: Mdns, @@ -93,36 +94,21 @@ async fn main() -> Result<(), Box> { ignored_member: bool, } - impl NetworkBehaviourEventProcess for MyBehaviour { - // Called when `floodsub` produces an event. - fn inject_event(&mut self, message: FloodsubEvent) { - if let FloodsubEvent::Message(message) = message { - println!( - "Received: '{:?}' from {:?}", - String::from_utf8_lossy(&message.data), - message.source - ); - } + #[derive(Debug)] + enum OutEvent { + Floodsub(FloodsubEvent), + Mdns(MdnsEvent), + } + + impl From for OutEvent { + fn from(v: MdnsEvent) -> Self { + Self::Mdns(v) } } - impl NetworkBehaviourEventProcess for MyBehaviour { - // Called when `mdns` produces an event. - fn inject_event(&mut self, event: MdnsEvent) { - match event { - MdnsEvent::Discovered(list) => { - for (peer, _) in list { - self.floodsub.add_node_to_partial_view(peer); - } - } - MdnsEvent::Expired(list) => { - for (peer, _) in list { - if !self.mdns.has_node(&peer) { - self.floodsub.remove_node_from_partial_view(&peer); - } - } - } - } + impl From for OutEvent { + fn from(v: FloodsubEvent) -> Self { + Self::Floodsub(v) } } @@ -166,11 +152,41 @@ async fn main() -> Result<(), Box> { } loop { match swarm.poll_next_unpin(cx) { - Poll::Ready(Some(event)) => { - if let SwarmEvent::NewListenAddr { address, .. } = event { - println!("Listening on {:?}", address); + Poll::Ready(Some(SwarmEvent::NewListenAddr { address, .. })) => { + println!("Listening on {:?}", address); + } + Poll::Ready(Some(SwarmEvent::Behaviour(OutEvent::Floodsub( + FloodsubEvent::Message(message), + )))) => { + println!( + "Received: '{:?}' from {:?}", + String::from_utf8_lossy(&message.data), + message.source + ); + } + Poll::Ready(Some(SwarmEvent::Behaviour(OutEvent::Mdns( + MdnsEvent::Discovered(list), + )))) => { + for (peer, _) in list { + swarm + .behaviour_mut() + .floodsub + .add_node_to_partial_view(peer); + } + } + Poll::Ready(Some(SwarmEvent::Behaviour(OutEvent::Mdns(MdnsEvent::Expired( + list, + ))))) => { + for (peer, _) in list { + if !swarm.behaviour_mut().mdns.has_node(&peer) { + swarm + .behaviour_mut() + .floodsub + .remove_node_from_partial_view(&peer); + } } } + Poll::Ready(Some(_)) => {} Poll::Ready(None) => return Poll::Ready(Ok(())), Poll::Pending => break, } diff --git a/examples/distributed-key-value-store.rs b/examples/distributed-key-value-store.rs index 2e5fa5a8531..cea3ca2a56b 100644 --- a/examples/distributed-key-value-store.rs +++ b/examples/distributed-key-value-store.rs @@ -71,6 +71,7 @@ async fn main() -> Result<(), Box> { // We create a custom network behaviour that combines Kademlia and mDNS. #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct MyBehaviour { kademlia: Kademlia, mdns: Mdns, diff --git a/examples/file-sharing.rs b/examples/file-sharing.rs new file mode 100644 index 00000000000..65ba87feaef --- /dev/null +++ b/examples/file-sharing.rs @@ -0,0 +1,733 @@ +// Copyright 2021 Protocol Labs. +// +// 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. + +//! # File sharing example +//! +//! Basic file sharing application with peers either providing or locating and +//! getting files by name. +//! +//! While obviously showcasing how to build a basic file sharing application, +//! the actual goal of this example is **to show how to integrate rust-libp2p +//! into a larger application**. +//! +//! ## Sample plot +//! +//! Assuming there are 3 nodes, A, B and C. A and B each provide a file while C +//! retrieves a file. +//! +//! Provider nodes A and B each provide a file, file FA and FB respectively. +//! They do so by advertising themselves as a provider for their file on a DHT +//! via [`libp2p-kad`]. The two, among other nodes of the network, are +//! interconnected via the DHT. +//! +//! Node C can locate the providers for file FA or FB on the DHT via +//! [`libp2p-kad`] without being connected to the specific node providing the +//! file, but any node of the DHT. Node C then connects to the corresponding +//! node and requests the file content of the file via +//! [`libp2p-request-response`]. +//! +//! ## Architectural properties +//! +//! - Clean clonable async/await interface ([`Client`]) to interact with the +//! network layer. +//! +//! - Single task driving the network layer, no locks required. +//! +//! ## Usage +//! +//! A two node setup with one node providing the file and one node requesting the file. +//! +//! 1. Run command below in one terminal. +//! +//! ``` +//! cargo run --example file-sharing -- \ +//! --listen-address /ip4/127.0.0.1/tcp/40837 \ +//! --secret-key-seed 1 \ +//! provide \ +//! --path \ +//! --name +//! ``` +//! +//! 2. Run command below in another terminal. +//! +//! ``` +//! cargo run --example file-sharing -- \ +//! --peer /ip4/127.0.0.1/tcp/40837/p2p/12D3KooWPjceQrSwdWXPyLLeABRXmuqt69Rg3sBYbU1Nft9HyQ6X \ +//! get \ +//! --name +//! ``` +//! +//! Note: The client does not need to be directly connected to the providing +//! peer, as long as both are connected to some node on the same DHT. + +use async_std::io; +use async_std::task::spawn; +use futures::prelude::*; +use libp2p::core::{Multiaddr, PeerId}; +use libp2p::multiaddr::Protocol; +use std::error::Error; +use std::path::PathBuf; +use structopt::StructOpt; + +#[async_std::main] +async fn main() -> Result<(), Box> { + env_logger::init(); + + let opt = Opt::from_args(); + + let (mut network_client, mut network_events, network_event_loop) = + network::new(opt.secret_key_seed).await?; + + // Spawn the network task for it to run in the background. + spawn(network_event_loop.run()); + + // In case a listen address was provided use it, otherwise listen on any + // address. + match opt.listen_address { + Some(addr) => network_client + .start_listening(addr) + .await + .expect("Listening not to fail."), + None => network_client + .start_listening("/ip4/0.0.0.0/tcp/0".parse()?) + .await + .expect("Listening not to fail."), + }; + + // In case the user provided an address of a peer on the CLI, dial it. + if let Some(addr) = opt.peer { + let peer_id = match addr.iter().last() { + Some(Protocol::P2p(hash)) => PeerId::from_multihash(hash).expect("Valid hash."), + _ => return Err("Expect peer multiaddr to contain peer ID.".into()), + }; + network_client + .dial(peer_id, addr) + .await + .expect("Dial to succeed"); + } + + match opt.argument { + // Providing a file. + CliArgument::Provide { path, name } => { + // Advertise oneself as a provider of the file on the DHT. + network_client.start_providing(name.clone()).await; + + loop { + match network_events.next().await { + // Reply with the content of the file on incoming requests. + Some(network::Event::InboundRequest { request, channel }) => { + if request == name { + let file_content = std::fs::read_to_string(&path)?; + network_client.respond_file(file_content, channel).await; + } + } + _ => todo!(), + } + } + } + // Locating and getting a file. + CliArgument::Get { name } => { + // Locate all nodes providing the file. + let providers = network_client.get_providers(name.clone()).await; + if providers.is_empty() { + return Err(format!("Could not find provider for file {}.", name).into()); + } + + // Request the content of the file from each node. + let requests = providers.into_iter().map(|p| { + let mut network_client = network_client.clone(); + let name = name.clone(); + async move { network_client.request_file(p, name).await }.boxed() + }); + + // Await the requests, ignore the remaining once a single one succeeds. + let file = futures::future::select_ok(requests) + .await + .map_err(|_| "None of the providers returned file.")? + .0; + + println!("Content of file {}: {}", name, file); + } + } + + Ok(()) +} + +#[derive(Debug, StructOpt)] +#[structopt(name = "libp2p file sharing example")] +struct Opt { + /// Fixed value to generate deterministic peer ID. + #[structopt(long)] + secret_key_seed: Option, + + #[structopt(long)] + peer: Option, + + #[structopt(long)] + listen_address: Option, + + #[structopt(subcommand)] + argument: CliArgument, +} + +#[derive(Debug, StructOpt)] +enum CliArgument { + Provide { + #[structopt(long)] + path: PathBuf, + #[structopt(long)] + name: String, + }, + Get { + #[structopt(long)] + name: String, + }, +} + +/// The network module, encapsulating all network related logic. +mod network { + use super::*; + use async_trait::async_trait; + use futures::channel::{mpsc, oneshot}; + use libp2p::core::either::EitherError; + use libp2p::core::upgrade::{read_length_prefixed, write_length_prefixed, ProtocolName}; + use libp2p::identity; + use libp2p::identity::ed25519; + use libp2p::kad::record::store::MemoryStore; + use libp2p::kad::{GetProvidersOk, Kademlia, KademliaEvent, QueryId, QueryResult}; + use libp2p::multiaddr::Protocol; + use libp2p::request_response::{ + ProtocolSupport, RequestId, RequestResponse, RequestResponseCodec, RequestResponseEvent, + RequestResponseMessage, ResponseChannel, + }; + use libp2p::swarm::{ProtocolsHandlerUpgrErr, SwarmBuilder, SwarmEvent}; + use libp2p::{NetworkBehaviour, Swarm}; + use std::collections::{HashMap, HashSet}; + use std::iter; + + /// Creates the network components, namely: + /// + /// - The network client to interact with the network layer from anywhere + /// within your application. + /// + /// - The network event stream, e.g. for incoming requests. + /// + /// - The network task driving the network itself. + pub async fn new( + secret_key_seed: Option, + ) -> Result<(Client, impl Stream, EventLoop), Box> { + // Create a public/private key pair, either random or based on a seed. + let id_keys = match secret_key_seed { + Some(seed) => { + let mut bytes = [0u8; 32]; + bytes[0] = seed; + let secret_key = ed25519::SecretKey::from_bytes(&mut bytes).expect( + "this returns `Err` only if the length is wrong; the length is correct; qed", + ); + identity::Keypair::Ed25519(secret_key.into()) + } + None => identity::Keypair::generate_ed25519(), + }; + let peer_id = id_keys.public().to_peer_id(); + + // Build the Swarm, connecting the lower layer transport logic with the + // higher layer network behaviour logic. + let swarm = SwarmBuilder::new( + libp2p::development_transport(id_keys).await?, + ComposedBehaviour { + kademlia: Kademlia::new(peer_id, MemoryStore::new(peer_id)), + request_response: RequestResponse::new( + FileExchangeCodec(), + iter::once((FileExchangeProtocol(), ProtocolSupport::Full)), + Default::default(), + ), + }, + peer_id, + ) + .build(); + + let (command_sender, command_receiver) = mpsc::channel(0); + let (event_sender, event_receiver) = mpsc::channel(0); + + Ok(( + Client { + sender: command_sender, + }, + event_receiver, + EventLoop::new(swarm, command_receiver, event_sender), + )) + } + + #[derive(Clone)] + pub struct Client { + sender: mpsc::Sender, + } + + impl Client { + /// Listen for incoming connections on the given address. + pub async fn start_listening( + &mut self, + addr: Multiaddr, + ) -> Result<(), Box> { + let (sender, receiver) = oneshot::channel(); + self.sender + .send(Command::StartListening { addr, sender }) + .await + .expect("Command receiver not to be dropped."); + receiver.await.expect("Sender not to be dropped.") + } + + /// Dial the given peer at the given address. + pub async fn dial( + &mut self, + peer_id: PeerId, + peer_addr: Multiaddr, + ) -> Result<(), Box> { + let (sender, receiver) = oneshot::channel(); + self.sender + .send(Command::Dial { + peer_id, + peer_addr, + sender, + }) + .await + .expect("Command receiver not to be dropped."); + receiver.await.expect("Sender not to be dropped.") + } + + /// Advertise the local node as the provider of the given file on the DHT. + pub async fn start_providing(&mut self, file_name: String) { + let (sender, receiver) = oneshot::channel(); + self.sender + .send(Command::StartProviding { file_name, sender }) + .await + .expect("Command receiver not to be dropped."); + receiver.await.expect("Sender not to be dropped."); + } + + /// Find the providers for the given file on the DHT. + pub async fn get_providers(&mut self, file_name: String) -> HashSet { + let (sender, receiver) = oneshot::channel(); + self.sender + .send(Command::GetProviders { file_name, sender }) + .await + .expect("Command receiver not to be dropped."); + receiver.await.expect("Sender not to be dropped.") + } + + /// Request the content of the given file from the given peer. + pub async fn request_file( + &mut self, + peer: PeerId, + file_name: String, + ) -> Result> { + let (sender, receiver) = oneshot::channel(); + self.sender + .send(Command::RequestFile { + file_name, + peer, + sender, + }) + .await + .expect("Command receiver not to be dropped."); + receiver.await.expect("Sender not be dropped.") + } + + /// Respond with the provided file content to the given request. + pub async fn respond_file(&mut self, file: String, channel: ResponseChannel) { + self.sender + .send(Command::RespondFile { file, channel }) + .await + .expect("Command receiver not to be dropped."); + } + } + + pub struct EventLoop { + swarm: Swarm, + command_receiver: mpsc::Receiver, + event_sender: mpsc::Sender, + pending_dial: HashMap>>>, + pending_start_providing: HashMap>, + pending_get_providers: HashMap>>, + pending_request_file: + HashMap>>>, + } + + impl EventLoop { + fn new( + swarm: Swarm, + command_receiver: mpsc::Receiver, + event_sender: mpsc::Sender, + ) -> Self { + Self { + swarm, + command_receiver, + event_sender, + pending_dial: Default::default(), + pending_start_providing: Default::default(), + pending_get_providers: Default::default(), + pending_request_file: Default::default(), + } + } + + pub async fn run(mut self) { + loop { + futures::select! { + event = self.swarm.next() => self.handle_event(event.expect("Swarm stream to be infinite.")).await , + command = self.command_receiver.next() => match command { + Some(c) => self.handle_command(c).await, + // Command channel closed, thus shutting down the network event loop. + None=> return, + }, + } + } + } + + async fn handle_event( + &mut self, + event: SwarmEvent< + ComposedEvent, + EitherError, io::Error>, + >, + ) { + match event { + SwarmEvent::Behaviour(ComposedEvent::Kademlia( + KademliaEvent::OutboundQueryCompleted { + id, + result: QueryResult::StartProviding(_), + .. + }, + )) => { + let sender: oneshot::Sender<()> = self + .pending_start_providing + .remove(&id) + .expect("Completed query to be previously pending."); + let _ = sender.send(()); + } + SwarmEvent::Behaviour(ComposedEvent::Kademlia( + KademliaEvent::OutboundQueryCompleted { + id, + result: QueryResult::GetProviders(Ok(GetProvidersOk { providers, .. })), + .. + }, + )) => { + let _ = self + .pending_get_providers + .remove(&id) + .expect("Completed query to be previously pending.") + .send(providers); + } + SwarmEvent::Behaviour(ComposedEvent::Kademlia(_)) => {} + SwarmEvent::Behaviour(ComposedEvent::RequestResponse( + RequestResponseEvent::Message { message, .. }, + )) => match message { + RequestResponseMessage::Request { + request, channel, .. + } => { + self.event_sender + .send(Event::InboundRequest { + request: request.0, + channel, + }) + .await + .expect("Event receiver not to be dropped."); + } + RequestResponseMessage::Response { + request_id, + response, + } => { + let _ = self + .pending_request_file + .remove(&request_id) + .expect("Request to still be pending.") + .send(Ok(response.0)); + } + }, + SwarmEvent::Behaviour(ComposedEvent::RequestResponse( + RequestResponseEvent::OutboundFailure { + request_id, error, .. + }, + )) => { + let _ = self + .pending_request_file + .remove(&request_id) + .expect("Request to still be pending.") + .send(Err(Box::new(error))); + } + SwarmEvent::Behaviour(ComposedEvent::RequestResponse( + RequestResponseEvent::ResponseSent { .. }, + )) => {} + SwarmEvent::NewListenAddr { address, .. } => { + let local_peer_id = *self.swarm.local_peer_id(); + println!( + "Local node is listening on {:?}", + address.with(Protocol::P2p(local_peer_id.into())) + ); + } + SwarmEvent::IncomingConnection { .. } => {} + SwarmEvent::ConnectionEstablished { + peer_id, endpoint, .. + } => { + if endpoint.is_dialer() { + if let Some(sender) = self.pending_dial.remove(&peer_id) { + let _ = sender.send(Ok(())); + } + } + } + SwarmEvent::ConnectionClosed { .. } => {} + SwarmEvent::UnreachableAddr { + peer_id, + attempts_remaining, + error, + .. + } => { + if attempts_remaining == 0 { + if let Some(sender) = self.pending_dial.remove(&peer_id) { + let _ = sender.send(Err(Box::new(error))); + } + } + } + e => panic!("{:?}", e), + } + } + + async fn handle_command(&mut self, command: Command) { + match command { + Command::StartListening { addr, sender } => { + let _ = match self.swarm.listen_on(addr) { + Ok(_) => sender.send(Ok(())), + Err(e) => sender.send(Err(Box::new(e))), + }; + } + Command::Dial { + peer_id, + peer_addr, + sender, + } => { + if self.pending_dial.contains_key(&peer_id) { + todo!("Already dialing peer."); + } else { + self.swarm + .behaviour_mut() + .kademlia + .add_address(&peer_id, peer_addr.clone()); + match self + .swarm + .dial_addr(peer_addr.with(Protocol::P2p(peer_id.into()))) + { + Ok(()) => { + self.pending_dial.insert(peer_id, sender); + } + Err(e) => { + let _ = sender.send(Err(Box::new(e))); + } + } + } + } + Command::StartProviding { file_name, sender } => { + let query_id = self + .swarm + .behaviour_mut() + .kademlia + .start_providing(file_name.into_bytes().into()) + .expect("No store error."); + self.pending_start_providing.insert(query_id, sender); + } + Command::GetProviders { file_name, sender } => { + let query_id = self + .swarm + .behaviour_mut() + .kademlia + .get_providers(file_name.into_bytes().into()); + self.pending_get_providers.insert(query_id, sender); + } + Command::RequestFile { + file_name, + peer, + sender, + } => { + let request_id = self + .swarm + .behaviour_mut() + .request_response + .send_request(&peer, FileRequest(file_name)); + self.pending_request_file.insert(request_id, sender); + } + Command::RespondFile { file, channel } => { + self.swarm + .behaviour_mut() + .request_response + .send_response(channel, FileResponse(file)) + .expect("Connection to peer to be still open."); + } + } + } + } + + #[derive(NetworkBehaviour)] + #[behaviour(out_event = "ComposedEvent")] + struct ComposedBehaviour { + request_response: RequestResponse, + kademlia: Kademlia, + } + + #[derive(Debug)] + enum ComposedEvent { + RequestResponse(RequestResponseEvent), + Kademlia(KademliaEvent), + } + + impl From> for ComposedEvent { + fn from(event: RequestResponseEvent) -> Self { + ComposedEvent::RequestResponse(event) + } + } + + impl From for ComposedEvent { + fn from(event: KademliaEvent) -> Self { + ComposedEvent::Kademlia(event) + } + } + + #[derive(Debug)] + enum Command { + StartListening { + addr: Multiaddr, + sender: oneshot::Sender>>, + }, + Dial { + peer_id: PeerId, + peer_addr: Multiaddr, + sender: oneshot::Sender>>, + }, + StartProviding { + file_name: String, + sender: oneshot::Sender<()>, + }, + GetProviders { + file_name: String, + sender: oneshot::Sender>, + }, + RequestFile { + file_name: String, + peer: PeerId, + sender: oneshot::Sender>>, + }, + RespondFile { + file: String, + channel: ResponseChannel, + }, + } + + pub enum Event { + InboundRequest { + request: String, + channel: ResponseChannel, + }, + } + + // Simple file exchange protocol + + #[derive(Debug, Clone)] + struct FileExchangeProtocol(); + #[derive(Clone)] + struct FileExchangeCodec(); + #[derive(Debug, Clone, PartialEq, Eq)] + struct FileRequest(String); + #[derive(Debug, Clone, PartialEq, Eq)] + pub struct FileResponse(String); + + impl ProtocolName for FileExchangeProtocol { + fn protocol_name(&self) -> &[u8] { + "/file-exchange/1".as_bytes() + } + } + + #[async_trait] + impl RequestResponseCodec for FileExchangeCodec { + type Protocol = FileExchangeProtocol; + type Request = FileRequest; + type Response = FileResponse; + + async fn read_request( + &mut self, + _: &FileExchangeProtocol, + io: &mut T, + ) -> io::Result + where + T: AsyncRead + Unpin + Send, + { + let vec = read_length_prefixed(io, 1_000_000).await?; + + if vec.is_empty() { + return Err(io::ErrorKind::UnexpectedEof.into()); + } + + Ok(FileRequest(String::from_utf8(vec).unwrap())) + } + + async fn read_response( + &mut self, + _: &FileExchangeProtocol, + io: &mut T, + ) -> io::Result + where + T: AsyncRead + Unpin + Send, + { + let vec = read_length_prefixed(io, 1_000_000).await?; + + if vec.is_empty() { + return Err(io::ErrorKind::UnexpectedEof.into()); + } + + Ok(FileResponse(String::from_utf8(vec).unwrap())) + } + + async fn write_request( + &mut self, + _: &FileExchangeProtocol, + io: &mut T, + FileRequest(data): FileRequest, + ) -> io::Result<()> + where + T: AsyncWrite + Unpin + Send, + { + write_length_prefixed(io, data).await?; + io.close().await?; + + Ok(()) + } + + async fn write_response( + &mut self, + _: &FileExchangeProtocol, + io: &mut T, + FileResponse(data): FileResponse, + ) -> io::Result<()> + where + T: AsyncWrite + Unpin + Send, + { + write_length_prefixed(io, data).await?; + io.close().await?; + + Ok(()) + } + } +} diff --git a/examples/ipfs-private.rs b/examples/ipfs-private.rs index f3c7afd496c..2bf7811186b 100644 --- a/examples/ipfs-private.rs +++ b/examples/ipfs-private.rs @@ -41,8 +41,7 @@ use libp2p::{ identify::{Identify, IdentifyConfig, IdentifyEvent}, identity, multiaddr::Protocol, - noise, - ping::{self, Ping, PingConfig, PingEvent}, + noise, ping, pnet::{PnetConfig, PreSharedKey}, swarm::{NetworkBehaviourEventProcess, SwarmEvent}, tcp::TcpConfig, @@ -164,10 +163,11 @@ fn main() -> Result<(), Box> { // We create a custom network behaviour that combines gossipsub, ping and identify. #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct MyBehaviour { gossipsub: Gossipsub, identify: Identify, - ping: Ping, + ping: ping::Behaviour, } impl NetworkBehaviourEventProcess for MyBehaviour { @@ -196,14 +196,13 @@ fn main() -> Result<(), Box> { } } - impl NetworkBehaviourEventProcess for MyBehaviour { + impl NetworkBehaviourEventProcess for MyBehaviour { // Called when `ping` produces an event. - fn inject_event(&mut self, event: PingEvent) { - use ping::handler::{PingFailure, PingSuccess}; + fn inject_event(&mut self, event: ping::Event) { match event { - PingEvent { + ping::Event { peer, - result: Result::Ok(PingSuccess::Ping { rtt }), + result: Result::Ok(ping::Success::Ping { rtt }), } => { println!( "ping: rtt to {} is {} ms", @@ -211,29 +210,29 @@ fn main() -> Result<(), Box> { rtt.as_millis() ); } - PingEvent { + ping::Event { peer, - result: Result::Ok(PingSuccess::Pong), + result: Result::Ok(ping::Success::Pong), } => { println!("ping: pong from {}", peer.to_base58()); } - PingEvent { + ping::Event { peer, - result: Result::Err(PingFailure::Timeout), + result: Result::Err(ping::Failure::Timeout), } => { println!("ping: timeout to {}", peer.to_base58()); } - PingEvent { + ping::Event { peer, - result: Result::Err(PingFailure::Unsupported), + result: Result::Err(ping::Failure::Unsupported), } => { println!("ping: {} does not support ping protocol", peer.to_base58()); } - PingEvent { + ping::Event { peer, - result: Result::Err(PingFailure::Other { error }), + result: Result::Err(ping::Failure::Other { error }), } => { - println!("ping: failure with {}: {}", peer.to_base58(), error); + println!("ping: ping::Failure with {}: {}", peer.to_base58(), error); } } } @@ -255,7 +254,7 @@ fn main() -> Result<(), Box> { "/ipfs/0.1.0".into(), local_key.public(), )), - ping: Ping::new(PingConfig::new()), + ping: ping::Behaviour::new(ping::Config::new()), }; println!("Subscribing to {:?}", gossipsub_topic); diff --git a/examples/ping.rs b/examples/ping.rs index f38b4fc4011..ef5f538ed9d 100644 --- a/examples/ping.rs +++ b/examples/ping.rs @@ -42,9 +42,8 @@ use futures::executor::block_on; use futures::prelude::*; -use libp2p::ping::{Ping, PingConfig}; use libp2p::swarm::{Swarm, SwarmEvent}; -use libp2p::{identity, PeerId}; +use libp2p::{identity, ping, PeerId}; use std::error::Error; use std::task::Poll; @@ -60,7 +59,7 @@ fn main() -> Result<(), Box> { // For illustrative purposes, the ping protocol is configured to // keep the connection alive, so a continuous sequence of pings // can be observed. - let behaviour = Ping::new(PingConfig::new().with_keep_alive(true)); + let behaviour = ping::Behaviour::new(ping::Config::new().with_keep_alive(true)); let mut swarm = Swarm::new(transport, behaviour, local_peer_id); diff --git a/misc/metrics/CHANGELOG.md b/misc/metrics/CHANGELOG.md new file mode 100644 index 00000000000..cad93476b9a --- /dev/null +++ b/misc/metrics/CHANGELOG.md @@ -0,0 +1,3 @@ +## Version 0.1.0 [unreleased] + +- Add initial version. \ No newline at end of file diff --git a/misc/metrics/Cargo.toml b/misc/metrics/Cargo.toml new file mode 100644 index 00000000000..f34c33d9d94 --- /dev/null +++ b/misc/metrics/Cargo.toml @@ -0,0 +1,29 @@ +[package] +name = "libp2p-metrics" +edition = "2018" +description = "Metrics for libp2p" +version = "0.1.0" +authors = ["Max Inden "] +license = "MIT" +repository = "https://github.com/libp2p/rust-libp2p" +keywords = ["peer-to-peer", "libp2p", "networking"] +categories = ["network-programming", "asynchronous"] + +[features] +identify = ["libp2p-identify"] +kad = ["libp2p-kad"] +ping = ["libp2p-ping"] + +[dependencies] +libp2p-core= { version = "0.30.0", path = "../../core" } +libp2p-identify = { version = "0.31.0", path = "../../protocols/identify", optional = true } +libp2p-kad = { version = "0.32.0", path = "../../protocols/kad", optional = true } +libp2p-ping = { version = "0.31.0", path = "../../protocols/ping", optional = true } +libp2p-swarm = { version = "0.31.0", path = "../../swarm" } +open-metrics-client = "0.12.0" + +[dev-dependencies] +env_logger = "0.8.1" +futures = "0.3.1" +libp2p = { path = "../../", features = ["metrics"] } +tide = "0.16" diff --git a/misc/metrics/examples/metrics.rs b/misc/metrics/examples/metrics.rs new file mode 100644 index 00000000000..f7ad8cf0fc6 --- /dev/null +++ b/misc/metrics/examples/metrics.rs @@ -0,0 +1,115 @@ +// Copyright 2021 Protocol Labs. +// +// 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. + +//! Example demonstrating `libp2p-metrics`. +//! +//! In one terminal run: +//! +//! ``` +//! cargo run --example metrics +//! ``` +//! +//! In a second terminal run: +//! +//! ``` +//! cargo run --example metrics -- +//! ``` +//! +//! Where `` is replaced by the listen address of the +//! first node reported in the first terminal. Look for `NewListenAddr`. +//! +//! In a third terminal run: +//! +//! ``` +//! curl localhost:/metrics +//! ``` +//! +//! Where `` is replaced by the listen +//! port of the metrics server of the first or the second node. Look for +//! `tide::server Server listening on`. +//! +//! You should see a long list of metrics printed to the terminal. Check the +//! `libp2p_ping` metrics, they should be `>0`. + +use futures::executor::block_on; +use futures::stream::StreamExt; +use libp2p::metrics::{Metrics, Recorder}; +use libp2p::ping::{Ping, PingConfig}; +use libp2p::swarm::SwarmEvent; +use libp2p::{identity, PeerId, Swarm}; +use open_metrics_client::encoding::text::encode; +use open_metrics_client::registry::Registry; +use std::error::Error; +use std::sync::{Arc, Mutex}; +use std::thread; + +fn main() -> Result<(), Box> { + tide::log::start(); + + let local_key = identity::Keypair::generate_ed25519(); + let local_peer_id = PeerId::from(local_key.public()); + tide::log::info!("Local peer id: {:?}", local_peer_id); + + let mut swarm = Swarm::new( + block_on(libp2p::development_transport(local_key))?, + Ping::new(PingConfig::new().with_keep_alive(true)), + local_peer_id, + ); + swarm.listen_on("/ip4/0.0.0.0/tcp/0".parse()?)?; + if let Some(addr) = std::env::args().nth(1) { + let remote = addr.parse()?; + swarm.dial_addr(remote)?; + tide::log::info!("Dialed {}", addr) + } + + let mut metric_registry = Registry::default(); + let metrics = Metrics::new(&mut metric_registry); + thread::spawn(move || block_on(metrics_server(metric_registry))); + + block_on(async { + loop { + match swarm.select_next_some().await { + SwarmEvent::Behaviour(ping_event) => { + tide::log::info!("{:?}", ping_event); + metrics.record(&ping_event); + } + swarm_event => { + tide::log::info!("{:?}", swarm_event); + metrics.record(&swarm_event); + } + } + } + }) +} + +pub async fn metrics_server(registry: Registry) -> std::result::Result<(), std::io::Error> { + let mut app = tide::with_state(Arc::new(Mutex::new(registry))); + + app.at("/metrics") + .get(|req: tide::Request>>| async move { + let mut encoded = Vec::new(); + encode(&mut encoded, &req.state().lock().unwrap()).unwrap(); + Ok(String::from_utf8(encoded).unwrap()) + }); + + app.listen("0.0.0.0:0").await?; + + Ok(()) +} diff --git a/misc/metrics/src/identify.rs b/misc/metrics/src/identify.rs new file mode 100644 index 00000000000..af95dfc0ee1 --- /dev/null +++ b/misc/metrics/src/identify.rs @@ -0,0 +1,122 @@ +// Copyright 2021 Protocol Labs. +// +// 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 open_metrics_client::metrics::counter::Counter; +use open_metrics_client::metrics::histogram::{exponential_buckets, Histogram}; +use open_metrics_client::registry::Registry; +use std::iter; + +pub struct Metrics { + error: Counter, + pushed: Counter, + received: Counter, + received_info_listen_addrs: Histogram, + received_info_protocols: Histogram, + sent: Counter, +} + +impl Metrics { + pub fn new(registry: &mut Registry) -> Self { + let sub_registry = registry.sub_registry_with_prefix("identify"); + + let error = Counter::default(); + sub_registry.register( + "errors", + "Number of errors while attempting to identify the remote", + Box::new(error.clone()), + ); + + let pushed = Counter::default(); + sub_registry.register( + "pushed", + "Number of times identification information of the local node has \ + been actively pushed to a peer.", + Box::new(pushed.clone()), + ); + + let received = Counter::default(); + sub_registry.register( + "received", + "Number of times identification information has been received from \ + a peer", + Box::new(received.clone()), + ); + + let received_info_listen_addrs = + Histogram::new(iter::once(0.0).chain(exponential_buckets(1.0, 2.0, 9))); + sub_registry.register( + "received_info_listen_addrs", + "Number of listen addresses for remote peer received in \ + identification information", + Box::new(received_info_listen_addrs.clone()), + ); + + let received_info_protocols = + Histogram::new(iter::once(0.0).chain(exponential_buckets(1.0, 2.0, 9))); + sub_registry.register( + "received_info_protocols", + "Number of protocols supported by the remote peer received in \ + identification information", + Box::new(received_info_protocols.clone()), + ); + + let sent = Counter::default(); + sub_registry.register( + "sent", + "Number of times identification information of the local node has \ + been sent to a peer in response to an identification request", + Box::new(sent.clone()), + ); + + Self { + error, + pushed, + received, + received_info_listen_addrs, + received_info_protocols, + sent, + } + } +} + +impl super::Recorder for super::Metrics { + fn record(&self, event: &libp2p_identify::IdentifyEvent) { + match event { + libp2p_identify::IdentifyEvent::Error { .. } => { + self.identify.error.inc(); + } + libp2p_identify::IdentifyEvent::Pushed { .. } => { + self.identify.pushed.inc(); + } + libp2p_identify::IdentifyEvent::Received { info, .. } => { + self.identify.received.inc(); + self.identify + .received_info_protocols + .observe(info.protocols.len() as f64); + self.identify + .received_info_listen_addrs + .observe(info.listen_addrs.len() as f64); + } + libp2p_identify::IdentifyEvent::Sent { .. } => { + self.identify.sent.inc(); + } + } + } +} diff --git a/misc/metrics/src/kad.rs b/misc/metrics/src/kad.rs new file mode 100644 index 00000000000..6b5f5cf2db4 --- /dev/null +++ b/misc/metrics/src/kad.rs @@ -0,0 +1,427 @@ +// Copyright 2021 Protocol Labs. +// +// 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 open_metrics_client::encoding::text::Encode; +use open_metrics_client::metrics::counter::Counter; +use open_metrics_client::metrics::family::Family; +use open_metrics_client::metrics::histogram::{exponential_buckets, Histogram}; +use open_metrics_client::registry::{Registry, Unit}; + +pub struct Metrics { + query_result_get_record_ok: Histogram, + query_result_get_record_error: Family, + + query_result_get_closest_peers_ok: Histogram, + query_result_get_closest_peers_error: Family, + + query_result_get_providers_ok: Histogram, + query_result_get_providers_error: Family, + + query_result_num_requests: Family, + query_result_num_success: Family, + query_result_num_failure: Family, + query_result_duration: Family, + + routing_updated: Family, + + inbound_requests: Family, +} + +impl Metrics { + pub fn new(registry: &mut Registry) -> Self { + let sub_registry = registry.sub_registry_with_prefix("kad"); + + let query_result_get_record_ok = Histogram::new(exponential_buckets(1.0, 2.0, 10)); + sub_registry.register( + "query_result_get_record_ok", + "Number of records returned by a successful Kademlia get record query.", + Box::new(query_result_get_record_ok.clone()), + ); + + let query_result_get_record_error = Family::default(); + sub_registry.register( + "query_result_get_record_error", + "Number of failed Kademlia get record queries.", + Box::new(query_result_get_record_error.clone()), + ); + + let query_result_get_closest_peers_ok = Histogram::new(exponential_buckets(1.0, 2.0, 10)); + sub_registry.register( + "query_result_get_closest_peers_ok", + "Number of closest peers returned by a successful Kademlia get closest peers query.", + Box::new(query_result_get_closest_peers_ok.clone()), + ); + + let query_result_get_closest_peers_error = Family::default(); + sub_registry.register( + "query_result_get_closest_peers_error", + "Number of failed Kademlia get closest peers queries.", + Box::new(query_result_get_closest_peers_error.clone()), + ); + + let query_result_get_providers_ok = Histogram::new(exponential_buckets(1.0, 2.0, 10)); + sub_registry.register( + "query_result_get_providers_ok", + "Number of providers returned by a successful Kademlia get providers query.", + Box::new(query_result_get_providers_ok.clone()), + ); + + let query_result_get_providers_error = Family::default(); + sub_registry.register( + "query_result_get_providers_error", + "Number of failed Kademlia get providers queries.", + Box::new(query_result_get_providers_error.clone()), + ); + + let query_result_num_requests = + Family::new_with_constructor(|| Histogram::new(exponential_buckets(1.0, 2.0, 10))); + sub_registry.register( + "query_result_num_requests", + "Number of requests started for a Kademlia query.", + Box::new(query_result_num_requests.clone()), + ); + + let query_result_num_success = + Family::new_with_constructor(|| Histogram::new(exponential_buckets(1.0, 2.0, 10))); + sub_registry.register( + "query_result_num_success", + "Number of successful requests of a Kademlia query.", + Box::new(query_result_num_success.clone()), + ); + + let query_result_num_failure = + Family::new_with_constructor(|| Histogram::new(exponential_buckets(1.0, 2.0, 10))); + sub_registry.register( + "query_result_num_failure", + "Number of failed requests of a Kademlia query.", + Box::new(query_result_num_failure.clone()), + ); + + let query_result_duration = + Family::new_with_constructor(|| Histogram::new(exponential_buckets(0.1, 2.0, 10))); + sub_registry.register_with_unit( + "query_result_duration", + "Duration of a Kademlia query.", + Unit::Seconds, + Box::new(query_result_duration.clone()), + ); + + let routing_updated = Family::default(); + sub_registry.register( + "routing_updated", + "Number of peers added, updated or evicted to, in or from a specific kbucket in the routing table", + Box::new(routing_updated.clone()), + ); + + let inbound_requests = Family::default(); + sub_registry.register( + "inbound_requests", + "Number of inbound requests", + Box::new(inbound_requests.clone()), + ); + + Self { + query_result_get_record_ok, + query_result_get_record_error, + + query_result_get_closest_peers_ok, + query_result_get_closest_peers_error, + + query_result_get_providers_ok, + query_result_get_providers_error, + + query_result_num_requests, + query_result_num_success, + query_result_num_failure, + query_result_duration, + + routing_updated, + + inbound_requests, + } + } +} + +impl super::Recorder for super::Metrics { + fn record(&self, event: &libp2p_kad::KademliaEvent) { + match event { + libp2p_kad::KademliaEvent::OutboundQueryCompleted { result, stats, .. } => { + self.kad + .query_result_num_requests + .get_or_create(&result.into()) + .observe(stats.num_requests().into()); + self.kad + .query_result_num_success + .get_or_create(&result.into()) + .observe(stats.num_successes().into()); + self.kad + .query_result_num_failure + .get_or_create(&result.into()) + .observe(stats.num_failures().into()); + if let Some(duration) = stats.duration() { + self.kad + .query_result_duration + .get_or_create(&result.into()) + .observe(duration.as_secs_f64()); + } + + match result { + libp2p_kad::QueryResult::GetRecord(result) => match result { + Ok(ok) => self + .kad + .query_result_get_record_ok + .observe(ok.records.len() as f64), + Err(error) => { + self.kad + .query_result_get_record_error + .get_or_create(&error.into()) + .inc(); + } + }, + libp2p_kad::QueryResult::GetClosestPeers(result) => match result { + Ok(ok) => self + .kad + .query_result_get_closest_peers_ok + .observe(ok.peers.len() as f64), + Err(error) => { + self.kad + .query_result_get_closest_peers_error + .get_or_create(&error.into()) + .inc(); + } + }, + libp2p_kad::QueryResult::GetProviders(result) => match result { + Ok(ok) => self + .kad + .query_result_get_providers_ok + .observe(ok.providers.len() as f64), + Err(error) => { + self.kad + .query_result_get_providers_error + .get_or_create(&error.into()) + .inc(); + } + }, + _ => {} + } + } + libp2p_kad::KademliaEvent::RoutingUpdated { + is_new_peer, + old_peer, + bucket_range: (low, _high), + .. + } => { + let bucket = low.ilog2().unwrap_or(0); + if *is_new_peer { + self.kad + .routing_updated + .get_or_create(&RoutingUpdated { + action: RoutingAction::Added, + bucket, + }) + .inc(); + } else { + self.kad + .routing_updated + .get_or_create(&RoutingUpdated { + action: RoutingAction::Updated, + bucket, + }) + .inc(); + } + + if old_peer.is_some() { + self.kad + .routing_updated + .get_or_create(&RoutingUpdated { + action: RoutingAction::Evicted, + bucket, + }) + .inc(); + } + } + + libp2p_kad::KademliaEvent::InboundRequestServed { request } => { + self.kad + .inbound_requests + .get_or_create(&request.into()) + .inc(); + } + _ => {} + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct QueryResult { + r#type: QueryType, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum QueryType { + Bootstrap, + GetClosestPeers, + GetProviders, + StartProviding, + RepublishProvider, + GetRecord, + PutRecord, + RepublishRecord, +} + +impl From<&libp2p_kad::QueryResult> for QueryResult { + fn from(result: &libp2p_kad::QueryResult) -> Self { + match result { + libp2p_kad::QueryResult::Bootstrap(_) => QueryResult { + r#type: QueryType::Bootstrap, + }, + libp2p_kad::QueryResult::GetClosestPeers(_) => QueryResult { + r#type: QueryType::GetClosestPeers, + }, + libp2p_kad::QueryResult::GetProviders(_) => QueryResult { + r#type: QueryType::GetProviders, + }, + libp2p_kad::QueryResult::StartProviding(_) => QueryResult { + r#type: QueryType::StartProviding, + }, + libp2p_kad::QueryResult::RepublishProvider(_) => QueryResult { + r#type: QueryType::RepublishProvider, + }, + libp2p_kad::QueryResult::GetRecord(_) => QueryResult { + r#type: QueryType::GetRecord, + }, + libp2p_kad::QueryResult::PutRecord(_) => QueryResult { + r#type: QueryType::PutRecord, + }, + libp2p_kad::QueryResult::RepublishRecord(_) => QueryResult { + r#type: QueryType::RepublishRecord, + }, + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct GetRecordResult { + error: GetRecordError, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum GetRecordError { + NotFound, + QuorumFailed, + Timeout, +} + +impl From<&libp2p_kad::GetRecordError> for GetRecordResult { + fn from(error: &libp2p_kad::GetRecordError) -> Self { + match error { + libp2p_kad::GetRecordError::NotFound { .. } => GetRecordResult { + error: GetRecordError::NotFound, + }, + libp2p_kad::GetRecordError::QuorumFailed { .. } => GetRecordResult { + error: GetRecordError::QuorumFailed, + }, + libp2p_kad::GetRecordError::Timeout { .. } => GetRecordResult { + error: GetRecordError::Timeout, + }, + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct GetClosestPeersResult { + error: GetClosestPeersError, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum GetClosestPeersError { + Timeout, +} + +impl From<&libp2p_kad::GetClosestPeersError> for GetClosestPeersResult { + fn from(error: &libp2p_kad::GetClosestPeersError) -> Self { + match error { + libp2p_kad::GetClosestPeersError::Timeout { .. } => GetClosestPeersResult { + error: GetClosestPeersError::Timeout, + }, + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct GetProvidersResult { + error: GetProvidersError, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum GetProvidersError { + Timeout, +} + +impl From<&libp2p_kad::GetProvidersError> for GetProvidersResult { + fn from(error: &libp2p_kad::GetProvidersError) -> Self { + match error { + libp2p_kad::GetProvidersError::Timeout { .. } => GetProvidersResult { + error: GetProvidersError::Timeout, + }, + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct RoutingUpdated { + action: RoutingAction, + bucket: u32, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum RoutingAction { + Added, + Updated, + Evicted, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct InboundRequest { + request: Request, +} + +impl From<&libp2p_kad::InboundRequest> for InboundRequest { + fn from(request: &libp2p_kad::InboundRequest) -> Self { + Self { + request: match request { + libp2p_kad::InboundRequest::FindNode { .. } => Request::FindNode, + libp2p_kad::InboundRequest::GetProvider { .. } => Request::GetProvider, + libp2p_kad::InboundRequest::AddProvider { .. } => Request::AddProvider, + libp2p_kad::InboundRequest::GetRecord { .. } => Request::GetRecord, + libp2p_kad::InboundRequest::PutRecord { .. } => Request::PutRecord, + }, + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum Request { + FindNode, + GetProvider, + AddProvider, + GetRecord, + PutRecord, +} diff --git a/misc/metrics/src/lib.rs b/misc/metrics/src/lib.rs new file mode 100644 index 00000000000..4582904ca49 --- /dev/null +++ b/misc/metrics/src/lib.rs @@ -0,0 +1,76 @@ +// Copyright 2021 Protocol Labs. +// +// 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. + +//! Auxiliary crate recording protocol and Swarm events and exposing them as +//! metrics in the [OpenMetrics] format. +//! +//! [OpenMetrics]: https://github.com/OpenObservability/OpenMetrics/ +//! +//! See `examples` directory for more. + +#[cfg(feature = "identify")] +mod identify; +#[cfg(feature = "kad")] +mod kad; +#[cfg(feature = "ping")] +mod ping; +mod swarm; + +use open_metrics_client::registry::Registry; + +/// Set of Swarm and protocol metrics derived from emitted events. +pub struct Metrics { + #[cfg(feature = "identify")] + identify: identify::Metrics, + #[cfg(feature = "kad")] + kad: kad::Metrics, + #[cfg(feature = "ping")] + ping: ping::Metrics, + swarm: swarm::Metrics, +} + +impl Metrics { + /// Create a new set of Swarm and protocol [`Metrics`]. + /// + /// ``` + /// use open_metrics_client::registry::Registry; + /// use libp2p_metrics::Metrics; + /// let mut registry = Registry::default(); + /// let metrics = Metrics::new(&mut registry); + /// ``` + pub fn new(registry: &mut Registry) -> Self { + let sub_registry = registry.sub_registry_with_prefix("libp2p"); + Self { + #[cfg(feature = "identify")] + identify: identify::Metrics::new(sub_registry), + #[cfg(feature = "kad")] + kad: kad::Metrics::new(sub_registry), + #[cfg(feature = "ping")] + ping: ping::Metrics::new(sub_registry), + swarm: swarm::Metrics::new(sub_registry), + } + } +} + +/// Recorder that can record Swarm and protocol events. +pub trait Recorder { + /// Record the given event. + fn record(&self, event: &Event); +} diff --git a/misc/metrics/src/ping.rs b/misc/metrics/src/ping.rs new file mode 100644 index 00000000000..abf7ec8e494 --- /dev/null +++ b/misc/metrics/src/ping.rs @@ -0,0 +1,109 @@ +// Copyright 2021 Protocol Labs. +// +// 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 open_metrics_client::encoding::text::Encode; +use open_metrics_client::metrics::counter::Counter; +use open_metrics_client::metrics::family::Family; +use open_metrics_client::metrics::histogram::{exponential_buckets, Histogram}; +use open_metrics_client::registry::{Registry, Unit}; + +#[derive(Clone, Hash, PartialEq, Eq, Encode)] +struct FailureLabels { + reason: Failure, +} + +impl From<&libp2p_ping::PingFailure> for FailureLabels { + fn from(failure: &libp2p_ping::PingFailure) -> Self { + match failure { + libp2p_ping::PingFailure::Timeout => FailureLabels { + reason: Failure::Timeout, + }, + libp2p_ping::PingFailure::Unsupported => FailureLabels { + reason: Failure::Unsupported, + }, + libp2p_ping::PingFailure::Other { .. } => FailureLabels { + reason: Failure::Other, + }, + } + } +} + +#[derive(Clone, Hash, PartialEq, Eq, Encode)] +enum Failure { + Timeout, + Unsupported, + Other, +} + +pub struct Metrics { + rtt: Histogram, + failure: Family, + pong_received: Counter, +} + +impl Metrics { + pub fn new(registry: &mut Registry) -> Self { + let sub_registry = registry.sub_registry_with_prefix("ping"); + + let rtt = Histogram::new(exponential_buckets(0.001, 2.0, 12)); + sub_registry.register_with_unit( + "rtt", + "Round-trip time sending a 'ping' and receiving a 'pong'", + Unit::Seconds, + Box::new(rtt.clone()), + ); + + let failure = Family::default(); + sub_registry.register( + "failure", + "Failure while sending a 'ping' or receiving a 'pong'", + Box::new(failure.clone()), + ); + + let pong_received = Counter::default(); + sub_registry.register( + "pong_received", + "Number of 'pong's received", + Box::new(pong_received.clone()), + ); + + Self { + rtt, + failure, + pong_received, + } + } +} + +impl super::Recorder for super::Metrics { + fn record(&self, event: &libp2p_ping::PingEvent) { + match &event.result { + Ok(libp2p_ping::PingSuccess::Pong) => { + self.ping.pong_received.inc(); + } + Ok(libp2p_ping::PingSuccess::Ping { rtt }) => { + self.ping.rtt.observe(rtt.as_secs_f64()); + } + Err(failure) => { + self.ping.failure.get_or_create(&failure.into()).inc(); + } + } + } +} diff --git a/misc/metrics/src/swarm.rs b/misc/metrics/src/swarm.rs new file mode 100644 index 00000000000..a0c9c72fedf --- /dev/null +++ b/misc/metrics/src/swarm.rs @@ -0,0 +1,267 @@ +// Copyright 2021 Protocol Labs. +// +// 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 open_metrics_client::encoding::text::Encode; +use open_metrics_client::metrics::counter::Counter; +use open_metrics_client::metrics::family::Family; +use open_metrics_client::registry::Registry; + +pub struct Metrics { + connections_incoming: Counter, + connections_incoming_error: Family, + + connections_established: Family, + connections_closed: Family, + + new_listen_addr: Counter, + expired_listen_addr: Counter, + + listener_closed: Counter, + listener_error: Counter, + + dial_attempt: Counter, + dial_unreachable_addr: Family, Counter>, + connected_to_banned_peer: Counter, +} + +impl Metrics { + pub fn new(registry: &mut Registry) -> Self { + let sub_registry = registry.sub_registry_with_prefix("swarm"); + + let connections_incoming = Counter::default(); + sub_registry.register( + "connections_incoming", + "Number of incoming connections", + Box::new(connections_incoming.clone()), + ); + + let connections_incoming_error = Family::default(); + sub_registry.register( + "connections_incoming_error", + "Number of incoming connection errors", + Box::new(connections_incoming_error.clone()), + ); + + let new_listen_addr = Counter::default(); + sub_registry.register( + "new_listen_addr", + "Number of new listen addresses", + Box::new(new_listen_addr.clone()), + ); + + let expired_listen_addr = Counter::default(); + sub_registry.register( + "expired_listen_addr", + "Number of expired listen addresses", + Box::new(expired_listen_addr.clone()), + ); + + let listener_closed = Counter::default(); + sub_registry.register( + "listener_closed", + "Number of listeners closed", + Box::new(listener_closed.clone()), + ); + + let listener_error = Counter::default(); + sub_registry.register( + "listener_error", + "Number of listener errors", + Box::new(listener_error.clone()), + ); + + let dial_attempt = Counter::default(); + sub_registry.register( + "dial_attempt", + "Number of dial attempts", + Box::new(dial_attempt.clone()), + ); + + let dial_unreachable_addr = Family::default(); + sub_registry.register( + "dial_unreachable_addr", + "Number of unreachable addresses dialed", + Box::new(dial_unreachable_addr.clone()), + ); + + let connected_to_banned_peer = Counter::default(); + sub_registry.register( + "connected_to_banned_peer", + "Number of connection attempts to banned peer", + Box::new(connected_to_banned_peer.clone()), + ); + + let connections_established = Family::default(); + sub_registry.register( + "connections_established", + "Number of connections established", + Box::new(connections_established.clone()), + ); + + let connections_closed = Family::default(); + sub_registry.register( + "connections_closed", + "Number of connections closed", + Box::new(connections_closed.clone()), + ); + + Self { + connections_incoming, + connections_incoming_error, + connections_established, + connections_closed, + new_listen_addr, + expired_listen_addr, + listener_closed, + listener_error, + dial_attempt, + dial_unreachable_addr, + connected_to_banned_peer, + } + } +} + +impl super::Recorder> + for super::Metrics +{ + fn record(&self, event: &libp2p_swarm::SwarmEvent) { + match event { + libp2p_swarm::SwarmEvent::Behaviour(_) => {} + libp2p_swarm::SwarmEvent::ConnectionEstablished { endpoint, .. } => { + self.swarm + .connections_established + .get_or_create(&ConnectionEstablishedLabels { + role: endpoint.into(), + }) + .inc(); + } + libp2p_swarm::SwarmEvent::ConnectionClosed { endpoint, .. } => { + self.swarm + .connections_closed + .get_or_create(&ConnectionClosedLabels { + role: endpoint.into(), + }) + .inc(); + } + libp2p_swarm::SwarmEvent::IncomingConnection { .. } => { + self.swarm.connections_incoming.inc(); + } + libp2p_swarm::SwarmEvent::IncomingConnectionError { error, .. } => { + self.swarm + .connections_incoming_error + .get_or_create(&IncomingConnectionErrorLabels { + error: error.into(), + }) + .inc(); + } + libp2p_swarm::SwarmEvent::BannedPeer { .. } => { + self.swarm.connected_to_banned_peer.inc(); + } + libp2p_swarm::SwarmEvent::UnreachableAddr { .. } => { + self.swarm + .dial_unreachable_addr + .get_or_create(&vec![("peer".into(), "known".into())]) + .inc(); + } + libp2p_swarm::SwarmEvent::UnknownPeerUnreachableAddr { .. } => { + self.swarm + .dial_unreachable_addr + .get_or_create(&vec![("peer".into(), "unknown".into())]) + .inc(); + } + libp2p_swarm::SwarmEvent::NewListenAddr { .. } => { + self.swarm.new_listen_addr.inc(); + } + libp2p_swarm::SwarmEvent::ExpiredListenAddr { .. } => { + self.swarm.expired_listen_addr.inc(); + } + libp2p_swarm::SwarmEvent::ListenerClosed { .. } => { + self.swarm.listener_closed.inc(); + } + libp2p_swarm::SwarmEvent::ListenerError { .. } => { + self.swarm.listener_error.inc(); + } + libp2p_swarm::SwarmEvent::Dialing(_) => { + self.swarm.dial_attempt.inc(); + } + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct ConnectionEstablishedLabels { + role: Role, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct ConnectionClosedLabels { + role: Role, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum Role { + Dialer, + Listener, +} + +impl From<&libp2p_core::ConnectedPoint> for Role { + fn from(point: &libp2p_core::ConnectedPoint) -> Self { + match point { + libp2p_core::ConnectedPoint::Dialer { .. } => Role::Dialer, + libp2p_core::ConnectedPoint::Listener { .. } => Role::Listener, + } + } +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +struct IncomingConnectionErrorLabels { + error: PendingConnectionError, +} + +#[derive(Encode, Hash, Clone, Eq, PartialEq)] +enum PendingConnectionError { + InvalidPeerId, + TransportErrorMultiaddrNotSupported, + TransportErrorOther, + Aborted, + Io, +} + +impl From<&libp2p_core::connection::PendingConnectionError> + for PendingConnectionError +{ + fn from(point: &libp2p_core::connection::PendingConnectionError) -> Self { + match point { + libp2p_core::connection::PendingConnectionError::InvalidPeerId => { + PendingConnectionError::InvalidPeerId + } + libp2p_core::connection::PendingConnectionError::Transport( + libp2p_core::transport::TransportError::MultiaddrNotSupported(_), + ) => PendingConnectionError::TransportErrorMultiaddrNotSupported, + libp2p_core::connection::PendingConnectionError::Transport( + libp2p_core::transport::TransportError::Other(_), + ) => PendingConnectionError::TransportErrorOther, + libp2p_core::connection::PendingConnectionError::Aborted => { + PendingConnectionError::Aborted + } + libp2p_core::connection::PendingConnectionError::IO(_) => PendingConnectionError::Io, + } + } +} diff --git a/misc/multistream-select/Cargo.toml b/misc/multistream-select/Cargo.toml index f839e3be123..2f6a72db9ef 100644 --- a/misc/multistream-select/Cargo.toml +++ b/misc/multistream-select/Cargo.toml @@ -20,7 +20,7 @@ unsigned-varint = "0.7" [dev-dependencies] async-std = "1.6.2" env_logger = "0.9" -libp2p-core = { path = "../../core" } +libp2p-core = { path = "../../core", default-features = false } libp2p-mplex = { path = "../../muxers/mplex" } libp2p-plaintext = { path = "../../transports/plaintext" } quickcheck = "0.9.0" diff --git a/misc/multistream-select/src/length_delimited.rs b/misc/multistream-select/src/length_delimited.rs index abb622eed30..981dfff82f8 100644 --- a/misc/multistream-select/src/length_delimited.rs +++ b/misc/multistream-select/src/length_delimited.rs @@ -125,7 +125,7 @@ impl LengthDelimited { let mut this = self.project(); while !this.write_buffer.is_empty() { - match this.inner.as_mut().poll_write(cx, &this.write_buffer) { + match this.inner.as_mut().poll_write(cx, this.write_buffer) { Poll::Pending => return Poll::Pending, Poll::Ready(Ok(0)) => { return Poll::Ready(Err(io::Error::new( diff --git a/misc/peer-id-generator/Cargo.toml b/misc/peer-id-generator/Cargo.toml index 1c23bf1f472..da1cad6ba91 100644 --- a/misc/peer-id-generator/Cargo.toml +++ b/misc/peer-id-generator/Cargo.toml @@ -11,5 +11,5 @@ categories = ["network-programming", "asynchronous"] publish = false [dependencies] -libp2p-core = { path = "../../core" } +libp2p-core = { path = "../../core", default-features = false } num_cpus = "1.8" diff --git a/muxers/mplex/CHANGELOG.md b/muxers/mplex/CHANGELOG.md index b0b697ae492..6cc5d5c6b0b 100644 --- a/muxers/mplex/CHANGELOG.md +++ b/muxers/mplex/CHANGELOG.md @@ -1,5 +1,7 @@ # 0.30.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) - Update dependencies. # 0.29.0 [2021-07-12] diff --git a/muxers/mplex/Cargo.toml b/muxers/mplex/Cargo.toml index df0033dea53..285fe386517 100644 --- a/muxers/mplex/Cargo.toml +++ b/muxers/mplex/Cargo.toml @@ -13,7 +13,7 @@ categories = ["network-programming", "asynchronous"] bytes = "1" futures = "0.3.1" asynchronous-codec = "0.6" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } log = "0.4" nohash-hasher = "0.2" parking_lot = "0.11" diff --git a/muxers/mplex/src/io.rs b/muxers/mplex/src/io.rs index 80da197a965..e475b15763b 100644 --- a/muxers/mplex/src/io.rs +++ b/muxers/mplex/src/io.rs @@ -1104,7 +1104,6 @@ mod tests { use async_std::task; use asynchronous_codec::{Decoder, Encoder}; use bytes::BytesMut; - use futures::prelude::*; use quickcheck::*; use rand::prelude::*; use std::collections::HashSet; diff --git a/muxers/yamux/CHANGELOG.md b/muxers/yamux/CHANGELOG.md index 697f3636f9f..7ca9c0bbbb6 100644 --- a/muxers/yamux/CHANGELOG.md +++ b/muxers/yamux/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.34.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. - Implement `From for YamuxError` instead of `Into`. diff --git a/muxers/yamux/Cargo.toml b/muxers/yamux/Cargo.toml index 28a47cb58cf..c9984b6514a 100644 --- a/muxers/yamux/Cargo.toml +++ b/muxers/yamux/Cargo.toml @@ -11,7 +11,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } parking_lot = "0.11" thiserror = "1.0" yamux = "0.9.0" diff --git a/protocols/floodsub/CHANGELOG.md b/protocols/floodsub/CHANGELOG.md index 6abdd6f810d..c0f65131c1a 100644 --- a/protocols/floodsub/CHANGELOG.md +++ b/protocols/floodsub/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.31.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.30.0 [2021-07-12] diff --git a/protocols/floodsub/Cargo.toml b/protocols/floodsub/Cargo.toml index b4b5b51f954..0b8bf67b0b2 100644 --- a/protocols/floodsub/Cargo.toml +++ b/protocols/floodsub/Cargo.toml @@ -13,7 +13,7 @@ categories = ["network-programming", "asynchronous"] cuckoofilter = "0.5.0" fnv = "1.0" futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } libp2p-swarm = { version = "0.31.0", path = "../../swarm" } log = "0.4" prost = "0.8" diff --git a/protocols/floodsub/src/layer.rs b/protocols/floodsub/src/layer.rs index eb5a7cb30b2..25b235e3364 100644 --- a/protocols/floodsub/src/layer.rs +++ b/protocols/floodsub/src/layer.rs @@ -29,7 +29,7 @@ use fnv::FnvHashSet; use libp2p_core::{connection::ConnectionId, PeerId}; use libp2p_swarm::{ DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, OneShotHandler, - PollParameters, ProtocolsHandler, + PollParameters, }; use log::warn; use smallvec::SmallVec; @@ -40,7 +40,12 @@ use std::{collections::VecDeque, iter}; /// Network behaviour that handles the floodsub protocol. pub struct Floodsub { /// Events that need to be yielded to the outside when polling. - events: VecDeque>, + events: VecDeque< + NetworkBehaviourAction< + FloodsubEvent, + OneShotHandler, + >, + >, config: FloodsubConfig, @@ -101,9 +106,11 @@ impl Floodsub { } if self.target_peers.insert(peer_id) { + let handler = self.new_handler(); self.events.push_back(NetworkBehaviourAction::DialPeer { peer_id, condition: DialPeerCondition::Disconnected, + handler, }); } } @@ -302,9 +309,11 @@ impl NetworkBehaviour for Floodsub { // We can be disconnected by the remote in case of inactivity for example, so we always // try to reconnect. if self.target_peers.contains(id) { + let handler = self.new_handler(); self.events.push_back(NetworkBehaviourAction::DialPeer { peer_id: *id, condition: DialPeerCondition::Disconnected, + handler, }); } } @@ -426,12 +435,7 @@ impl NetworkBehaviour for Floodsub { &mut self, _: &mut Context<'_>, _: &mut impl PollParameters, - ) -> Poll< - NetworkBehaviourAction< - ::InEvent, - Self::OutEvent, - >, - > { + ) -> Poll> { if let Some(event) = self.events.pop_front() { return Poll::Ready(event); } diff --git a/protocols/gossipsub/CHANGELOG.md b/protocols/gossipsub/CHANGELOG.md index 350c3d7a971..da528c79b42 100644 --- a/protocols/gossipsub/CHANGELOG.md +++ b/protocols/gossipsub/CHANGELOG.md @@ -1,5 +1,11 @@ # 0.33.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + +- Improve internal peer tracking. + [PR 2175](https://github.com/libp2p/rust-libp2p/pull/2175) + - Update dependencies. - Allow `message_id_fn`s to accept closures that capture variables. diff --git a/protocols/gossipsub/Cargo.toml b/protocols/gossipsub/Cargo.toml index 1b99bfbf2e7..115051a5ef4 100644 --- a/protocols/gossipsub/Cargo.toml +++ b/protocols/gossipsub/Cargo.toml @@ -11,7 +11,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] libp2p-swarm = { version = "0.31.0", path = "../../swarm" } -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } bytes = "1.0" byteorder = "1.3.4" fnv = "1.0.7" diff --git a/protocols/gossipsub/src/behaviour.rs b/protocols/gossipsub/src/behaviour.rs index a4f4ec24bfc..8da39b7e8de 100644 --- a/protocols/gossipsub/src/behaviour.rs +++ b/protocols/gossipsub/src/behaviour.rs @@ -41,8 +41,8 @@ use libp2p_core::{ multiaddr::Protocol::Ip6, ConnectedPoint, Multiaddr, PeerId, }; use libp2p_swarm::{ - DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, - ProtocolsHandler, + DialPeerCondition, IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, + NotifyHandler, PollParameters, }; use crate::backoff::BackoffStorage; @@ -157,8 +157,8 @@ enum PublishConfig { impl PublishConfig { pub fn get_own_id(&self) -> Option<&PeerId> { match self { - Self::Signing { author, .. } => Some(&author), - Self::Author(author) => Some(&author), + Self::Signing { author, .. } => Some(author), + Self::Author(author) => Some(author), _ => None, } } @@ -193,7 +193,7 @@ impl From for PublishConfig { } type GossipsubNetworkBehaviourAction = - NetworkBehaviourAction, GossipsubEvent>; + NetworkBehaviourAction>; /// Network behaviour that handles the gossipsub protocol. /// @@ -381,7 +381,7 @@ where // We do not allow configurations where a published message would also be rejected if it // were received locally. - validate_config(&privacy, &config.validation_mode())?; + validate_config(&privacy, config.validation_mode())?; // Set up message publishing parameters. @@ -425,8 +425,8 @@ where impl Gossipsub where - D: DataTransform, - F: TopicSubscriptionFilter, + D: DataTransform + Send + 'static, + F: TopicSubscriptionFilter + Send + 'static, { /// Lists the hashes of the topics we are currently subscribed to. pub fn topics(&self) -> impl Iterator { @@ -990,7 +990,7 @@ where get_random_peers( &self.topic_peers, &self.connected_peers, - &topic_hash, + topic_hash, self.config.prune_peers(), |p| p != peer && !self.score_below_threshold(p, |_| 0.0).0, ) @@ -1043,9 +1043,11 @@ where if !self.peer_topics.contains_key(peer_id) { // Connect to peer debug!("Connecting to explicit peer {:?}", peer_id); + let handler = self.new_handler(); self.events.push_back(NetworkBehaviourAction::DialPeer { peer_id: *peer_id, condition: DialPeerCondition::Disconnected, + handler, }); } } @@ -1241,6 +1243,15 @@ where let mut do_px = self.config.do_px(); + // For each topic, if a peer has grafted us, then we necessarily must be in their mesh + // and they must be subscribed to the topic. Ensure we have recorded the mapping. + for topic in &topics { + self.peer_topics + .entry(*peer_id) + .or_default() + .insert(topic.clone()); + } + // we don't GRAFT to/from explicit peers; complain loudly if this happens if self.explicit_peers.contains(peer_id) { warn!("GRAFT: ignoring request from direct peer {}", peer_id); @@ -1283,7 +1294,7 @@ where peer_score.add_penalty(peer_id, 1); } } - //no PX + // no PX do_px = false; to_prune_topics.insert(topic_hash.clone()); @@ -1326,7 +1337,7 @@ where *peer_id, vec![&topic_hash], &self.mesh, - self.peer_topics.get(&peer_id), + self.peer_topics.get(peer_id), &mut self.events, &self.connected_peers, ); @@ -1385,7 +1396,7 @@ where always_update_backoff: bool, ) { let mut update_backoff = always_update_backoff; - if let Some(peers) = self.mesh.get_mut(&topic_hash) { + if let Some(peers) = self.mesh.get_mut(topic_hash) { // remove the peer if it exists in the mesh if peers.remove(peer_id) { debug!( @@ -1405,7 +1416,7 @@ where *peer_id, topic_hash, &self.mesh, - self.peer_topics.get(&peer_id), + self.peer_topics.get(peer_id), &mut self.events, &self.connected_peers, ); @@ -1418,7 +1429,7 @@ where self.config.prune_backoff() }; // is there a backoff specified by the peer? if so obey it. - self.backoffs.update_backoff(&topic_hash, peer_id, time); + self.backoffs.update_backoff(topic_hash, peer_id, time); } } @@ -1484,9 +1495,11 @@ where self.px_peers.insert(peer_id); // dial peer + let handler = self.new_handler(); self.events.push_back(NetworkBehaviourAction::DialPeer { peer_id, condition: DialPeerCondition::Disconnected, + handler, }); } } @@ -1557,7 +1570,7 @@ where own_id != propagation_source && raw_message.source.as_ref().map_or(false, |s| s == own_id) } else { - self.published_message_ids.contains(&msg_id) + self.published_message_ids.contains(msg_id) }; if self_published { @@ -2163,7 +2176,7 @@ where "HEARTBEAT: Fanout topic removed due to timeout. Topic: {:?}", topic_hash ); - fanout.remove(&topic_hash); + fanout.remove(topic_hash); return false; } true @@ -2182,7 +2195,7 @@ where // is the peer still subscribed to the topic? match self.peer_topics.get(peer) { Some(topics) => { - if !topics.contains(&topic_hash) || score(peer) < publish_threshold { + if !topics.contains(topic_hash) || score(peer) < publish_threshold { debug!( "HEARTBEAT: Peer removed from fanout for topic: {:?}", topic_hash @@ -2278,7 +2291,7 @@ where fn emit_gossip(&mut self) { let mut rng = thread_rng(); for (topic_hash, peers) in self.mesh.iter().chain(self.fanout.iter()) { - let mut message_ids = self.mcache.get_gossip_message_ids(&topic_hash); + let mut message_ids = self.mcache.get_gossip_message_ids(topic_hash); if message_ids.is_empty() { return; } @@ -2306,7 +2319,7 @@ where let to_msg_peers = get_random_peers_dynamic( &self.topic_peers, &self.connected_peers, - &topic_hash, + topic_hash, n_map, |peer| { !peers.contains(peer) @@ -2425,7 +2438,7 @@ where *peer, topic_hash, &self.mesh, - self.peer_topics.get(&peer), + self.peer_topics.get(peer), &mut self.events, &self.connected_peers, ); @@ -2470,7 +2483,7 @@ where // add mesh peers let topic = &message.topic; // mesh - if let Some(mesh_peers) = self.mesh.get(&topic) { + if let Some(mesh_peers) = self.mesh.get(topic) { for peer_id in mesh_peers { if Some(peer_id) != propagation_source && Some(peer_id) != message.source.as_ref() { recipient_peers.insert(*peer_id); @@ -2808,34 +2821,33 @@ where // Ignore connections from blacklisted peers. if self.blacklisted_peers.contains(peer_id) { debug!("Ignoring connection from blacklisted peer: {}", peer_id); - return; - } - - debug!("New peer connected: {}", peer_id); - // We need to send our subscriptions to the newly-connected node. - let mut subscriptions = vec![]; - for topic_hash in self.mesh.keys() { - subscriptions.push(GossipsubSubscription { - topic_hash: topic_hash.clone(), - action: GossipsubSubscriptionAction::Subscribe, - }); - } + } else { + debug!("New peer connected: {}", peer_id); + // We need to send our subscriptions to the newly-connected node. + let mut subscriptions = vec![]; + for topic_hash in self.mesh.keys() { + subscriptions.push(GossipsubSubscription { + topic_hash: topic_hash.clone(), + action: GossipsubSubscriptionAction::Subscribe, + }); + } - if !subscriptions.is_empty() { - // send our subscriptions to the peer - if self - .send_message( - *peer_id, - GossipsubRpc { - messages: Vec::new(), - subscriptions, - control_msgs: Vec::new(), - } - .into_protobuf(), - ) - .is_err() - { - error!("Failed to send subscriptions, message too large"); + if !subscriptions.is_empty() { + // send our subscriptions to the peer + if self + .send_message( + *peer_id, + GossipsubRpc { + messages: Vec::new(), + subscriptions, + control_msgs: Vec::new(), + } + .into_protobuf(), + ) + .is_err() + { + error!("Failed to send subscriptions, message too large"); + } } } @@ -2854,9 +2866,10 @@ where let topics = match self.peer_topics.get(peer_id) { Some(topics) => (topics), None => { - if !self.blacklisted_peers.contains(peer_id) { - debug!("Disconnected node, not in connected nodes"); - } + debug_assert!( + self.blacklisted_peers.contains(peer_id), + "Disconnected node not in connected list" + ); return; } }; @@ -2864,13 +2877,13 @@ where // remove peer from all mappings for topic in topics { // check the mesh for the topic - if let Some(mesh_peers) = self.mesh.get_mut(&topic) { + if let Some(mesh_peers) = self.mesh.get_mut(topic) { // check if the peer is in the mesh and remove it mesh_peers.remove(peer_id); } // remove from topic_peers - if let Some(peer_list) = self.topic_peers.get_mut(&topic) { + if let Some(peer_list) = self.topic_peers.get_mut(topic) { if !peer_list.remove(peer_id) { // debugging purposes warn!( @@ -2887,15 +2900,15 @@ where // remove from fanout self.fanout - .get_mut(&topic) + .get_mut(topic) .map(|peers| peers.remove(peer_id)); } - - //forget px and outbound status for this peer - self.px_peers.remove(peer_id); - self.outbound_peers.remove(peer_id); } + // Forget px and outbound status for this peer + self.px_peers.remove(peer_id); + self.outbound_peers.remove(peer_id); + // Remove peer from peer_topics and connected_peers // NOTE: It is possible the peer has already been removed from all mappings if it does not // support the protocol. @@ -2913,11 +2926,6 @@ where connection_id: &ConnectionId, endpoint: &ConnectedPoint, ) { - // Ignore connections from blacklisted peers. - if self.blacklisted_peers.contains(peer_id) { - return; - } - // Check if the peer is an outbound peer if let ConnectedPoint::Dialer { .. } = endpoint { // Diverging from the go implementation we only want to consider a peer as outbound peer @@ -2935,7 +2943,7 @@ where // Add the IP to the peer scoring system if let Some((peer_score, ..)) = &mut self.peer_score { if let Some(ip) = get_ip_addr(endpoint.get_remote_address()) { - peer_score.add_ip(&peer_id, ip); + peer_score.add_ip(peer_id, ip); } else { trace!( "Couldn't extract ip from endpoint of peer {} with endpoint {:?}", @@ -2965,6 +2973,7 @@ where peer_id: &PeerId, connection_id: &ConnectionId, endpoint: &ConnectedPoint, + _: ::Handler, ) { // Remove IP from peer scoring system if let Some((peer_score, ..)) = &mut self.peer_score { @@ -3032,7 +3041,7 @@ where ) } if let Some(ip) = get_ip_addr(endpoint_new.get_remote_address()) { - peer_score.add_ip(&peer, ip); + peer_score.add_ip(peer, ip); } else { trace!( "Couldn't extract ip from endpoint of peer {} with endpoint {:?}", @@ -3165,47 +3174,12 @@ where &mut self, cx: &mut Context<'_>, _: &mut impl PollParameters, - ) -> Poll< - NetworkBehaviourAction< - ::InEvent, - Self::OutEvent, - >, - > { + ) -> Poll> { if let Some(event) = self.events.pop_front() { - return Poll::Ready(match event { - NetworkBehaviourAction::NotifyHandler { - peer_id, - handler, - event: send_event, - } => { - // clone send event reference if others references are present - let event = Arc::try_unwrap(send_event).unwrap_or_else(|e| (*e).clone()); - NetworkBehaviourAction::NotifyHandler { - peer_id, - event, - handler, - } - } - NetworkBehaviourAction::GenerateEvent(e) => { - NetworkBehaviourAction::GenerateEvent(e) - } - NetworkBehaviourAction::DialAddress { address } => { - NetworkBehaviourAction::DialAddress { address } - } - NetworkBehaviourAction::DialPeer { peer_id, condition } => { - NetworkBehaviourAction::DialPeer { peer_id, condition } - } - NetworkBehaviourAction::ReportObservedAddr { address, score } => { - NetworkBehaviourAction::ReportObservedAddr { address, score } - } - NetworkBehaviourAction::CloseConnection { - peer_id, - connection, - } => NetworkBehaviourAction::CloseConnection { - peer_id, - connection, - }, - }); + return Poll::Ready(event.map_in(|e: Arc| { + // clone send event reference if others references are present + Arc::try_unwrap(e).unwrap_or_else(|e| (*e).clone()) + })); } // update scores @@ -3392,7 +3366,7 @@ impl fmt::Debug for Gossipsub) -> fmt::Result { f.debug_struct("Gossipsub") .field("config", &self.config) - .field("events", &self.events) + .field("events", &self.events.len()) .field("control_pool", &self.control_pool) .field("publish_config", &self.publish_config) .field("topic_peers", &self.topic_peers) diff --git a/protocols/gossipsub/src/behaviour/tests.rs b/protocols/gossipsub/src/behaviour/tests.rs index 81b2267fdbb..5794f2e0054 100644 --- a/protocols/gossipsub/src/behaviour/tests.rs +++ b/protocols/gossipsub/src/behaviour/tests.rs @@ -67,7 +67,7 @@ mod tests { F: TopicSubscriptionFilter + Clone + Default + Send + 'static, { pub fn create_network(self) -> (Gossipsub, Vec, Vec) { - let keypair = libp2p_core::identity::Keypair::generate_secp256k1(); + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); // create a gossipsub struct let mut gs: Gossipsub = Gossipsub::new_with_subscription_filter_and_transform( MessageAuthenticity::Signed(keypair), @@ -747,7 +747,7 @@ mod tests { // check that our subscriptions are sent to each of the peers // collect all the SendEvents - let send_events: Vec<&NetworkBehaviourAction, GossipsubEvent>> = gs + let send_events: Vec<_> = gs .events .iter() .filter(|e| match e { @@ -1336,13 +1336,14 @@ mod tests { //add peer as explicit peer gs.add_explicit_peer(&peer); - let dial_events: Vec<&NetworkBehaviourAction, GossipsubEvent>> = gs + let dial_events: Vec<_> = gs .events .iter() .filter(|e| match e { NetworkBehaviourAction::DialPeer { peer_id, condition: DialPeerCondition::Disconnected, + handler: _, } => peer_id == &peer, _ => false, }) @@ -1388,6 +1389,7 @@ mod tests { NetworkBehaviourAction::DialPeer { peer_id, condition: DialPeerCondition::Disconnected, + handler: _, } => peer_id == peer, _ => false, }) @@ -1406,6 +1408,7 @@ mod tests { NetworkBehaviourAction::DialPeer { peer_id, condition: DialPeerCondition::Disconnected, + handler: _, } => peer_id == peer, _ => false, }) @@ -1819,6 +1822,7 @@ mod tests { NetworkBehaviourAction::DialPeer { peer_id, condition: DialPeerCondition::Disconnected, + handler: _, } => Some(peer_id.clone()), _ => None, }) @@ -5228,4 +5232,36 @@ mod tests { //nobody got penalized assert!(gs1.peer_score.as_ref().unwrap().0.score(&p2) >= original_score); } + + #[test] + /// Test nodes that send grafts without subscriptions. + fn test_graft_without_subscribe() { + // The node should: + // - Create an empty vector in mesh[topic] + // - Send subscription request to all peers + // - run JOIN(topic) + + let topic = String::from("test_subscribe"); + let subscribe_topic = vec![topic.clone()]; + let subscribe_topic_hash = vec![Topic::new(topic.clone()).hash()]; + let (mut gs, peers, topic_hashes) = inject_nodes1() + .peer_no(1) + .topics(subscribe_topic) + .to_subscribe(false) + .create_network(); + + assert!( + gs.mesh.get(&topic_hashes[0]).is_some(), + "Subscribe should add a new entry to the mesh[topic] hashmap" + ); + + // The node sends a graft for the subscribe topic. + gs.handle_graft(&peers[0], subscribe_topic_hash); + + // The node disconnects + gs.inject_disconnected(&peers[0]); + + // We unsubscribe from the topic. + let _ = gs.unsubscribe(&Topic::new(topic)); + } } diff --git a/protocols/gossipsub/src/protocol.rs b/protocols/gossipsub/src/protocol.rs index 199d210452a..c0a3ec6a9bd 100644 --- a/protocols/gossipsub/src/protocol.rs +++ b/protocols/gossipsub/src/protocol.rs @@ -194,7 +194,7 @@ impl GossipsubCodec { } }; - let source = match PeerId::from_bytes(&from) { + let source = match PeerId::from_bytes(from) { Ok(v) => v, Err(_) => { debug!("Signature verification failed: Invalid Peer Id"); @@ -214,8 +214,8 @@ impl GossipsubCodec { // obtained from the inlined source peer_id. let public_key = match message .key - .as_ref() - .map(|key| PublicKey::from_protobuf_encoding(&key)) + .as_deref() + .map(PublicKey::from_protobuf_encoding) { Some(Ok(key)) => key, _ => match PublicKey::from_protobuf_encoding(&source.to_bytes()[2..]) { @@ -600,7 +600,7 @@ mod tests { fn arbitrary(g: &mut G) -> Self { let keypair = if g.gen() { // Small enough to be inlined. - Keypair::generate_secp256k1() + Keypair::generate_ed25519() } else { // Too large to be inlined. let mut rsa_key = hex::decode("308204bd020100300d06092a864886f70d0101010500048204a7308204a30201000282010100ef930f41a71288b643c1cbecbf5f72ab53992249e2b00835bf07390b6745419f3848cbcc5b030faa127bc88cdcda1c1d6f3ff699f0524c15ab9d2c9d8015f5d4bd09881069aad4e9f91b8b0d2964d215cdbbae83ddd31a7622a8228acee07079f6e501aea95508fa26c6122816ef7b00ac526d422bd12aed347c37fff6c1c307f3ba57bb28a7f28609e0bdcc839da4eedca39f5d2fa855ba4b0f9c763e9764937db929a1839054642175312a3de2d3405c9d27bdf6505ef471ce85c5e015eee85bf7874b3d512f715de58d0794fd8afe021c197fbd385bb88a930342fac8da31c27166e2edab00fa55dc1c3814448ba38363077f4e8fe2bdea1c081f85f1aa6f02030100010282010028ff427a1aac1a470e7b4879601a6656193d3857ea79f33db74df61e14730e92bf9ffd78200efb0c40937c3356cbe049cd32e5f15be5c96d5febcaa9bd3484d7fded76a25062d282a3856a1b3b7d2c525cdd8434beae147628e21adf241dd64198d5819f310d033743915ba40ea0b6acdbd0533022ad6daa1ff42de51885f9e8bab2306c6ef1181902d1cd7709006eba1ab0587842b724e0519f295c24f6d848907f772ae9a0953fc931f4af16a07df450fb8bfa94572562437056613647818c238a6ff3f606cffa0533e4b8755da33418dfbc64a85110b1a036623c947400a536bb8df65e5ebe46f2dfd0cfc86e7aeeddd7574c253e8fbf755562b3669525d902818100f9fff30c6677b78dd31ec7a634361438457e80be7a7faf390903067ea8355faa78a1204a82b6e99cb7d9058d23c1ecf6cfe4a900137a00cecc0113fd68c5931602980267ea9a95d182d48ba0a6b4d5dd32fdac685cb2e5d8b42509b2eb59c9579ea6a67ccc7547427e2bd1fb1f23b0ccb4dd6ba7d206c8dd93253d70a451701302818100f5530dfef678d73ce6a401ae47043af10a2e3f224c71ae933035ecd68ccbc4df52d72bc6ca2b17e8faf3e548b483a2506c0369ab80df3b137b54d53fac98f95547c2bc245b416e650ce617e0d29db36066f1335a9ba02ad3e0edf9dc3d58fd835835042663edebce81803972696c789012847cb1f854ab2ac0a1bd3867ac7fb502818029c53010d456105f2bf52a9a8482bca2224a5eac74bf3cc1a4d5d291fafcdffd15a6a6448cce8efdd661f6617ca5fc37c8c885cc3374e109ac6049bcbf72b37eabf44602a2da2d4a1237fd145c863e6d75059976de762d9d258c42b0984e2a2befa01c95217c3ee9c736ff209c355466ff99375194eff943bc402ea1d172a1ed02818027175bf493bbbfb8719c12b47d967bf9eac061c90a5b5711172e9095c38bb8cc493c063abffe4bea110b0a2f22ac9311b3947ba31b7ef6bfecf8209eebd6d86c316a2366bbafda7279b2b47d5bb24b6202254f249205dcad347b574433f6593733b806f84316276c1990a016ce1bbdbe5f650325acc7791aefe515ecc60063bd02818100b6a2077f4adcf15a17092d9c4a346d6022ac48f3861b73cf714f84c440a07419a7ce75a73b9cbff4597c53c128bf81e87b272d70428a272d99f90cd9b9ea1033298e108f919c6477400145a102df3fb5601ffc4588203cf710002517bfa24e6ad32f4d09c6b1a995fa28a3104131bedd9072f3b4fb4a5c2056232643d310453f").unwrap(); diff --git a/protocols/identify/CHANGELOG.md b/protocols/identify/CHANGELOG.md index 2b5a255d73e..b05cafa75e8 100644 --- a/protocols/identify/CHANGELOG.md +++ b/protocols/identify/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.31.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.30.0 [2021-07-12] diff --git a/protocols/identify/Cargo.toml b/protocols/identify/Cargo.toml index 8213edc8dc6..6f6c37d12e2 100644 --- a/protocols/identify/Cargo.toml +++ b/protocols/identify/Cargo.toml @@ -11,7 +11,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } libp2p-swarm = { version = "0.31.0", path = "../../swarm" } log = "0.4.1" prost = "0.8" diff --git a/protocols/identify/src/identify.rs b/protocols/identify/src/identify.rs index caa737f669c..d75dfb72054 100644 --- a/protocols/identify/src/identify.rs +++ b/protocols/identify/src/identify.rs @@ -27,8 +27,9 @@ use libp2p_core::{ ConnectedPoint, Multiaddr, PeerId, PublicKey, }; use libp2p_swarm::{ - AddressScore, DialPeerCondition, NegotiatedSubstream, NetworkBehaviour, NetworkBehaviourAction, - NotifyHandler, PollParameters, ProtocolsHandler, ProtocolsHandlerUpgrErr, + AddressScore, DialError, DialPeerCondition, IntoProtocolsHandler, NegotiatedSubstream, + NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, ProtocolsHandler, + ProtocolsHandlerUpgrErr, }; use std::{ collections::{HashMap, HashSet, VecDeque}, @@ -52,7 +53,7 @@ pub struct Identify { /// Pending replies to send. pending_replies: VecDeque, /// Pending events to be emitted when polled. - events: VecDeque>, + events: VecDeque>, /// Peers to which an active push with current information about /// the local peer should be sent. pending_push: HashSet, @@ -173,9 +174,11 @@ impl Identify { for p in peers { if self.pending_push.insert(p) { if !self.connected.contains_key(&p) { + let handler = self.new_handler(); self.events.push_back(NetworkBehaviourAction::DialPeer { peer_id: p, condition: DialPeerCondition::Disconnected, + handler, }); } } @@ -213,13 +216,14 @@ impl NetworkBehaviour for Identify { peer_id: &PeerId, conn: &ConnectionId, _: &ConnectedPoint, + _: ::Handler, ) { if let Some(addrs) = self.connected.get_mut(peer_id) { addrs.remove(conn); } } - fn inject_dial_failure(&mut self, peer_id: &PeerId) { + fn inject_dial_failure(&mut self, peer_id: &PeerId, _: Self::ProtocolsHandler, _: DialError) { if !self.connected.contains_key(peer_id) { self.pending_push.remove(peer_id); } @@ -292,12 +296,7 @@ impl NetworkBehaviour for Identify { &mut self, cx: &mut Context<'_>, params: &mut impl PollParameters, - ) -> Poll< - NetworkBehaviourAction< - ::InEvent, - Self::OutEvent, - >, - > { + ) -> Poll> { if let Some(event) = self.events.pop_front() { return Poll::Ready(event); } diff --git a/protocols/identify/src/protocol.rs b/protocols/identify/src/protocol.rs index 9604e660e9f..2df49bf54f3 100644 --- a/protocols/identify/src/protocol.rs +++ b/protocols/identify/src/protocol.rs @@ -258,7 +258,7 @@ fn parse_proto_msg(msg: impl AsRef<[u8]>) -> Result { #[cfg(test)] mod tests { use super::*; - use futures::{channel::oneshot, prelude::*}; + use futures::channel::oneshot; use libp2p_core::{ identity, upgrade::{self, apply_inbound, apply_outbound}, diff --git a/protocols/kad/CHANGELOG.md b/protocols/kad/CHANGELOG.md index 0cf78776687..e344f9c2546 100644 --- a/protocols/kad/CHANGELOG.md +++ b/protocols/kad/CHANGELOG.md @@ -1,7 +1,15 @@ # 0.32.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. +- Introduce `KademliaStoreInserts` option, which allows to filter records (see + [PR 2163]). + +[PR 2163]: https://github.com/libp2p/rust-libp2p/pull/2163 + # 0.31.0 [2021-07-12] - Update dependencies. diff --git a/protocols/kad/Cargo.toml b/protocols/kad/Cargo.toml index 2dd5e7287c0..20fe6fe2c75 100644 --- a/protocols/kad/Cargo.toml +++ b/protocols/kad/Cargo.toml @@ -17,7 +17,7 @@ fnv = "1.0" asynchronous-codec = "0.6" futures = "0.3.1" log = "0.4" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } libp2p-swarm = { version = "0.31.0", path = "../../swarm" } prost = "0.8" rand = "0.7.2" diff --git a/protocols/kad/src/behaviour.rs b/protocols/kad/src/behaviour.rs index 4d7e3207334..74b5616ff91 100644 --- a/protocols/kad/src/behaviour.rs +++ b/protocols/kad/src/behaviour.rs @@ -43,7 +43,8 @@ use libp2p_core::{ ConnectedPoint, Multiaddr, PeerId, }; use libp2p_swarm::{ - DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, + DialError, DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, + PollParameters, }; use log::{debug, info, warn}; use smallvec::SmallVec; @@ -69,6 +70,9 @@ pub struct Kademlia { /// Configuration of the wire protocol. protocol_config: KademliaProtocolConfig, + /// Configuration of [`RecordStore`] filtering. + record_filtering: KademliaStoreInserts, + /// The currently active (i.e. in-progress) queries. queries: QueryPool, @@ -95,7 +99,7 @@ pub struct Kademlia { connection_idle_timeout: Duration, /// Queued events to return when the behaviour is being polled. - queued_events: VecDeque, KademliaEvent>>, + queued_events: VecDeque>>, /// The currently known addresses of the local node. local_addrs: HashSet, @@ -131,6 +135,29 @@ pub enum KademliaBucketInserts { Manual, } +/// The configurable filtering strategies for the acceptance of +/// incoming records. +/// +/// This can be used for e.g. signature verification or validating +/// the accompanying [`Key`]. +/// +/// [`Key`]: crate::record::Key +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum KademliaStoreInserts { + /// Whenever a (provider) record is received, + /// the record is forwarded immediately to the [`RecordStore`]. + Unfiltered, + /// Whenever a (provider) record is received, an event is emitted. + /// Provider records generate a [`KademliaEvent::InboundAddProviderRequest`], + /// normal records generate a [`KademliaEvent::InboundPutRecordRequest`]. + /// + /// When deemed valid, a (provider) record needs to be explicitly stored in + /// the [`RecordStore`] via [`RecordStore::put`] or [`RecordStore::add_provider`], + /// whichever is applicable. A mutable reference to the [`RecordStore`] can + /// be retrieved via [`Kademlia::store_mut`]. + FilterBoth, +} + /// The configuration for the `Kademlia` behaviour. /// /// The configuration is consumed by [`Kademlia::new`]. @@ -142,6 +169,7 @@ pub struct KademliaConfig { record_ttl: Option, record_replication_interval: Option, record_publication_interval: Option, + record_filtering: KademliaStoreInserts, provider_record_ttl: Option, provider_publication_interval: Option, connection_idle_timeout: Duration, @@ -175,6 +203,7 @@ impl Default for KademliaConfig { record_ttl: Some(Duration::from_secs(36 * 60 * 60)), record_replication_interval: Some(Duration::from_secs(60 * 60)), record_publication_interval: Some(Duration::from_secs(24 * 60 * 60)), + record_filtering: KademliaStoreInserts::Unfiltered, provider_publication_interval: Some(Duration::from_secs(12 * 60 * 60)), provider_record_ttl: Some(Duration::from_secs(24 * 60 * 60)), connection_idle_timeout: Duration::from_secs(10), @@ -259,6 +288,15 @@ impl KademliaConfig { self } + /// Sets whether or not records should be filtered before being stored. + /// + /// See [`KademliaStoreInserts`] for the different values. + /// Defaults to [`KademliaStoreInserts::Unfiltered`]. + pub fn set_record_filtering(&mut self, filtering: KademliaStoreInserts) -> &mut Self { + self.record_filtering = filtering; + self + } + /// Sets the (re-)replication interval for stored records. /// /// Periodic replication of stored records ensures that the records @@ -357,6 +395,7 @@ impl KademliaConfig { impl Kademlia where for<'a> TStore: RecordStore<'a>, + TStore: Send + 'static, { /// Creates a new `Kademlia` network behaviour with a default configuration. pub fn new(id: PeerId, store: TStore) -> Self { @@ -393,6 +432,7 @@ where kbuckets: KBucketsTable::new(local_key, config.kbucket_pending_timeout), kbucket_inserts: config.kbucket_inserts, protocol_config: config.protocol_config, + record_filtering: config.record_filtering, queued_events: VecDeque::with_capacity(config.query_config.replication_factor.get()), queries: QueryPool::new(config.query_config), connected_peers: Default::default(), @@ -523,10 +563,12 @@ where RoutingUpdate::Failed } kbucket::InsertResult::Pending { disconnected } => { + let handler = self.new_handler(); self.queued_events .push_back(NetworkBehaviourAction::DialPeer { peer_id: disconnected.into_preimage(), condition: DialPeerCondition::Disconnected, + handler, }); RoutingUpdate::Pending } @@ -1102,10 +1144,12 @@ where // // Only try dialing peer if not currently connected. if !self.connected_peers.contains(disconnected.preimage()) { + let handler = self.new_handler(); self.queued_events .push_back(NetworkBehaviourAction::DialPeer { peer_id: disconnected.into_preimage(), condition: DialPeerCondition::Disconnected, + handler, }) } } @@ -1572,22 +1616,33 @@ where // The record is cloned because of the weird libp2p protocol // requirement to send back the value in the response, although this // is a waste of resources. - match self.store.put(record.clone()) { - Ok(()) => debug!( - "Record stored: {:?}; {} bytes", - record.key, - record.value.len() - ), - Err(e) => { - info!("Record not stored: {:?}", e); + match self.record_filtering { + KademliaStoreInserts::Unfiltered => match self.store.put(record.clone()) { + Ok(()) => debug!( + "Record stored: {:?}; {} bytes", + record.key, + record.value.len() + ), + Err(e) => { + info!("Record not stored: {:?}", e); + self.queued_events + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: source, + handler: NotifyHandler::One(connection), + event: KademliaHandlerIn::Reset(request_id), + }); + return; + } + }, + KademliaStoreInserts::FilterBoth => { self.queued_events - .push_back(NetworkBehaviourAction::NotifyHandler { - peer_id: source, - handler: NotifyHandler::One(connection), - event: KademliaHandlerIn::Reset(request_id), - }); - - return; + .push_back(NetworkBehaviourAction::GenerateEvent( + KademliaEvent::InboundPutRecordRequest { + source, + connection, + record: record.clone(), + }, + )); } } } @@ -1620,8 +1675,18 @@ where expires: self.provider_record_ttl.map(|ttl| Instant::now() + ttl), addresses: provider.multiaddrs, }; - if let Err(e) = self.store.add_provider(record) { - info!("Provider record not stored: {:?}", e); + match self.record_filtering { + KademliaStoreInserts::Unfiltered => { + if let Err(e) = self.store.add_provider(record) { + info!("Provider record not stored: {:?}", e); + } + } + KademliaStoreInserts::FilterBoth => { + self.queued_events + .push_back(NetworkBehaviourAction::GenerateEvent( + KademliaEvent::InboundAddProviderRequest { record }, + )); + } } } } @@ -1800,9 +1865,32 @@ where } } - fn inject_dial_failure(&mut self, peer_id: &PeerId) { - for query in self.queries.iter_mut() { - query.on_failure(peer_id); + fn inject_dial_failure( + &mut self, + peer_id: &PeerId, + _: Self::ProtocolsHandler, + error: DialError, + ) { + match error { + DialError::Banned + | DialError::ConnectionLimit(_) + | DialError::InvalidAddress(_) + | DialError::UnreachableAddr(_) + | DialError::LocalPeerId + | DialError::NoAddresses => { + for query in self.queries.iter_mut() { + query.on_failure(peer_id); + } + } + DialError::DialPeerConditionFalse( + DialPeerCondition::Disconnected | DialPeerCondition::NotDialing, + ) => { + // We might (still) be connected, or about to be connected, thus do not report the + // failure to the queries. + } + DialError::DialPeerConditionFalse(DialPeerCondition::Always) => { + unreachable!("DialPeerCondition::Always can not trigger DialPeerConditionFalse."); + } } } @@ -2097,7 +2185,7 @@ where &mut self, cx: &mut Context<'_>, parameters: &mut impl PollParameters, - ) -> Poll, Self::OutEvent>> { + ) -> Poll> { let now = Instant::now(); // Calculate the available capacity for queries triggered by background jobs. @@ -2195,10 +2283,12 @@ where }); } else if &peer_id != self.kbuckets.local_key().preimage() { query.inner.pending_rpcs.push((peer_id, event)); + let handler = self.new_handler(); self.queued_events .push_back(NetworkBehaviourAction::DialPeer { peer_id, condition: DialPeerCondition::Disconnected, + handler, }); } } @@ -2257,6 +2347,20 @@ pub struct PeerRecord { /// See [`NetworkBehaviour::poll`]. #[derive(Debug)] pub enum KademliaEvent { + /// A peer sent a [`KademliaHandlerIn::PutRecord`] request and filtering is enabled. + /// + /// See [`KademliaStoreInserts`] and [`KademliaConfig::set_record_filtering`]. + InboundPutRecordRequest { + source: PeerId, + connection: ConnectionId, + record: Record, + }, + + /// A peer sent a [`KademliaHandlerIn::AddProvider`] request and filtering [`KademliaStoreInserts::FilterBoth`] is enabled. + /// + /// See [`KademliaStoreInserts`] and [`KademliaConfig::set_record_filtering`] for details.. + InboundAddProviderRequest { record: ProviderRecord }, + /// An inbound request has been received and handled. // // Note on the difference between 'request' and 'query': A request is a diff --git a/protocols/kad/src/behaviour/test.rs b/protocols/kad/src/behaviour/test.rs index 110c1e2221e..fc855b049ea 100644 --- a/protocols/kad/src/behaviour/test.rs +++ b/protocols/kad/src/behaviour/test.rs @@ -488,7 +488,7 @@ fn get_record_not_found() { /// is equal to the configured replication factor. #[test] fn put_record() { - fn prop(records: Vec, seed: Seed) { + fn prop(records: Vec, seed: Seed, filter_records: bool, drop_records: bool) { let mut rng = StdRng::from_seed(seed.0); let replication_factor = NonZeroUsize::new(rng.gen_range(1, (K_VALUE.get() / 2) + 1)).unwrap(); @@ -501,6 +501,10 @@ fn put_record() { config.disjoint_query_paths(true); } + if filter_records { + config.set_record_filtering(KademliaStoreInserts::FilterBoth); + } + let mut swarms = { let mut fully_connected_swarms = build_fully_connected_nodes_with_config(num_total - 1, config.clone()); @@ -596,6 +600,22 @@ fn put_record() { } } } + Poll::Ready(Some(SwarmEvent::Behaviour( + KademliaEvent::InboundPutRecordRequest { record, .. }, + ))) => { + assert_ne!( + swarm.behaviour().record_filtering, + KademliaStoreInserts::Unfiltered + ); + if !drop_records { + // Accept the record + swarm + .behaviour_mut() + .store_mut() + .put(record) + .expect("record is stored"); + } + } // Ignore any other event. Poll::Ready(Some(_)) => (), e @ Poll::Ready(_) => panic!("Unexpected return value: {:?}", e), @@ -650,21 +670,29 @@ fn put_record() { }) .collect::>(); - assert_eq!(actual.len(), replication_factor.get()); - - let actual_not_expected = actual.difference(&expected).collect::>(); - assert!( - actual_not_expected.is_empty(), - "Did not expect records to be stored on nodes {:?}.", - actual_not_expected, - ); - - let expected_not_actual = expected.difference(&actual).collect::>(); - assert!( - expected_not_actual.is_empty(), - "Expected record to be stored on nodes {:?}.", - expected_not_actual, - ); + if swarms[0].behaviour().record_filtering != KademliaStoreInserts::Unfiltered + && drop_records + { + assert_eq!(actual.len(), 0); + } else { + assert_eq!(actual.len(), replication_factor.get()); + + let actual_not_expected = + actual.difference(&expected).collect::>(); + assert!( + actual_not_expected.is_empty(), + "Did not expect records to be stored on nodes {:?}.", + actual_not_expected, + ); + + let expected_not_actual = + expected.difference(&actual).collect::>(); + assert!( + expected_not_actual.is_empty(), + "Expected record to be stored on nodes {:?}.", + expected_not_actual, + ); + } } if republished { @@ -692,7 +720,9 @@ fn put_record() { })) } - QuickCheck::new().tests(3).quickcheck(prop as fn(_, _) -> _) + QuickCheck::new() + .tests(4) + .quickcheck(prop as fn(_, _, _, _) -> _) } #[test] diff --git a/protocols/kad/src/handler.rs b/protocols/kad/src/handler.rs index 3c955bb428a..b92c89bfee5 100644 --- a/protocols/kad/src/handler.rs +++ b/protocols/kad/src/handler.rs @@ -803,16 +803,12 @@ fn advance_substream( true, ), Err(error) => { - let event = if let Some(user_data) = user_data { - Some(ProtocolsHandlerEvent::Custom( - KademliaHandlerEvent::QueryError { - error: KademliaHandlerQueryErr::Io(error), - user_data, - }, - )) - } else { - None - }; + let event = user_data.map(|user_data| { + ProtocolsHandlerEvent::Custom(KademliaHandlerEvent::QueryError { + error: KademliaHandlerQueryErr::Io(error), + user_data, + }) + }); (None, event, false) } @@ -823,16 +819,12 @@ fn advance_substream( false, ), Poll::Ready(Err(error)) => { - let event = if let Some(user_data) = user_data { - Some(ProtocolsHandlerEvent::Custom( - KademliaHandlerEvent::QueryError { - error: KademliaHandlerQueryErr::Io(error), - user_data, - }, - )) - } else { - None - }; + let event = user_data.map(|user_data| { + ProtocolsHandlerEvent::Custom(KademliaHandlerEvent::QueryError { + error: KademliaHandlerQueryErr::Io(error), + user_data, + }) + }); (None, event, false) } @@ -857,16 +849,12 @@ fn advance_substream( false, ), Poll::Ready(Err(error)) => { - let event = if let Some(user_data) = user_data { - Some(ProtocolsHandlerEvent::Custom( - KademliaHandlerEvent::QueryError { - error: KademliaHandlerQueryErr::Io(error), - user_data, - }, - )) - } else { - None - }; + let event = user_data.map(|user_data| { + ProtocolsHandlerEvent::Custom(KademliaHandlerEvent::QueryError { + error: KademliaHandlerQueryErr::Io(error), + user_data, + }) + }); (None, event, false) } diff --git a/protocols/kad/src/kbucket/entry.rs b/protocols/kad/src/kbucket/entry.rs index 3447146007b..7ccf2017e99 100644 --- a/protocols/kad/src/kbucket/entry.rs +++ b/protocols/kad/src/kbucket/entry.rs @@ -201,7 +201,7 @@ where let (node, status, _pos) = self .0 .bucket - .remove(&self.0.key) + .remove(self.0.key) .expect("We can only build a PresentEntry if the entry is in the bucket; QED"); EntryView { node, status } } diff --git a/protocols/kad/src/lib.rs b/protocols/kad/src/lib.rs index 0fbeb61587d..3b8a0d21e91 100644 --- a/protocols/kad/src/lib.rs +++ b/protocols/kad/src/lib.rs @@ -48,7 +48,7 @@ pub use behaviour::{ QueryStats, }; pub use behaviour::{ - Kademlia, KademliaBucketInserts, KademliaCaching, KademliaConfig, KademliaEvent, Quorum, + Kademlia, KademliaCaching, KademliaConfig, KademliaEvent, KademliaStoreInserts, Quorum, }; pub use protocol::KadConnectionType; pub use query::QueryId; diff --git a/protocols/kad/src/query/peers/fixed.rs b/protocols/kad/src/query/peers/fixed.rs index e4be4094eb1..022a3de6f9f 100644 --- a/protocols/kad/src/query/peers/fixed.rs +++ b/protocols/kad/src/query/peers/fixed.rs @@ -58,6 +58,7 @@ enum PeerState { } impl FixedPeersIter { + #[allow(clippy::needless_collect)] pub fn new(peers: I, parallelism: NonZeroUsize) -> Self where I: IntoIterator, diff --git a/protocols/mdns/CHANGELOG.md b/protocols/mdns/CHANGELOG.md index 899222c62ab..3b633de5030 100644 --- a/protocols/mdns/CHANGELOG.md +++ b/protocols/mdns/CHANGELOG.md @@ -1,12 +1,18 @@ # 0.32.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. - Add support for IPv6. To enable set the multicast address in `MdnsConfig` to `IPV6_MDNS_MULTICAST_ADDRESS`. See [PR 2161] for details. - + +- Prevent timers from firing at the same time. See [PR 2212] for details. + [PR 2161]: https://github.com/libp2p/rust-libp2p/pull/2161/ +[PR 2212]: https://github.com/libp2p/rust-libp2p/pull/2212/ # 0.31.0 [2021-07-12] diff --git a/protocols/mdns/Cargo.toml b/protocols/mdns/Cargo.toml index 64cf9263f8a..43370a9bffd 100644 --- a/protocols/mdns/Cargo.toml +++ b/protocols/mdns/Cargo.toml @@ -16,7 +16,7 @@ dns-parser = "0.8.0" futures = "0.3.13" if-watch = "0.2.0" lazy_static = "1.4.0" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } libp2p-swarm = { version = "0.31.0", path = "../../swarm" } log = "0.4.14" rand = "0.8.3" diff --git a/protocols/mdns/src/behaviour.rs b/protocols/mdns/src/behaviour.rs index 2a170e2d839..156c805a4d7 100644 --- a/protocols/mdns/src/behaviour.rs +++ b/protocols/mdns/src/behaviour.rs @@ -153,6 +153,13 @@ impl Mdns { Async::new(socket)? }; let if_watch = if_watch::IfWatcher::new().await?; + // randomize timer to prevent all converging and firing at the same time. + let query_interval = { + use rand::Rng; + let mut rng = rand::thread_rng(); + let jitter = rng.gen_range(0..100); + config.query_interval + Duration::from_millis(jitter) + }; Ok(Self { recv_socket, send_socket, @@ -162,9 +169,9 @@ impl Mdns { discovered_nodes: SmallVec::new(), closest_expiration: None, events: Default::default(), - query_interval: config.query_interval, + query_interval, ttl: config.ttl, - timeout: Timer::interval(config.query_interval), + timeout: Timer::interval(query_interval), multicast_addr: config.multicast_addr, }) } @@ -179,10 +186,19 @@ impl Mdns { self.discovered_nodes.iter().map(|(p, _, _)| p) } + fn reset_timer(&mut self) { + self.timeout.set_interval(self.query_interval); + } + + fn fire_timer(&mut self) { + self.timeout + .set_interval_at(Instant::now(), self.query_interval); + } + fn inject_mdns_packet(&mut self, packet: MdnsPacket, params: &impl PollParameters) { match packet { MdnsPacket::Query(query) => { - self.timeout.set_interval(self.query_interval); + self.reset_timer(); log::trace!("sending response"); for packet in build_query_response( query.query_id(), @@ -210,7 +226,7 @@ impl Mdns { let mut addrs: Vec = Vec::new(); for addr in peer.addresses() { - if let Some(new_addr) = address_translation(&addr, &observed) { + if let Some(new_addr) = address_translation(addr, &observed) { addrs.push(new_addr.clone()) } addrs.push(addr.clone()) @@ -279,20 +295,14 @@ impl NetworkBehaviour for Mdns { } fn inject_new_listen_addr(&mut self, _id: ListenerId, _addr: &Multiaddr) { - self.timeout - .set_interval_at(Instant::now(), self.query_interval); + self.fire_timer(); } fn poll( &mut self, cx: &mut Context<'_>, params: &mut impl PollParameters, - ) -> Poll< - NetworkBehaviourAction< - ::InEvent, - Self::OutEvent, - >, - > { + ) -> Poll> { while let Poll::Ready(event) = Pin::new(&mut self.if_watch).poll(cx) { let socket = self.recv_socket.get_ref(); match event { @@ -307,8 +317,7 @@ impl NetworkBehaviour for Mdns { if let Err(err) = socket.join_multicast_v4(&multicast, &addr) { log::error!("join multicast failed: {}", err); } else { - self.timeout - .set_interval_at(Instant::now(), self.query_interval); + self.fire_timer(); } } } @@ -318,8 +327,7 @@ impl NetworkBehaviour for Mdns { if let Err(err) = socket.join_multicast_v6(&multicast, 0) { log::error!("join multicast failed: {}", err); } else { - self.timeout - .set_interval_at(Instant::now(), self.query_interval); + self.fire_timer(); } } } diff --git a/protocols/mdns/src/dns.rs b/protocols/mdns/src/dns.rs index 92fd980c89d..9bd1181dd74 100644 --- a/protocols/mdns/src/dns.rs +++ b/protocols/mdns/src/dns.rs @@ -228,11 +228,11 @@ fn query_response_packet(id: u16, peer_id: &[u8], records: &[Vec], ttl: u32) // Peer Id. append_u16(&mut out, peer_id.len() as u16); - out.extend_from_slice(&peer_id); + out.extend_from_slice(peer_id); // The TXT records. for record in records { - out.extend_from_slice(&record); + out.extend_from_slice(record); } out diff --git a/protocols/ping/CHANGELOG.md b/protocols/ping/CHANGELOG.md index 9d7e86a7f98..e4de6070fc3 100644 --- a/protocols/ping/CHANGELOG.md +++ b/protocols/ping/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.31.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. - Don't close connection if ping protocol is unsupported by remote. @@ -15,6 +18,14 @@ [PR 2149]: https://github.com/libp2p/rust-libp2p/pull/2149/ +- Rename types as per [discussion 2174]. + `Ping` has been renamed to `Behaviour`. + The `Ping` prefix has been removed from various types like `PingEvent`. + Users should prefer importing the ping protocol as a module (`use libp2p::ping;`), + and refer to its types via `ping::`. For example: `ping::Behaviour` or `ping::Event`. + + [discussion 2174]: https://github.com/libp2p/rust-libp2p/discussions/2174 + # 0.30.0 [2021-07-12] - Update dependencies. diff --git a/protocols/ping/Cargo.toml b/protocols/ping/Cargo.toml index 7d3c093ee98..24924c48217 100644 --- a/protocols/ping/Cargo.toml +++ b/protocols/ping/Cargo.toml @@ -11,7 +11,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } libp2p-swarm = { version = "0.31.0", path = "../../swarm" } log = "0.4.1" rand = "0.7.2" diff --git a/protocols/ping/src/handler.rs b/protocols/ping/src/handler.rs index 1c4233e2b22..435a5048485 100644 --- a/protocols/ping/src/handler.rs +++ b/protocols/ping/src/handler.rs @@ -39,7 +39,7 @@ use wasm_timer::Delay; /// The configuration for outbound pings. #[derive(Clone, Debug)] -pub struct PingConfig { +pub struct Config { /// The timeout of an outbound ping. timeout: Duration, /// The duration between the last successful outbound or inbound ping @@ -54,13 +54,13 @@ pub struct PingConfig { keep_alive: bool, } -impl PingConfig { +impl Config { /// Creates a new `PingConfig` with the following default settings: /// - /// * [`PingConfig::with_interval`] 15s - /// * [`PingConfig::with_timeout`] 20s - /// * [`PingConfig::with_max_failures`] 1 - /// * [`PingConfig::with_keep_alive`] false + /// * [`Config::with_interval`] 15s + /// * [`Config::with_timeout`] 20s + /// * [`Config::with_max_failures`] 1 + /// * [`Config::with_keep_alive`] false /// /// These settings have the following effect: /// @@ -116,12 +116,9 @@ impl PingConfig { } } -/// The result of an inbound or outbound ping. -pub type PingResult = Result; - /// The successful result of processing an inbound or outbound ping. #[derive(Debug)] -pub enum PingSuccess { +pub enum Success { /// Received a ping and sent back a pong. Pong, /// Sent a ping and received back a pong. @@ -132,7 +129,7 @@ pub enum PingSuccess { /// An outbound ping failure. #[derive(Debug)] -pub enum PingFailure { +pub enum Failure { /// The ping timed out, i.e. no response was received within the /// configured ping timeout. Timeout, @@ -144,22 +141,22 @@ pub enum PingFailure { }, } -impl fmt::Display for PingFailure { +impl fmt::Display for Failure { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - PingFailure::Timeout => f.write_str("Ping timeout"), - PingFailure::Other { error } => write!(f, "Ping error: {}", error), - PingFailure::Unsupported => write!(f, "Ping protocol not supported"), + Failure::Timeout => f.write_str("Ping timeout"), + Failure::Other { error } => write!(f, "Ping error: {}", error), + Failure::Unsupported => write!(f, "Ping protocol not supported"), } } } -impl Error for PingFailure { +impl Error for Failure { fn source(&self) -> Option<&(dyn Error + 'static)> { match self { - PingFailure::Timeout => None, - PingFailure::Other { error } => Some(&**error), - PingFailure::Unsupported => None, + Failure::Timeout => None, + Failure::Other { error } => Some(&**error), + Failure::Unsupported => None, } } } @@ -168,14 +165,14 @@ impl Error for PingFailure { /// and answering ping queries. /// /// If the remote doesn't respond, produces an error that closes the connection. -pub struct PingHandler { +pub struct Handler { /// Configuration options. - config: PingConfig, + config: Config, /// The timer used for the delay to the next ping as well as /// the ping timeout. timer: Delay, /// Outbound ping failures that are pending to be processed by `poll()`. - pending_errors: VecDeque, + pending_errors: VecDeque, /// The number of consecutive ping failures that occurred. /// /// Each successful ping resets this counter to 0. @@ -203,10 +200,10 @@ enum State { Active, } -impl PingHandler { +impl Handler { /// Builds a new `PingHandler` with the given configuration. - pub fn new(config: PingConfig) -> Self { - PingHandler { + pub fn new(config: Config) -> Self { + Handler { config, timer: Delay::new(Duration::new(0, 0)), pending_errors: VecDeque::with_capacity(2), @@ -218,10 +215,10 @@ impl PingHandler { } } -impl ProtocolsHandler for PingHandler { +impl ProtocolsHandler for Handler { type InEvent = Void; - type OutEvent = PingResult; - type Error = PingFailure; + type OutEvent = crate::Result; + type Error = Failure; type InboundProtocol = protocol::Ping; type OutboundProtocol = protocol::Ping; type OutboundOpenInfo = (); @@ -253,8 +250,8 @@ impl ProtocolsHandler for PingHandler { return; } // Note: This timeout only covers protocol negotiation. - ProtocolsHandlerUpgrErr::Timeout => PingFailure::Timeout, - e => PingFailure::Other { error: Box::new(e) }, + ProtocolsHandlerUpgrErr::Timeout => Failure::Timeout, + e => Failure::Other { error: Box::new(e) }, }; self.pending_errors.push_front(error); @@ -271,14 +268,14 @@ impl ProtocolsHandler for PingHandler { fn poll( &mut self, cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll> { match self.state { State::Inactive { reported: true } => { return Poll::Pending; // nothing to do on this connection } State::Inactive { reported: false } => { self.state = State::Inactive { reported: true }; - return Poll::Ready(ProtocolsHandlerEvent::Custom(Err(PingFailure::Unsupported))); + return Poll::Ready(ProtocolsHandlerEvent::Custom(Err(Failure::Unsupported))); } State::Active => {} } @@ -294,7 +291,7 @@ impl ProtocolsHandler for PingHandler { Poll::Ready(Ok(stream)) => { // A ping from a remote peer has been answered, wait for the next. self.inbound = Some(protocol::recv_ping(stream).boxed()); - return Poll::Ready(ProtocolsHandlerEvent::Custom(Ok(PingSuccess::Pong))); + return Poll::Ready(ProtocolsHandlerEvent::Custom(Ok(Success::Pong))); } } } @@ -328,7 +325,7 @@ impl ProtocolsHandler for PingHandler { Some(PingState::Ping(mut ping)) => match ping.poll_unpin(cx) { Poll::Pending => { if self.timer.poll_unpin(cx).is_ready() { - self.pending_errors.push_front(PingFailure::Timeout); + self.pending_errors.push_front(Failure::Timeout); } else { self.outbound = Some(PingState::Ping(ping)); break; @@ -338,13 +335,13 @@ impl ProtocolsHandler for PingHandler { self.failures = 0; self.timer.reset(self.config.interval); self.outbound = Some(PingState::Idle(stream)); - return Poll::Ready(ProtocolsHandlerEvent::Custom(Ok(PingSuccess::Ping { + return Poll::Ready(ProtocolsHandlerEvent::Custom(Ok(Success::Ping { rtt, }))); } Poll::Ready(Err(e)) => { self.pending_errors - .push_front(PingFailure::Other { error: Box::new(e) }); + .push_front(Failure::Other { error: Box::new(e) }); } }, Some(PingState::Idle(stream)) => match self.timer.poll_unpin(cx) { @@ -357,7 +354,7 @@ impl ProtocolsHandler for PingHandler { self.outbound = Some(PingState::Ping(protocol::send_ping(stream).boxed())); } Poll::Ready(Err(e)) => { - return Poll::Ready(ProtocolsHandlerEvent::Close(PingFailure::Other { + return Poll::Ready(ProtocolsHandlerEvent::Close(Failure::Other { error: Box::new(e), })) } diff --git a/protocols/ping/src/lib.rs b/protocols/ping/src/lib.rs index d4e3828f430..e0c6bdf1103 100644 --- a/protocols/ping/src/lib.rs +++ b/protocols/ping/src/lib.rs @@ -40,70 +40,85 @@ //! [`Swarm`]: libp2p_swarm::Swarm //! [`Transport`]: libp2p_core::Transport -pub mod handler; -pub mod protocol; - -use handler::PingHandler; -pub use handler::{PingConfig, PingFailure, PingResult, PingSuccess}; +mod handler; +mod protocol; +use handler::Handler; +pub use handler::{Config, Failure, Success}; use libp2p_core::{connection::ConnectionId, PeerId}; use libp2p_swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters}; -use std::{collections::VecDeque, task::Context, task::Poll}; -use void::Void; +use std::{ + collections::VecDeque, + task::{Context, Poll}, +}; + +#[deprecated( + since = "0.30.0", + note = "Use re-exports that omit `Ping` prefix, i.e. `libp2p::ping::Config` etc" +)] +pub use self::{ + Config as PingConfig, Event as PingEvent, Failure as PingFailure, Result as PingResult, + Success as PingSuccess, +}; +#[deprecated(since = "0.30.0", note = "Use libp2p::ping::Behaviour instead.")] +pub use Behaviour as Ping; + +/// The result of an inbound or outbound ping. +pub type Result = std::result::Result; -/// `Ping` is a [`NetworkBehaviour`] that responds to inbound pings and +/// A [`NetworkBehaviour`] that responds to inbound pings and /// periodically sends outbound pings on every established connection. /// /// See the crate root documentation for more information. -pub struct Ping { +pub struct Behaviour { /// Configuration for outbound pings. - config: PingConfig, + config: Config, /// Queue of events to yield to the swarm. - events: VecDeque, + events: VecDeque, } /// Event generated by the `Ping` network behaviour. #[derive(Debug)] -pub struct PingEvent { +pub struct Event { /// The peer ID of the remote. pub peer: PeerId, /// The result of an inbound or outbound ping. - pub result: PingResult, + pub result: Result, } -impl Ping { +impl Behaviour { /// Creates a new `Ping` network behaviour with the given configuration. - pub fn new(config: PingConfig) -> Self { - Ping { + pub fn new(config: Config) -> Self { + Self { config, events: VecDeque::new(), } } } -impl Default for Ping { +impl Default for Behaviour { fn default() -> Self { - Ping::new(PingConfig::new()) + Self::new(Config::new()) } } -impl NetworkBehaviour for Ping { - type ProtocolsHandler = PingHandler; - type OutEvent = PingEvent; +impl NetworkBehaviour for Behaviour { + type ProtocolsHandler = Handler; + type OutEvent = Event; fn new_handler(&mut self) -> Self::ProtocolsHandler { - PingHandler::new(self.config.clone()) + Handler::new(self.config.clone()) } - fn inject_event(&mut self, peer: PeerId, _: ConnectionId, result: PingResult) { - self.events.push_front(PingEvent { peer, result }) + fn inject_event(&mut self, peer: PeerId, _: ConnectionId, result: Result) { + self.events.push_front(Event { peer, result }) } fn poll( &mut self, _: &mut Context<'_>, _: &mut impl PollParameters, - ) -> Poll> { + ) -> Poll> { if let Some(e) = self.events.pop_back() { Poll::Ready(NetworkBehaviourAction::GenerateEvent(e)) } else { diff --git a/protocols/ping/tests/ping.rs b/protocols/ping/tests/ping.rs index 3f10ace782b..e0d876bf512 100644 --- a/protocols/ping/tests/ping.rs +++ b/protocols/ping/tests/ping.rs @@ -29,7 +29,7 @@ use libp2p_core::{ }; use libp2p_mplex as mplex; use libp2p_noise as noise; -use libp2p_ping::*; +use libp2p_ping as ping; use libp2p_swarm::{DummyBehaviour, KeepAlive, Swarm, SwarmEvent}; use libp2p_tcp::TcpConfig; use libp2p_yamux as yamux; @@ -40,15 +40,15 @@ use std::{num::NonZeroU8, time::Duration}; #[test] fn ping_pong() { fn prop(count: NonZeroU8, muxer: MuxerChoice) { - let cfg = PingConfig::new() + let cfg = ping::Config::new() .with_keep_alive(true) .with_interval(Duration::from_millis(10)); let (peer1_id, trans) = mk_transport(muxer); - let mut swarm1 = Swarm::new(trans, Ping::new(cfg.clone()), peer1_id.clone()); + let mut swarm1 = Swarm::new(trans, ping::Behaviour::new(cfg.clone()), peer1_id.clone()); let (peer2_id, trans) = mk_transport(muxer); - let mut swarm2 = Swarm::new(trans, Ping::new(cfg), peer2_id.clone()); + let mut swarm2 = Swarm::new(trans, ping::Behaviour::new(cfg), peer2_id.clone()); let (mut tx, mut rx) = mpsc::channel::(1); @@ -63,16 +63,16 @@ fn ping_pong() { loop { match swarm1.select_next_some().await { SwarmEvent::NewListenAddr { address, .. } => tx.send(address).await.unwrap(), - SwarmEvent::Behaviour(PingEvent { + SwarmEvent::Behaviour(ping::Event { peer, - result: Ok(PingSuccess::Ping { rtt }), + result: Ok(ping::Success::Ping { rtt }), }) => { count1 -= 1; if count1 == 0 { return (pid1.clone(), peer, rtt); } } - SwarmEvent::Behaviour(PingEvent { result: Err(e), .. }) => { + SwarmEvent::Behaviour(ping::Event { result: Err(e), .. }) => { panic!("Ping failure: {:?}", e) } _ => {} @@ -86,16 +86,16 @@ fn ping_pong() { loop { match swarm2.select_next_some().await { - SwarmEvent::Behaviour(PingEvent { + SwarmEvent::Behaviour(ping::Event { peer, - result: Ok(PingSuccess::Ping { rtt }), + result: Ok(ping::Success::Ping { rtt }), }) => { count2 -= 1; if count2 == 0 { return (pid2.clone(), peer, rtt); } } - SwarmEvent::Behaviour(PingEvent { result: Err(e), .. }) => { + SwarmEvent::Behaviour(ping::Event { result: Err(e), .. }) => { panic!("Ping failure: {:?}", e) } _ => {} @@ -117,17 +117,17 @@ fn ping_pong() { #[test] fn max_failures() { fn prop(max_failures: NonZeroU8, muxer: MuxerChoice) { - let cfg = PingConfig::new() + let cfg = ping::Config::new() .with_keep_alive(true) .with_interval(Duration::from_millis(10)) .with_timeout(Duration::from_millis(0)) .with_max_failures(max_failures.into()); let (peer1_id, trans) = mk_transport(muxer); - let mut swarm1 = Swarm::new(trans, Ping::new(cfg.clone()), peer1_id.clone()); + let mut swarm1 = Swarm::new(trans, ping::Behaviour::new(cfg.clone()), peer1_id.clone()); let (peer2_id, trans) = mk_transport(muxer); - let mut swarm2 = Swarm::new(trans, Ping::new(cfg), peer2_id.clone()); + let mut swarm2 = Swarm::new(trans, ping::Behaviour::new(cfg), peer2_id.clone()); let (mut tx, mut rx) = mpsc::channel::(1); @@ -140,13 +140,13 @@ fn max_failures() { loop { match swarm1.select_next_some().await { SwarmEvent::NewListenAddr { address, .. } => tx.send(address).await.unwrap(), - SwarmEvent::Behaviour(PingEvent { - result: Ok(PingSuccess::Ping { .. }), + SwarmEvent::Behaviour(ping::Event { + result: Ok(ping::Success::Ping { .. }), .. }) => { count1 = 0; // there may be an occasional success } - SwarmEvent::Behaviour(PingEvent { result: Err(_), .. }) => { + SwarmEvent::Behaviour(ping::Event { result: Err(_), .. }) => { count1 += 1; } SwarmEvent::ConnectionClosed { .. } => return count1, @@ -162,13 +162,13 @@ fn max_failures() { loop { match swarm2.select_next_some().await { - SwarmEvent::Behaviour(PingEvent { - result: Ok(PingSuccess::Ping { .. }), + SwarmEvent::Behaviour(ping::Event { + result: Ok(ping::Success::Ping { .. }), .. }) => { count2 = 0; // there may be an occasional success } - SwarmEvent::Behaviour(PingEvent { result: Err(_), .. }) => { + SwarmEvent::Behaviour(ping::Event { result: Err(_), .. }) => { count2 += 1; } SwarmEvent::ConnectionClosed { .. } => return count2, @@ -197,7 +197,7 @@ fn unsupported_doesnt_fail() { let (peer2_id, trans) = mk_transport(MuxerChoice::Mplex); let mut swarm2 = Swarm::new( trans, - Ping::new(PingConfig::new().with_keep_alive(true)), + ping::Behaviour::new(ping::Config::new().with_keep_alive(true)), peer2_id.clone(), ); @@ -220,8 +220,8 @@ fn unsupported_doesnt_fail() { loop { match swarm2.select_next_some().await { - SwarmEvent::Behaviour(PingEvent { - result: Err(PingFailure::Unsupported), + SwarmEvent::Behaviour(ping::Event { + result: Err(ping::Failure::Unsupported), .. }) => { swarm2.disconnect_peer_id(peer1_id).unwrap(); diff --git a/protocols/relay/CHANGELOG.md b/protocols/relay/CHANGELOG.md index 3e7deca20e4..9f2857bfcc1 100644 --- a/protocols/relay/CHANGELOG.md +++ b/protocols/relay/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.4.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. - Implement `Debug` for `RelayHandlerEvent` and `RelayHandlerIn`. See [PR 2183]. diff --git a/protocols/relay/Cargo.toml b/protocols/relay/Cargo.toml index 159dd7c016a..aedfe262f0b 100644 --- a/protocols/relay/Cargo.toml +++ b/protocols/relay/Cargo.toml @@ -14,7 +14,7 @@ asynchronous-codec = "0.6" bytes = "1" futures = "0.3.1" futures-timer = "3" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } libp2p-swarm = { version = "0.31.0", path = "../../swarm" } log = "0.4" pin-project = "1" diff --git a/protocols/relay/examples/relay.rs b/protocols/relay/examples/relay.rs index 5b8de71702b..52506ce3d3f 100644 --- a/protocols/relay/examples/relay.rs +++ b/protocols/relay/examples/relay.rs @@ -60,13 +60,12 @@ use futures::executor::block_on; use futures::stream::StreamExt; use libp2p::dns::DnsConfig; -use libp2p::ping::{Ping, PingConfig, PingEvent}; use libp2p::plaintext; use libp2p::relay::{Relay, RelayConfig}; use libp2p::swarm::SwarmEvent; use libp2p::tcp::TcpConfig; use libp2p::Transport; -use libp2p::{core::upgrade, identity::ed25519}; +use libp2p::{core::upgrade, identity::ed25519, ping}; use libp2p::{identity, NetworkBehaviour, PeerId, Swarm}; use std::error::Error; use std::task::{Context, Poll}; @@ -99,8 +98,8 @@ fn main() -> Result<(), Box> { let behaviour = Behaviour { relay: relay_behaviour, - ping: Ping::new( - PingConfig::new() + ping: ping::Behaviour::new( + ping::Config::new() .with_keep_alive(true) .with_interval(Duration::from_secs(1)), ), @@ -210,21 +209,21 @@ fn get_client_listen_address(opt: &Opt) -> String { } #[derive(NetworkBehaviour)] -#[behaviour(out_event = "Event", event_process = false)] +#[behaviour(out_event = "Event")] struct Behaviour { relay: Relay, - ping: Ping, + ping: ping::Behaviour, } #[derive(Debug)] enum Event { Relay(()), - Ping(PingEvent), + Ping(ping::Event), } -impl From for Event { - fn from(e: PingEvent) -> Self { - Event::Ping(e) +impl From for Event { + fn from(v: ping::Event) -> Self { + Self::Ping(v) } } diff --git a/protocols/relay/src/behaviour.rs b/protocols/relay/src/behaviour.rs index 9b17eca2c51..8fe510883aa 100644 --- a/protocols/relay/src/behaviour.rs +++ b/protocols/relay/src/behaviour.rs @@ -29,7 +29,8 @@ use libp2p_core::connection::{ConnectedPoint, ConnectionId, ListenerId}; use libp2p_core::multiaddr::Multiaddr; use libp2p_core::PeerId; use libp2p_swarm::{ - DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, + DialError, DialPeerCondition, IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, + NotifyHandler, PollParameters, }; use std::collections::{hash_map::Entry, HashMap, HashSet, VecDeque}; use std::task::{Context, Poll}; @@ -45,7 +46,7 @@ pub struct Relay { /// [`Self::listeners`] or [`Self::listener_any_relay`]. outbox_to_listeners: VecDeque<(PeerId, BehaviourToListenerMsg)>, /// Events that need to be yielded to the outside when polling. - outbox_to_swarm: VecDeque>, + outbox_to_swarm: VecDeque>, /// List of peers the network is connected to. connected_peers: HashMap>, @@ -295,13 +296,26 @@ impl NetworkBehaviour for Relay { .push_back(NetworkBehaviourAction::NotifyHandler { peer_id: *peer_id, handler: NotifyHandler::Any, - event: event, + event, }); } } } - fn inject_dial_failure(&mut self, peer_id: &PeerId) { + fn inject_dial_failure( + &mut self, + peer_id: &PeerId, + _: Self::ProtocolsHandler, + error: DialError, + ) { + if let DialError::DialPeerConditionFalse( + DialPeerCondition::Disconnected | DialPeerCondition::NotDialing, + ) = error + { + // Return early. The dial, that this dial was canceled for, might still succeed. + return; + } + if let Entry::Occupied(o) = self.listeners.entry(*peer_id) { if matches!(o.get(), RelayListener::Connecting { .. }) { // By removing the entry, the channel to the listener is dropped and thus the @@ -340,6 +354,7 @@ impl NetworkBehaviour for Relay { peer: &PeerId, connection: &ConnectionId, _: &ConnectedPoint, + _: ::Handler, ) { // Remove connection from the set of connections for the given peer. In case the set is // empty it will be removed in `inject_disconnected`. @@ -472,10 +487,12 @@ impl NetworkBehaviour for Relay { src_connection_id: connection, }, ); + let handler = self.new_handler(); self.outbox_to_swarm .push_back(NetworkBehaviourAction::DialPeer { peer_id: dest_id, condition: DialPeerCondition::NotDialing, + handler, }); } else { self.outbox_to_swarm @@ -562,7 +579,7 @@ impl NetworkBehaviour for Relay { &mut self, cx: &mut Context<'_>, poll_parameters: &mut impl PollParameters, - ) -> Poll> { + ) -> Poll> { if !self.outbox_to_listeners.is_empty() { let relay_peer_id = self.outbox_to_listeners[0].0; @@ -623,7 +640,7 @@ impl NetworkBehaviour for Relay { dst_peer_id, send_back, })) => { - if let Some(_) = self.connected_peers.get(&relay_peer_id) { + if self.connected_peers.get(&relay_peer_id).is_some() { // In case we are already listening via the relay, // prefer the primary connection. let handler = self @@ -668,6 +685,7 @@ impl NetworkBehaviour for Relay { return Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id: relay_peer_id, condition: DialPeerCondition::Disconnected, + handler: self.new_handler(), }); } } @@ -734,6 +752,7 @@ impl NetworkBehaviour for Relay { return Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id: relay_peer_id, condition: DialPeerCondition::Disconnected, + handler: self.new_handler(), }); } } diff --git a/protocols/relay/src/protocol/incoming_dst_req.rs b/protocols/relay/src/protocol/incoming_dst_req.rs index d68a15121f5..589cbc3d709 100644 --- a/protocols/relay/src/protocol/incoming_dst_req.rs +++ b/protocols/relay/src/protocol/incoming_dst_req.rs @@ -50,10 +50,7 @@ pub struct IncomingDstReq { impl IncomingDstReq { /// Creates a `IncomingDstReq`. pub(crate) fn new(stream: Framed, src: Peer) -> Self { - IncomingDstReq { - stream: stream, - src, - } + IncomingDstReq { stream, src } } /// Returns the peer id of the source that is being relayed. diff --git a/protocols/relay/tests/lib.rs b/protocols/relay/tests/lib.rs index 0829ec87d7b..5f33c293252 100644 --- a/protocols/relay/tests/lib.rs +++ b/protocols/relay/tests/lib.rs @@ -31,7 +31,7 @@ use libp2p_core::transport::{MemoryTransport, Transport, TransportError}; use libp2p_core::{identity, upgrade, PeerId}; use libp2p_identify::{Identify, IdentifyConfig, IdentifyEvent, IdentifyInfo}; use libp2p_kad::{GetClosestPeersOk, Kademlia, KademliaEvent, QueryResult}; -use libp2p_ping::{Ping, PingConfig, PingEvent}; +use libp2p_ping as ping; use libp2p_plaintext::PlainText2Config; use libp2p_relay::{Relay, RelayConfig}; use libp2p_swarm::protocols_handler::KeepAlive; @@ -174,7 +174,7 @@ fn src_connect_to_dst_listening_via_relay() { // Source Node waiting for Ping from Destination Node via Relay. loop { match src_swarm.select_next_some().await { - SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + SwarmEvent::Behaviour(CombinedEvent::Ping(ping::Event { peer, result: Ok(_), })) => { @@ -249,7 +249,7 @@ fn src_connect_to_dst_not_listening_via_active_relay() { // Source Node waiting for Ping from Destination Node via Relay. loop { match src_swarm.select_next_some().await { - SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + SwarmEvent::Behaviour(CombinedEvent::Ping(ping::Event { peer, result: Ok(_), })) => { @@ -337,7 +337,7 @@ fn src_connect_to_dst_via_established_connection_to_relay() { // Source Node waiting for Ping from Destination Node via Relay. loop { match src_swarm.select_next_some().await { - SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + SwarmEvent::Behaviour(CombinedEvent::Ping(ping::Event { peer, result: Ok(_), })) => { @@ -690,7 +690,7 @@ fn firewalled_src_discover_firewalled_dst_via_kad_and_connect_to_dst_via_routabl // Source Node waiting for Ping from Destination Node via Relay. loop { match src_swarm.select_next_some().await { - SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + SwarmEvent::Behaviour(CombinedEvent::Ping(ping::Event { peer, result: Ok(_), })) => { @@ -1019,7 +1019,7 @@ fn yield_incoming_connection_through_correct_listener() { if address == relay_1_addr_incl_circuit || address == relay_2_addr_incl_circuit || address == dst_addr => {} - SwarmEvent::Behaviour(CombinedEvent::Ping(PingEvent { + SwarmEvent::Behaviour(CombinedEvent::Ping(ping::Event { peer, result: Ok(_), })) => { @@ -1129,10 +1129,14 @@ fn yield_incoming_connection_through_correct_listener() { } #[derive(NetworkBehaviour)] -#[behaviour(out_event = "CombinedEvent", poll_method = "poll")] +#[behaviour( + out_event = "CombinedEvent", + poll_method = "poll", + event_process = true +)] struct CombinedBehaviour { relay: Relay, - ping: Ping, + ping: ping::Behaviour, kad: Kademlia, identify: Identify, @@ -1143,15 +1147,16 @@ struct CombinedBehaviour { #[derive(Debug)] enum CombinedEvent { Kad(KademliaEvent), - Ping(PingEvent), + Ping(ping::Event), } impl CombinedBehaviour { - fn poll( + fn poll( &mut self, _: &mut Context, _: &mut impl PollParameters, - ) -> Poll> { + ) -> Poll::ProtocolsHandler>> + { if !self.events.is_empty() { return Poll::Ready(NetworkBehaviourAction::GenerateEvent(self.events.remove(0))); } @@ -1160,8 +1165,8 @@ impl CombinedBehaviour { } } -impl NetworkBehaviourEventProcess for CombinedBehaviour { - fn inject_event(&mut self, event: PingEvent) { +impl NetworkBehaviourEventProcess for CombinedBehaviour { + fn inject_event(&mut self, event: ping::Event) { self.events.push(CombinedEvent::Ping(event)); } } @@ -1194,6 +1199,7 @@ impl NetworkBehaviourEventProcess<()> for CombinedBehaviour { } #[derive(NetworkBehaviour)] +#[behaviour(event_process = true)] struct CombinedKeepAliveBehaviour { relay: Relay, keep_alive: DummyBehaviour, @@ -1287,7 +1293,7 @@ fn build_swarm(reachability: Reachability, relay_mode: RelayMode) -> Swarm"] +license = "MIT" +repository = "https://github.com/libp2p/rust-libp2p" +keywords = ["peer-to-peer", "libp2p", "networking"] +categories = ["network-programming", "asynchronous"] + +[dependencies] +asynchronous-codec = "0.6" +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } +libp2p-swarm = { version = "0.31.0", path = "../../swarm" } +prost = "0.8" +void = "1" +log = "0.4" +futures = { version = "0.3", default-features = false, features = ["std"] } +thiserror = "1" +unsigned-varint = { version = "0.7", features = ["asynchronous_codec"] } +bimap = "0.6.1" +sha2 = "0.9" +rand = "0.8" +wasm-timer = "0.2" + +[dev-dependencies] +libp2p = { path = "../.." } +rand = "0.8" +async-std = { version = "1", features = ["attributes"] } +env_logger = "0.8" +async-trait = "0.1" +tokio = { version = "1", features = [ "rt-multi-thread", "time", "macros", "sync", "process", "fs", "net" ] } + +[build-dependencies] +prost-build = "0.8" diff --git a/protocols/rendezvous/build.rs b/protocols/rendezvous/build.rs new file mode 100644 index 00000000000..fa982fa3d90 --- /dev/null +++ b/protocols/rendezvous/build.rs @@ -0,0 +1,3 @@ +fn main() { + prost_build::compile_protos(&["src/rpc.proto"], &["src"]).unwrap(); +} diff --git a/protocols/rendezvous/examples/discover.rs b/protocols/rendezvous/examples/discover.rs new file mode 100644 index 00000000000..28abdf6d102 --- /dev/null +++ b/protocols/rendezvous/examples/discover.rs @@ -0,0 +1,140 @@ +// Copyright 2021 COMIT Network. +// +// 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 futures::StreamExt; +use libp2p::core::identity; +use libp2p::core::PeerId; +use libp2p::multiaddr::Protocol; +use libp2p::ping::{Ping, PingConfig, PingEvent, PingSuccess}; +use libp2p::swarm::Swarm; +use libp2p::swarm::SwarmEvent; +use libp2p::{development_transport, rendezvous, Multiaddr}; +use std::time::Duration; + +const NAMESPACE: &str = "rendezvous"; + +#[async_std::main] +async fn main() { + env_logger::init(); + + let identity = identity::Keypair::generate_ed25519(); + let rendezvous_point_address = "/ip4/127.0.0.1/tcp/62649".parse::().unwrap(); + let rendezvous_point = "12D3KooWDpJ7As7BWAwRMfu1VU2WCqNjvq387JEYKDBj4kx6nXTN" + .parse() + .unwrap(); + + let mut swarm = Swarm::new( + development_transport(identity.clone()).await.unwrap(), + MyBehaviour { + rendezvous: rendezvous::client::Behaviour::new(identity.clone()), + ping: Ping::new(PingConfig::new().with_interval(Duration::from_secs(1))), + }, + PeerId::from(identity.public()), + ); + + log::info!("Local peer id: {}", swarm.local_peer_id()); + + let _ = swarm.dial_addr(rendezvous_point_address.clone()); + + while let Some(event) = swarm.next().await { + match event { + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == rendezvous_point => { + log::info!( + "Connected to rendezvous point, discovering nodes in '{}' namespace ...", + NAMESPACE + ); + + swarm.behaviour_mut().rendezvous.discover( + Some(rendezvous::Namespace::new(NAMESPACE.to_string()).unwrap()), + None, + None, + rendezvous_point, + ); + } + SwarmEvent::UnreachableAddr { error, address, .. } + | SwarmEvent::UnknownPeerUnreachableAddr { error, address, .. } + if address == rendezvous_point_address => + { + log::error!( + "Failed to connect to rendezvous point at {}: {}", + address, + error + ); + return; + } + SwarmEvent::Behaviour(MyEvent::Rendezvous(rendezvous::client::Event::Discovered { + registrations, + .. + })) => { + for registration in registrations { + for address in registration.record.addresses() { + let peer = registration.record.peer_id(); + log::info!("Discovered peer {} at {}", peer, address); + + let p2p_suffix = Protocol::P2p(*peer.as_ref()); + let address_with_p2p = + if !address.ends_with(&Multiaddr::empty().with(p2p_suffix.clone())) { + address.clone().with(p2p_suffix) + } else { + address.clone() + }; + + swarm.dial_addr(address_with_p2p).unwrap() + } + } + } + SwarmEvent::Behaviour(MyEvent::Ping(PingEvent { + peer, + result: Ok(PingSuccess::Ping { rtt }), + })) if peer != rendezvous_point => { + log::info!("Ping to {} is {}ms", peer, rtt.as_millis()) + } + other => { + log::debug!("Unhandled {:?}", other); + } + } + } +} + +#[derive(Debug)] +enum MyEvent { + Rendezvous(rendezvous::client::Event), + Ping(PingEvent), +} + +impl From for MyEvent { + fn from(event: rendezvous::client::Event) -> Self { + MyEvent::Rendezvous(event) + } +} + +impl From for MyEvent { + fn from(event: PingEvent) -> Self { + MyEvent::Ping(event) + } +} + +#[derive(libp2p::NetworkBehaviour)] +#[behaviour(event_process = false)] +#[behaviour(out_event = "MyEvent")] +struct MyBehaviour { + rendezvous: rendezvous::client::Behaviour, + ping: Ping, +} diff --git a/protocols/rendezvous/examples/register.rs b/protocols/rendezvous/examples/register.rs new file mode 100644 index 00000000000..2c02bca12ef --- /dev/null +++ b/protocols/rendezvous/examples/register.rs @@ -0,0 +1,139 @@ +// Copyright 2021 COMIT Network. +// +// 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 futures::StreamExt; +use libp2p::core::identity; +use libp2p::core::PeerId; +use libp2p::ping::{Ping, PingConfig, PingEvent, PingSuccess}; +use libp2p::swarm::Swarm; +use libp2p::swarm::SwarmEvent; +use libp2p::{development_transport, rendezvous}; +use libp2p::{Multiaddr, NetworkBehaviour}; +use libp2p_swarm::AddressScore; +use std::time::Duration; + +#[async_std::main] +async fn main() { + env_logger::init(); + + let rendezvous_point_address = "/ip4/127.0.0.1/tcp/62649".parse::().unwrap(); + let rendezvous_point = "12D3KooWDpJ7As7BWAwRMfu1VU2WCqNjvq387JEYKDBj4kx6nXTN" + .parse() + .unwrap(); + + let identity = identity::Keypair::generate_ed25519(); + + let mut swarm = Swarm::new( + development_transport(identity.clone()).await.unwrap(), + MyBehaviour { + rendezvous: rendezvous::client::Behaviour::new(identity.clone()), + ping: Ping::new(PingConfig::new().with_interval(Duration::from_secs(1))), + }, + PeerId::from(identity.public()), + ); + + // In production the external address should be the publicly facing IP address of the rendezvous point. + // This address is recorded in the registration entry by the rendezvous point. + let external_address = "/ip4/127.0.0.1/tcp/0".parse::().unwrap(); + swarm.add_external_address(external_address, AddressScore::Infinite); + + log::info!("Local peer id: {}", swarm.local_peer_id()); + + let _ = swarm.listen_on("/ip4/0.0.0.0/tcp/0".parse().unwrap()); + + swarm.dial_addr(rendezvous_point_address).unwrap(); + + while let Some(event) = swarm.next().await { + match event { + SwarmEvent::NewListenAddr { address, .. } => { + log::info!("Listening on {}", address); + } + SwarmEvent::ConnectionClosed { + peer_id, + cause: Some(error), + .. + } if peer_id == rendezvous_point => { + log::error!("Lost connection to rendezvous point {}", error); + } + SwarmEvent::ConnectionEstablished { peer_id, .. } if peer_id == rendezvous_point => { + swarm.behaviour_mut().rendezvous.register( + rendezvous::Namespace::from_static("rendezvous"), + rendezvous_point, + None, + ); + log::info!("Connection established with rendezvous point {}", peer_id); + } + // once `/identify` did its job, we know our external address and can register + SwarmEvent::Behaviour(MyEvent::Rendezvous(rendezvous::client::Event::Registered { + namespace, + ttl, + rendezvous_node, + })) => { + log::info!( + "Registered for namespace '{}' at rendezvous point {} for the next {} seconds", + namespace, + rendezvous_node, + ttl + ); + } + SwarmEvent::Behaviour(MyEvent::Rendezvous( + rendezvous::client::Event::RegisterFailed(error), + )) => { + log::error!("Failed to register {}", error); + return; + } + SwarmEvent::Behaviour(MyEvent::Ping(PingEvent { + peer, + result: Ok(PingSuccess::Ping { rtt }), + })) if peer != rendezvous_point => { + log::info!("Ping to {} is {}ms", peer, rtt.as_millis()) + } + other => { + log::debug!("Unhandled {:?}", other); + } + } + } +} + +#[derive(Debug)] +enum MyEvent { + Rendezvous(rendezvous::client::Event), + Ping(PingEvent), +} + +impl From for MyEvent { + fn from(event: rendezvous::client::Event) -> Self { + MyEvent::Rendezvous(event) + } +} + +impl From for MyEvent { + fn from(event: PingEvent) -> Self { + MyEvent::Ping(event) + } +} + +#[derive(NetworkBehaviour)] +#[behaviour(event_process = false)] +#[behaviour(out_event = "MyEvent")] +struct MyBehaviour { + rendezvous: rendezvous::client::Behaviour, + ping: Ping, +} diff --git a/protocols/rendezvous/examples/register_with_identify.rs b/protocols/rendezvous/examples/register_with_identify.rs new file mode 100644 index 00000000000..25450d4ab02 --- /dev/null +++ b/protocols/rendezvous/examples/register_with_identify.rs @@ -0,0 +1,145 @@ +// Copyright 2021 COMIT Network. +// +// 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 futures::StreamExt; +use libp2p::core::identity; +use libp2p::core::PeerId; +use libp2p::identify::{Identify, IdentifyConfig, IdentifyEvent}; +use libp2p::ping::{Ping, PingConfig, PingEvent, PingSuccess}; +use libp2p::swarm::Swarm; +use libp2p::swarm::SwarmEvent; +use libp2p::{development_transport, rendezvous}; +use libp2p::{Multiaddr, NetworkBehaviour}; +use std::time::Duration; + +#[async_std::main] +async fn main() { + env_logger::init(); + + let rendezvous_point_address = "/ip4/127.0.0.1/tcp/62649".parse::().unwrap(); + let rendezvous_point = "12D3KooWDpJ7As7BWAwRMfu1VU2WCqNjvq387JEYKDBj4kx6nXTN" + .parse() + .unwrap(); + + let identity = identity::Keypair::generate_ed25519(); + + let mut swarm = Swarm::new( + development_transport(identity.clone()).await.unwrap(), + MyBehaviour { + identify: Identify::new(IdentifyConfig::new( + "rendezvous-example/1.0.0".to_string(), + identity.public(), + )), + rendezvous: rendezvous::client::Behaviour::new(identity.clone()), + ping: Ping::new(PingConfig::new().with_interval(Duration::from_secs(1))), + }, + PeerId::from(identity.public()), + ); + + log::info!("Local peer id: {}", swarm.local_peer_id()); + + let _ = swarm.listen_on("/ip4/0.0.0.0/tcp/0".parse().unwrap()); + + swarm.dial_addr(rendezvous_point_address).unwrap(); + + while let Some(event) = swarm.next().await { + match event { + SwarmEvent::NewListenAddr { address, .. } => { + log::info!("Listening on {}", address); + } + SwarmEvent::ConnectionClosed { + peer_id, + cause: Some(error), + .. + } if peer_id == rendezvous_point => { + log::error!("Lost connection to rendezvous point {}", error); + } + // once `/identify` did its job, we know our external address and can register + SwarmEvent::Behaviour(MyEvent::Identify(IdentifyEvent::Received { .. })) => { + swarm.behaviour_mut().rendezvous.register( + rendezvous::Namespace::from_static("rendezvous"), + rendezvous_point, + None, + ); + } + SwarmEvent::Behaviour(MyEvent::Rendezvous(rendezvous::client::Event::Registered { + namespace, + ttl, + rendezvous_node, + })) => { + log::info!( + "Registered for namespace '{}' at rendezvous point {} for the next {} seconds", + namespace, + rendezvous_node, + ttl + ); + } + SwarmEvent::Behaviour(MyEvent::Rendezvous( + rendezvous::client::Event::RegisterFailed(error), + )) => { + log::error!("Failed to register {}", error); + return; + } + SwarmEvent::Behaviour(MyEvent::Ping(PingEvent { + peer, + result: Ok(PingSuccess::Ping { rtt }), + })) if peer != rendezvous_point => { + log::info!("Ping to {} is {}ms", peer, rtt.as_millis()) + } + other => { + log::debug!("Unhandled {:?}", other); + } + } + } +} + +#[derive(Debug)] +enum MyEvent { + Rendezvous(rendezvous::client::Event), + Identify(IdentifyEvent), + Ping(PingEvent), +} + +impl From for MyEvent { + fn from(event: rendezvous::client::Event) -> Self { + MyEvent::Rendezvous(event) + } +} + +impl From for MyEvent { + fn from(event: IdentifyEvent) -> Self { + MyEvent::Identify(event) + } +} + +impl From for MyEvent { + fn from(event: PingEvent) -> Self { + MyEvent::Ping(event) + } +} + +#[derive(NetworkBehaviour)] +#[behaviour(event_process = false)] +#[behaviour(out_event = "MyEvent")] +struct MyBehaviour { + identify: Identify, + rendezvous: rendezvous::client::Behaviour, + ping: Ping, +} diff --git a/protocols/rendezvous/examples/rendezvous_point.rs b/protocols/rendezvous/examples/rendezvous_point.rs new file mode 100644 index 00000000000..bbbe9b973be --- /dev/null +++ b/protocols/rendezvous/examples/rendezvous_point.rs @@ -0,0 +1,112 @@ +// Copyright 2021 COMIT Network. +// +// 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 futures::StreamExt; +use libp2p::core::identity; +use libp2p::core::PeerId; +use libp2p::ping::{Ping, PingEvent}; +use libp2p::swarm::{Swarm, SwarmEvent}; +use libp2p::NetworkBehaviour; +use libp2p::{development_transport, rendezvous}; + +#[tokio::main] +async fn main() { + env_logger::init(); + + let bytes = [0u8; 32]; + let key = identity::ed25519::SecretKey::from_bytes(bytes).expect("we always pass 32 bytes"); + let identity = identity::Keypair::Ed25519(key.into()); + + let mut swarm = Swarm::new( + development_transport(identity.clone()).await.unwrap(), + MyBehaviour { + rendezvous: rendezvous::server::Behaviour::new(rendezvous::server::Config::default()), + ping: Ping::default(), + }, + PeerId::from(identity.public()), + ); + + log::info!("Local peer id: {}", swarm.local_peer_id()); + + swarm + .listen_on("/ip4/0.0.0.0/tcp/62649".parse().unwrap()) + .unwrap(); + + while let Some(event) = swarm.next().await { + match event { + SwarmEvent::ConnectionEstablished { peer_id, .. } => { + log::info!("Connected to {}", peer_id); + } + SwarmEvent::ConnectionClosed { peer_id, .. } => { + log::info!("Disconnected from {}", peer_id); + } + SwarmEvent::Behaviour(MyEvent::Rendezvous( + rendezvous::server::Event::PeerRegistered { peer, registration }, + )) => { + log::info!( + "Peer {} registered for namespace '{}'", + peer, + registration.namespace + ); + } + SwarmEvent::Behaviour(MyEvent::Rendezvous( + rendezvous::server::Event::DiscoverServed { + enquirer, + registrations, + }, + )) => { + log::info!( + "Served peer {} with {} registrations", + enquirer, + registrations.len() + ); + } + other => { + log::debug!("Unhandled {:?}", other); + } + } + } +} + +#[derive(Debug)] +enum MyEvent { + Rendezvous(rendezvous::server::Event), + Ping(PingEvent), +} + +impl From for MyEvent { + fn from(event: rendezvous::server::Event) -> Self { + MyEvent::Rendezvous(event) + } +} + +impl From for MyEvent { + fn from(event: PingEvent) -> Self { + MyEvent::Ping(event) + } +} + +#[derive(NetworkBehaviour)] +#[behaviour(event_process = false)] +#[behaviour(out_event = "MyEvent")] +struct MyBehaviour { + rendezvous: rendezvous::server::Behaviour, + ping: Ping, +} diff --git a/protocols/rendezvous/src/client.rs b/protocols/rendezvous/src/client.rs new file mode 100644 index 00000000000..73eb90edb38 --- /dev/null +++ b/protocols/rendezvous/src/client.rs @@ -0,0 +1,337 @@ +// Copyright 2021 COMIT Network. +// +// 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::codec::{Cookie, ErrorCode, Namespace, NewRegistration, Registration, Ttl}; +use crate::handler; +use crate::handler::outbound; +use crate::handler::outbound::OpenInfo; +use crate::substream_handler::SubstreamProtocolsHandler; +use futures::future::BoxFuture; +use futures::future::FutureExt; +use futures::stream::FuturesUnordered; +use futures::stream::StreamExt; +use libp2p_core::connection::ConnectionId; +use libp2p_core::identity::error::SigningError; +use libp2p_core::identity::Keypair; +use libp2p_core::{Multiaddr, PeerId, PeerRecord}; +use libp2p_swarm::{ + CloseConnection, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, +}; +use std::collections::{HashMap, VecDeque}; +use std::iter::FromIterator; +use std::task::{Context, Poll}; +use std::time::Duration; + +pub struct Behaviour { + events: VecDeque< + NetworkBehaviourAction< + Event, + SubstreamProtocolsHandler, + >, + >, + keypair: Keypair, + pending_register_requests: Vec<(Namespace, PeerId, Option)>, + + /// Hold addresses of all peers that we have discovered so far. + /// + /// Storing these internally allows us to assist the [`libp2p_swarm::Swarm`] in dialing by returning addresses from [`NetworkBehaviour::addresses_of_peer`]. + discovered_peers: HashMap<(PeerId, Namespace), Vec>, + + /// Tracks the expiry of registrations that we have discovered and stored in `discovered_peers` otherwise we have a memory leak. + expiring_registrations: FuturesUnordered>, +} + +impl Behaviour { + /// Create a new instance of the rendezvous [`NetworkBehaviour`]. + pub fn new(keypair: Keypair) -> Self { + Self { + events: Default::default(), + keypair, + pending_register_requests: vec![], + discovered_peers: Default::default(), + expiring_registrations: FuturesUnordered::from_iter(vec![ + futures::future::pending().boxed() + ]), + } + } + + /// Register our external addresses in the given namespace with the given rendezvous peer. + /// + /// External addresses are either manually added via [`libp2p_swarm::Swarm::add_external_address`] or reported + /// by other [`NetworkBehaviour`]s via [`NetworkBehaviourAction::ReportObservedAddr`]. + pub fn register(&mut self, namespace: Namespace, rendezvous_node: PeerId, ttl: Option) { + self.pending_register_requests + .push((namespace, rendezvous_node, ttl)); + } + + /// Unregister ourselves from the given namespace with the given rendezvous peer. + pub fn unregister(&mut self, namespace: Namespace, rendezvous_node: PeerId) { + self.events + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: rendezvous_node, + event: handler::OutboundInEvent::NewSubstream { + open_info: OpenInfo::UnregisterRequest(namespace), + }, + handler: NotifyHandler::Any, + }); + } + + /// Discover other peers at a given rendezvous peer. + /// + /// If desired, the registrations can be filtered by a namespace. + /// If no namespace is given, peers from all namespaces will be returned. + /// A successfully discovery returns a cookie within [`Event::Discovered`]. + /// Such a cookie can be used to only fetch the _delta_ of registrations since + /// the cookie was acquired. + pub fn discover( + &mut self, + ns: Option, + cookie: Option, + limit: Option, + rendezvous_node: PeerId, + ) { + self.events + .push_back(NetworkBehaviourAction::NotifyHandler { + peer_id: rendezvous_node, + event: handler::OutboundInEvent::NewSubstream { + open_info: OpenInfo::DiscoverRequest { + namespace: ns, + cookie, + limit, + }, + }, + handler: NotifyHandler::Any, + }); + } +} + +#[derive(Debug, thiserror::Error)] +pub enum RegisterError { + #[error("We don't know about any externally reachable addresses of ours")] + NoExternalAddresses, + #[error("Failed to make a new PeerRecord")] + FailedToMakeRecord(#[from] SigningError), + #[error("Failed to register with Rendezvous node")] + Remote { + rendezvous_node: PeerId, + namespace: Namespace, + error: ErrorCode, + }, +} + +#[derive(Debug)] +#[allow(clippy::large_enum_variant)] +pub enum Event { + /// We successfully discovered other nodes with using the contained rendezvous node. + Discovered { + rendezvous_node: PeerId, + registrations: Vec, + cookie: Cookie, + }, + /// We failed to discover other nodes on the contained rendezvous node. + DiscoverFailed { + rendezvous_node: PeerId, + namespace: Option, + error: ErrorCode, + }, + /// We successfully registered with the contained rendezvous node. + Registered { + rendezvous_node: PeerId, + ttl: Ttl, + namespace: Namespace, + }, + /// We failed to register with the contained rendezvous node. + RegisterFailed(RegisterError), + /// The connection details we learned from this node expired. + Expired { peer: PeerId }, +} + +impl NetworkBehaviour for Behaviour { + type ProtocolsHandler = + SubstreamProtocolsHandler; + type OutEvent = Event; + + fn new_handler(&mut self) -> Self::ProtocolsHandler { + let initial_keep_alive = Duration::from_secs(30); + + SubstreamProtocolsHandler::new_outbound_only(initial_keep_alive) + } + + fn addresses_of_peer(&mut self, peer: &PeerId) -> Vec { + self.discovered_peers + .iter() + .filter_map(|((candidate, _), addresses)| (candidate == peer).then(|| addresses)) + .flatten() + .cloned() + .collect() + } + + fn inject_event( + &mut self, + peer_id: PeerId, + connection_id: ConnectionId, + event: handler::OutboundOutEvent, + ) { + let new_events = match event { + handler::OutboundOutEvent::InboundEvent { message, .. } => void::unreachable(message), + handler::OutboundOutEvent::OutboundEvent { message, .. } => handle_outbound_event( + message, + peer_id, + &mut self.discovered_peers, + &mut self.expiring_registrations, + ), + handler::OutboundOutEvent::InboundError { error, .. } => void::unreachable(error), + handler::OutboundOutEvent::OutboundError { error, .. } => { + log::warn!("Connection with peer {} failed: {}", peer_id, error); + + vec![NetworkBehaviourAction::CloseConnection { + peer_id, + connection: CloseConnection::One(connection_id), + }] + } + }; + + self.events.extend(new_events); + } + + fn poll( + &mut self, + cx: &mut Context<'_>, + poll_params: &mut impl PollParameters, + ) -> Poll> { + if let Some(event) = self.events.pop_front() { + return Poll::Ready(event); + } + + if let Some((namespace, rendezvous_node, ttl)) = self.pending_register_requests.pop() { + // Update our external addresses based on the Swarm's current knowledge. + // It doesn't make sense to register addresses on which we are not reachable, hence this should not be configurable from the outside. + let external_addresses = poll_params + .external_addresses() + .map(|r| r.addr) + .collect::>(); + + if external_addresses.is_empty() { + return Poll::Ready(NetworkBehaviourAction::GenerateEvent( + Event::RegisterFailed(RegisterError::NoExternalAddresses), + )); + } + + let action = match PeerRecord::new(self.keypair.clone(), external_addresses) { + Ok(peer_record) => NetworkBehaviourAction::NotifyHandler { + peer_id: rendezvous_node, + event: handler::OutboundInEvent::NewSubstream { + open_info: OpenInfo::RegisterRequest(NewRegistration { + namespace, + record: peer_record, + ttl, + }), + }, + handler: NotifyHandler::Any, + }, + Err(signing_error) => NetworkBehaviourAction::GenerateEvent(Event::RegisterFailed( + RegisterError::FailedToMakeRecord(signing_error), + )), + }; + + return Poll::Ready(action); + } + + if let Some(expired_registration) = + futures::ready!(self.expiring_registrations.poll_next_unpin(cx)) + { + self.discovered_peers.remove(&expired_registration); + return Poll::Ready(NetworkBehaviourAction::GenerateEvent(Event::Expired { + peer: expired_registration.0, + })); + } + + Poll::Pending + } +} + +fn handle_outbound_event( + event: outbound::OutEvent, + peer_id: PeerId, + discovered_peers: &mut HashMap<(PeerId, Namespace), Vec>, + expiring_registrations: &mut FuturesUnordered>, +) -> Vec< + NetworkBehaviourAction< + Event, + SubstreamProtocolsHandler, + >, +> { + match event { + outbound::OutEvent::Registered { namespace, ttl } => { + vec![NetworkBehaviourAction::GenerateEvent(Event::Registered { + rendezvous_node: peer_id, + ttl, + namespace, + })] + } + outbound::OutEvent::RegisterFailed(namespace, error) => { + vec![NetworkBehaviourAction::GenerateEvent( + Event::RegisterFailed(RegisterError::Remote { + rendezvous_node: peer_id, + namespace, + error, + }), + )] + } + outbound::OutEvent::Discovered { + registrations, + cookie, + } => { + discovered_peers.extend(registrations.iter().map(|registration| { + let peer_id = registration.record.peer_id(); + let namespace = registration.namespace.clone(); + + let addresses = registration.record.addresses().to_vec(); + + ((peer_id, namespace), addresses) + })); + expiring_registrations.extend(registrations.iter().cloned().map(|registration| { + async move { + // if the timer errors we consider it expired + let _ = + wasm_timer::Delay::new(Duration::from_secs(registration.ttl as u64)).await; + + (registration.record.peer_id(), registration.namespace) + } + .boxed() + })); + + vec![NetworkBehaviourAction::GenerateEvent(Event::Discovered { + rendezvous_node: peer_id, + registrations, + cookie, + })] + } + outbound::OutEvent::DiscoverFailed { namespace, error } => { + vec![NetworkBehaviourAction::GenerateEvent( + Event::DiscoverFailed { + rendezvous_node: peer_id, + namespace, + error, + }, + )] + } + } +} diff --git a/protocols/rendezvous/src/codec.rs b/protocols/rendezvous/src/codec.rs new file mode 100644 index 00000000000..d050ff8ca9a --- /dev/null +++ b/protocols/rendezvous/src/codec.rs @@ -0,0 +1,622 @@ +// Copyright 2021 COMIT Network. +// +// 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::DEFAULT_TTL; +use asynchronous_codec::{Bytes, BytesMut, Decoder, Encoder}; +use libp2p_core::{peer_record, signed_envelope, PeerRecord, SignedEnvelope}; +use rand::RngCore; +use std::convert::{TryFrom, TryInto}; +use std::fmt; +use unsigned_varint::codec::UviBytes; + +pub type Ttl = u64; + +#[derive(Debug, Clone)] +pub enum Message { + Register(NewRegistration), + RegisterResponse(Result), + Unregister(Namespace), + Discover { + namespace: Option, + cookie: Option, + limit: Option, + }, + DiscoverResponse(Result<(Vec, Cookie), ErrorCode>), +} + +#[derive(Debug, PartialEq, Eq, Hash, Clone)] +pub struct Namespace(String); + +impl Namespace { + /// Creates a new [`Namespace`] from a static string. + /// + /// This will panic if the namespace is too long. We accepting panicking in this case because we are enforcing a `static lifetime which means this value can only be a constant in the program and hence we hope the developer checked that it is of an acceptable length. + pub fn from_static(value: &'static str) -> Self { + if value.len() > 255 { + panic!("Namespace '{}' is too long!", value) + } + + Namespace(value.to_owned()) + } + + pub fn new(value: String) -> Result { + if value.len() > 255 { + return Err(NamespaceTooLong); + } + + Ok(Namespace(value)) + } +} + +impl From for String { + fn from(namespace: Namespace) -> Self { + namespace.0 + } +} + +impl fmt::Display for Namespace { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.0) + } +} + +impl PartialEq for Namespace { + fn eq(&self, other: &str) -> bool { + self.0.eq(other) + } +} + +impl PartialEq for str { + fn eq(&self, other: &Namespace) -> bool { + other.0.eq(self) + } +} + +#[derive(Debug, thiserror::Error)] +#[error("Namespace is too long")] +pub struct NamespaceTooLong; + +#[derive(Debug, Eq, PartialEq, Hash, Clone)] +pub struct Cookie { + id: u64, + namespace: Option, +} + +impl Cookie { + /// Construct a new [`Cookie`] for a given namespace. + /// + /// This cookie will only be valid for subsequent DISCOVER requests targeting the same namespace. + pub fn for_namespace(namespace: Namespace) -> Self { + Self { + id: rand::thread_rng().next_u64(), + namespace: Some(namespace), + } + } + + /// Construct a new [`Cookie`] for a DISCOVER request that inquires about all namespaces. + pub fn for_all_namespaces() -> Self { + Self { + id: rand::random(), + namespace: None, + } + } + + pub fn into_wire_encoding(self) -> Vec { + let id_bytes = self.id.to_be_bytes(); + let namespace = self.namespace.map(|ns| ns.0).unwrap_or_default(); + + let mut buffer = Vec::with_capacity(id_bytes.len() + namespace.len()); + buffer.extend_from_slice(&id_bytes); + buffer.extend_from_slice(namespace.as_bytes()); + + buffer + } + + pub fn from_wire_encoding(mut bytes: Vec) -> Result { + // check length early to avoid panic during slicing + if bytes.len() < 8 { + return Err(InvalidCookie); + } + + let namespace = bytes.split_off(8); + let namespace = if namespace.is_empty() { + None + } else { + Some( + Namespace::new(String::from_utf8(namespace).map_err(|_| InvalidCookie)?) + .map_err(|_| InvalidCookie)?, + ) + }; + + let bytes = <[u8; 8]>::try_from(bytes).map_err(|_| InvalidCookie)?; + let id = u64::from_be_bytes(bytes); + + Ok(Self { id, namespace }) + } + + pub fn namespace(&self) -> Option<&Namespace> { + self.namespace.as_ref() + } +} + +#[derive(Debug, thiserror::Error)] +#[error("The cookie was malformed")] +pub struct InvalidCookie; + +#[derive(Debug, Clone)] +pub struct NewRegistration { + pub namespace: Namespace, + pub record: PeerRecord, + pub ttl: Option, +} + +impl NewRegistration { + pub fn new(namespace: Namespace, record: PeerRecord, ttl: Option) -> Self { + Self { + namespace, + record, + ttl, + } + } + + pub fn effective_ttl(&self) -> Ttl { + self.ttl.unwrap_or(DEFAULT_TTL) + } +} + +#[derive(Debug, Clone, PartialEq)] +pub struct Registration { + pub namespace: Namespace, + pub record: PeerRecord, + pub ttl: Ttl, +} + +#[derive(Debug, Copy, Clone, PartialEq, Eq)] +pub enum ErrorCode { + InvalidNamespace, + InvalidSignedPeerRecord, + InvalidTtl, + InvalidCookie, + NotAuthorized, + InternalError, + Unavailable, +} + +pub struct RendezvousCodec { + /// Codec to encode/decode the Unsigned varint length prefix of the frames. + length_codec: UviBytes, +} + +impl Default for RendezvousCodec { + fn default() -> Self { + let mut length_codec = UviBytes::default(); + length_codec.set_max_len(1024 * 1024); // 1MB + + Self { length_codec } + } +} + +impl Encoder for RendezvousCodec { + type Item = Message; + type Error = Error; + + fn encode(&mut self, item: Self::Item, dst: &mut BytesMut) -> Result<(), Self::Error> { + use prost::Message; + + let message = wire::Message::from(item); + + let mut buf = Vec::with_capacity(message.encoded_len()); + + message + .encode(&mut buf) + .expect("Buffer has sufficient capacity"); + + // Length prefix the protobuf message, ensuring the max limit is not hit + self.length_codec.encode(Bytes::from(buf), dst)?; + + Ok(()) + } +} + +impl Decoder for RendezvousCodec { + type Item = Message; + type Error = Error; + + fn decode(&mut self, src: &mut BytesMut) -> Result, Self::Error> { + use prost::Message; + + let message = match self.length_codec.decode(src)? { + Some(p) => p, + None => return Ok(None), + }; + + let message = wire::Message::decode(message)?; + + Ok(Some(message.try_into()?)) + } +} + +#[derive(Debug, thiserror::Error)] +pub enum Error { + #[error("Failed to encode message as bytes")] + Encode(#[from] prost::EncodeError), + #[error("Failed to decode message from bytes")] + Decode(#[from] prost::DecodeError), + #[error("Failed to read/write")] + Io(#[from] std::io::Error), + #[error("Failed to convert wire message to internal data model")] + ConversionError(#[from] ConversionError), +} + +impl From for wire::Message { + fn from(message: Message) -> Self { + use wire::message::*; + + match message { + Message::Register(NewRegistration { + namespace, + record, + ttl, + }) => wire::Message { + r#type: Some(MessageType::Register.into()), + register: Some(Register { + ns: Some(namespace.into()), + ttl, + signed_peer_record: Some( + record.into_signed_envelope().into_protobuf_encoding(), + ), + }), + register_response: None, + unregister: None, + discover: None, + discover_response: None, + }, + Message::RegisterResponse(Ok(ttl)) => wire::Message { + r#type: Some(MessageType::RegisterResponse.into()), + register_response: Some(RegisterResponse { + status: Some(ResponseStatus::Ok.into()), + status_text: None, + ttl: Some(ttl), + }), + register: None, + discover: None, + unregister: None, + discover_response: None, + }, + Message::RegisterResponse(Err(error)) => wire::Message { + r#type: Some(MessageType::RegisterResponse.into()), + register_response: Some(RegisterResponse { + status: Some(ResponseStatus::from(error).into()), + status_text: None, + ttl: None, + }), + register: None, + discover: None, + unregister: None, + discover_response: None, + }, + Message::Unregister(namespace) => wire::Message { + r#type: Some(MessageType::Unregister.into()), + unregister: Some(Unregister { + ns: Some(namespace.into()), + id: None, + }), + register: None, + register_response: None, + discover: None, + discover_response: None, + }, + Message::Discover { + namespace, + cookie, + limit, + } => wire::Message { + r#type: Some(MessageType::Discover.into()), + discover: Some(Discover { + ns: namespace.map(|ns| ns.into()), + cookie: cookie.map(|cookie| cookie.into_wire_encoding()), + limit, + }), + register: None, + register_response: None, + unregister: None, + discover_response: None, + }, + Message::DiscoverResponse(Ok((registrations, cookie))) => wire::Message { + r#type: Some(MessageType::DiscoverResponse.into()), + discover_response: Some(DiscoverResponse { + registrations: registrations + .into_iter() + .map(|reggo| Register { + ns: Some(reggo.namespace.into()), + ttl: Some(reggo.ttl), + signed_peer_record: Some( + reggo.record.into_signed_envelope().into_protobuf_encoding(), + ), + }) + .collect(), + status: Some(ResponseStatus::Ok.into()), + status_text: None, + cookie: Some(cookie.into_wire_encoding()), + }), + register: None, + discover: None, + unregister: None, + register_response: None, + }, + Message::DiscoverResponse(Err(error)) => wire::Message { + r#type: Some(MessageType::DiscoverResponse.into()), + discover_response: Some(DiscoverResponse { + registrations: Vec::new(), + status: Some(ResponseStatus::from(error).into()), + status_text: None, + cookie: None, + }), + register: None, + discover: None, + unregister: None, + register_response: None, + }, + } + } +} + +impl TryFrom for Message { + type Error = ConversionError; + + fn try_from(message: wire::Message) -> Result { + use wire::message::*; + + let message = match message { + wire::Message { + r#type: Some(0), + register: + Some(Register { + ns, + ttl, + signed_peer_record: Some(signed_peer_record), + }), + .. + } => Message::Register(NewRegistration { + namespace: ns + .map(Namespace::new) + .transpose()? + .ok_or(ConversionError::MissingNamespace)?, + ttl, + record: PeerRecord::from_signed_envelope(SignedEnvelope::from_protobuf_encoding( + &signed_peer_record, + )?)?, + }), + wire::Message { + r#type: Some(1), + register_response: + Some(RegisterResponse { + status: Some(0), + ttl, + .. + }), + .. + } => Message::RegisterResponse(Ok(ttl.ok_or(ConversionError::MissingTtl)?)), + wire::Message { + r#type: Some(3), + discover: Some(Discover { ns, limit, cookie }), + .. + } => Message::Discover { + namespace: ns.map(Namespace::new).transpose()?, + cookie: cookie.map(Cookie::from_wire_encoding).transpose()?, + limit, + }, + wire::Message { + r#type: Some(4), + discover_response: + Some(DiscoverResponse { + registrations, + status: Some(0), + cookie: Some(cookie), + .. + }), + .. + } => { + let registrations = registrations + .into_iter() + .map(|reggo| { + Ok(Registration { + namespace: reggo + .ns + .map(Namespace::new) + .transpose()? + .ok_or(ConversionError::MissingNamespace)?, + record: PeerRecord::from_signed_envelope( + SignedEnvelope::from_protobuf_encoding( + ®go + .signed_peer_record + .ok_or(ConversionError::MissingSignedPeerRecord)?, + )?, + )?, + ttl: reggo.ttl.ok_or(ConversionError::MissingTtl)?, + }) + }) + .collect::, ConversionError>>()?; + let cookie = Cookie::from_wire_encoding(cookie)?; + + Message::DiscoverResponse(Ok((registrations, cookie))) + } + wire::Message { + r#type: Some(1), + register_response: + Some(RegisterResponse { + status: Some(error_code), + .. + }), + .. + } => { + let error_code = wire::message::ResponseStatus::from_i32(error_code) + .ok_or(ConversionError::BadStatusCode)? + .try_into()?; + Message::RegisterResponse(Err(error_code)) + } + wire::Message { + r#type: Some(2), + unregister: Some(Unregister { ns, .. }), + .. + } => Message::Unregister( + ns.map(Namespace::new) + .transpose()? + .ok_or(ConversionError::MissingNamespace)?, + ), + wire::Message { + r#type: Some(4), + discover_response: + Some(DiscoverResponse { + status: Some(error_code), + .. + }), + .. + } => { + let error = wire::message::ResponseStatus::from_i32(error_code) + .ok_or(ConversionError::BadStatusCode)? + .try_into()?; + Message::DiscoverResponse(Err(error)) + } + _ => return Err(ConversionError::InconsistentWireMessage), + }; + + Ok(message) + } +} + +#[derive(Debug, thiserror::Error)] +pub enum ConversionError { + #[error("The wire message is consistent")] + InconsistentWireMessage, + #[error("Missing namespace field")] + MissingNamespace, + #[error("Invalid namespace")] + InvalidNamespace(#[from] NamespaceTooLong), + #[error("Missing signed peer record field")] + MissingSignedPeerRecord, + #[error("Missing TTL field")] + MissingTtl, + #[error("Bad status code")] + BadStatusCode, + #[error("Failed to decode signed envelope")] + BadSignedEnvelope(#[from] signed_envelope::DecodingError), + #[error("Failed to decode envelope as signed peer record")] + BadSignedPeerRecord(#[from] peer_record::FromEnvelopeError), + #[error(transparent)] + BadCookie(#[from] InvalidCookie), + #[error("The requested PoW difficulty is out of range")] + PoWDifficultyOutOfRange, + #[error("The provided PoW hash is not 32 bytes long")] + BadPoWHash, +} + +impl ConversionError { + pub fn to_error_code(&self) -> ErrorCode { + match self { + ConversionError::MissingNamespace => ErrorCode::InvalidNamespace, + ConversionError::MissingSignedPeerRecord => ErrorCode::InvalidSignedPeerRecord, + ConversionError::BadSignedEnvelope(_) => ErrorCode::InvalidSignedPeerRecord, + ConversionError::BadSignedPeerRecord(_) => ErrorCode::InvalidSignedPeerRecord, + ConversionError::BadCookie(_) => ErrorCode::InvalidCookie, + ConversionError::MissingTtl => ErrorCode::InvalidTtl, + ConversionError::InconsistentWireMessage => ErrorCode::InternalError, + ConversionError::BadStatusCode => ErrorCode::InternalError, + ConversionError::PoWDifficultyOutOfRange => ErrorCode::InternalError, + ConversionError::BadPoWHash => ErrorCode::InternalError, + ConversionError::InvalidNamespace(_) => ErrorCode::InvalidNamespace, + } + } +} + +impl TryFrom for ErrorCode { + type Error = UnmappableStatusCode; + + fn try_from(value: wire::message::ResponseStatus) -> Result { + use wire::message::ResponseStatus::*; + + let code = match value { + Ok => return Err(UnmappableStatusCode(value)), + EInvalidNamespace => ErrorCode::InvalidNamespace, + EInvalidSignedPeerRecord => ErrorCode::InvalidSignedPeerRecord, + EInvalidTtl => ErrorCode::InvalidTtl, + EInvalidCookie => ErrorCode::InvalidCookie, + ENotAuthorized => ErrorCode::NotAuthorized, + EInternalError => ErrorCode::InternalError, + EUnavailable => ErrorCode::Unavailable, + }; + + Result::Ok(code) + } +} + +impl From for wire::message::ResponseStatus { + fn from(error_code: ErrorCode) -> Self { + use wire::message::ResponseStatus::*; + + match error_code { + ErrorCode::InvalidNamespace => EInvalidNamespace, + ErrorCode::InvalidSignedPeerRecord => EInvalidSignedPeerRecord, + ErrorCode::InvalidTtl => EInvalidTtl, + ErrorCode::InvalidCookie => EInvalidCookie, + ErrorCode::NotAuthorized => ENotAuthorized, + ErrorCode::InternalError => EInternalError, + ErrorCode::Unavailable => EUnavailable, + } + } +} + +impl From for ConversionError { + fn from(_: UnmappableStatusCode) -> Self { + ConversionError::InconsistentWireMessage + } +} + +#[derive(Debug, thiserror::Error)] +#[error("The response code ({0:?}) cannot be mapped to our ErrorCode enum")] +pub struct UnmappableStatusCode(wire::message::ResponseStatus); + +mod wire { + include!(concat!(env!("OUT_DIR"), "/rendezvous.pb.rs")); +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn cookie_wire_encoding_roundtrip() { + let cookie = Cookie::for_namespace(Namespace::from_static("foo")); + + let bytes = cookie.clone().into_wire_encoding(); + let parsed = Cookie::from_wire_encoding(bytes).unwrap(); + + assert_eq!(parsed, cookie); + } + + #[test] + fn cookie_wire_encoding_length() { + let cookie = Cookie::for_namespace(Namespace::from_static("foo")); + + let bytes = cookie.into_wire_encoding(); + + assert_eq!(bytes.len(), 8 + 3) + } +} diff --git a/protocols/rendezvous/src/handler.rs b/protocols/rendezvous/src/handler.rs new file mode 100644 index 00000000000..b4883825e25 --- /dev/null +++ b/protocols/rendezvous/src/handler.rs @@ -0,0 +1,48 @@ +// Copyright 2021 COMIT Network. +// +// 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::codec; +use crate::codec::Message; +use void::Void; + +const PROTOCOL_IDENT: &[u8] = b"/rendezvous/1.0.0"; + +pub mod inbound; +pub mod outbound; + +/// Errors that can occur while interacting with a substream. +#[derive(Debug, thiserror::Error)] +pub enum Error { + #[error("Reading message {0:?} at this stage is a protocol violation")] + BadMessage(Message), + #[error("Failed to write message to substream")] + WriteMessage(#[source] codec::Error), + #[error("Failed to read message from substream")] + ReadMessage(#[source] codec::Error), + #[error("Substream ended unexpectedly mid-protocol")] + UnexpectedEndOfStream, +} + +pub type OutboundInEvent = crate::substream_handler::InEvent; +pub type OutboundOutEvent = + crate::substream_handler::OutEvent; + +pub type InboundInEvent = crate::substream_handler::InEvent<(), inbound::InEvent, Void>; +pub type InboundOutEvent = crate::substream_handler::OutEvent; diff --git a/protocols/rendezvous/src/handler/inbound.rs b/protocols/rendezvous/src/handler/inbound.rs new file mode 100644 index 00000000000..8a18f366c68 --- /dev/null +++ b/protocols/rendezvous/src/handler/inbound.rs @@ -0,0 +1,189 @@ +// Copyright 2021 COMIT Network. +// +// 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::codec::{ + Cookie, ErrorCode, Message, Namespace, NewRegistration, Registration, RendezvousCodec, Ttl, +}; +use crate::handler::Error; +use crate::handler::PROTOCOL_IDENT; +use crate::substream_handler::{Next, PassthroughProtocol, SubstreamHandler}; +use asynchronous_codec::Framed; +use futures::{SinkExt, StreamExt}; +use libp2p_swarm::{NegotiatedSubstream, SubstreamProtocol}; +use std::fmt; +use std::task::{Context, Poll}; + +/// The state of an inbound substream (i.e. the remote node opened it). +#[allow(clippy::large_enum_variant)] +pub enum Stream { + /// We are in the process of reading a message from the substream. + PendingRead(Framed), + /// We read a message, dispatched it to the behaviour and are waiting for the response. + PendingBehaviour(Framed), + /// We are in the process of sending a response. + PendingSend(Framed, Message), + /// We've sent the message and are now closing down the substream. + PendingClose(Framed), +} + +impl fmt::Debug for Stream { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Stream::PendingRead(_) => write!(f, "Inbound::PendingRead"), + Stream::PendingBehaviour(_) => write!(f, "Inbound::PendingBehaviour"), + Stream::PendingSend(_, _) => write!(f, "Inbound::PendingSend"), + Stream::PendingClose(_) => write!(f, "Inbound::PendingClose"), + } + } +} + +#[derive(Debug, Clone)] +pub enum OutEvent { + RegistrationRequested(NewRegistration), + UnregisterRequested(Namespace), + DiscoverRequested { + namespace: Option, + cookie: Option, + limit: Option, + }, +} + +#[derive(Debug)] +pub enum InEvent { + RegisterResponse { + ttl: Ttl, + }, + DeclineRegisterRequest(ErrorCode), + DiscoverResponse { + discovered: Vec, + cookie: Cookie, + }, + DeclineDiscoverRequest(ErrorCode), +} + +impl SubstreamHandler for Stream { + type InEvent = InEvent; + type OutEvent = OutEvent; + type Error = Error; + type OpenInfo = (); + + fn upgrade( + open_info: Self::OpenInfo, + ) -> SubstreamProtocol { + SubstreamProtocol::new(PassthroughProtocol::new(PROTOCOL_IDENT), open_info) + } + + fn new(substream: NegotiatedSubstream, _: Self::OpenInfo) -> Self { + Stream::PendingRead(Framed::new(substream, RendezvousCodec::default())) + } + + fn inject_event(self, event: Self::InEvent) -> Self { + match (event, self) { + (InEvent::RegisterResponse { ttl }, Stream::PendingBehaviour(substream)) => { + Stream::PendingSend(substream, Message::RegisterResponse(Ok(ttl))) + } + (InEvent::DeclineRegisterRequest(error), Stream::PendingBehaviour(substream)) => { + Stream::PendingSend(substream, Message::RegisterResponse(Err(error))) + } + ( + InEvent::DiscoverResponse { discovered, cookie }, + Stream::PendingBehaviour(substream), + ) => Stream::PendingSend( + substream, + Message::DiscoverResponse(Ok((discovered, cookie))), + ), + (InEvent::DeclineDiscoverRequest(error), Stream::PendingBehaviour(substream)) => { + Stream::PendingSend(substream, Message::DiscoverResponse(Err(error))) + } + (event, inbound) => { + debug_assert!(false, "{:?} cannot handle event {:?}", inbound, event); + + inbound + } + } + } + + fn advance(self, cx: &mut Context<'_>) -> Result, Self::Error> { + let next_state = match self { + Stream::PendingRead(mut substream) => { + match substream.poll_next_unpin(cx).map_err(Error::ReadMessage)? { + Poll::Ready(Some(msg)) => { + let event = match msg { + Message::Register(registration) => { + OutEvent::RegistrationRequested(registration) + } + Message::Discover { + cookie, + namespace, + limit, + } => OutEvent::DiscoverRequested { + cookie, + namespace, + limit, + }, + Message::Unregister(namespace) => { + OutEvent::UnregisterRequested(namespace) + } + other => return Err(Error::BadMessage(other)), + }; + + Next::EmitEvent { + event, + next_state: Stream::PendingBehaviour(substream), + } + } + Poll::Ready(None) => return Err(Error::UnexpectedEndOfStream), + Poll::Pending => Next::Pending { + next_state: Stream::PendingRead(substream), + }, + } + } + Stream::PendingBehaviour(substream) => Next::Pending { + next_state: Stream::PendingBehaviour(substream), + }, + Stream::PendingSend(mut substream, message) => match substream + .poll_ready_unpin(cx) + .map_err(Error::WriteMessage)? + { + Poll::Ready(()) => { + substream + .start_send_unpin(message) + .map_err(Error::WriteMessage)?; + + Next::Continue { + next_state: Stream::PendingClose(substream), + } + } + Poll::Pending => Next::Pending { + next_state: Stream::PendingSend(substream, message), + }, + }, + Stream::PendingClose(mut substream) => match substream.poll_close_unpin(cx) { + Poll::Ready(Ok(())) => Next::Done, + Poll::Ready(Err(_)) => Next::Done, // there is nothing we can do about an error during close + Poll::Pending => Next::Pending { + next_state: Stream::PendingClose(substream), + }, + }, + }; + + Ok(next_state) + } +} diff --git a/protocols/rendezvous/src/handler/outbound.rs b/protocols/rendezvous/src/handler/outbound.rs new file mode 100644 index 00000000000..ab06040ca19 --- /dev/null +++ b/protocols/rendezvous/src/handler/outbound.rs @@ -0,0 +1,132 @@ +// Copyright 2021 COMIT Network. +// +// 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::codec::{Cookie, Message, NewRegistration, RendezvousCodec}; +use crate::handler::Error; +use crate::handler::PROTOCOL_IDENT; +use crate::substream_handler::{FutureSubstream, Next, PassthroughProtocol, SubstreamHandler}; +use crate::{ErrorCode, Namespace, Registration, Ttl}; +use asynchronous_codec::Framed; +use futures::{SinkExt, TryFutureExt, TryStreamExt}; +use libp2p_swarm::{NegotiatedSubstream, SubstreamProtocol}; +use std::task::Context; +use void::Void; + +pub struct Stream(FutureSubstream); + +impl SubstreamHandler for Stream { + type InEvent = Void; + type OutEvent = OutEvent; + type Error = Error; + type OpenInfo = OpenInfo; + + fn upgrade( + open_info: Self::OpenInfo, + ) -> SubstreamProtocol { + SubstreamProtocol::new(PassthroughProtocol::new(PROTOCOL_IDENT), open_info) + } + + fn new(substream: NegotiatedSubstream, info: Self::OpenInfo) -> Self { + let mut stream = Framed::new(substream, RendezvousCodec::default()); + let sent_message = match info { + OpenInfo::RegisterRequest(new_registration) => Message::Register(new_registration), + OpenInfo::UnregisterRequest(namespace) => Message::Unregister(namespace), + OpenInfo::DiscoverRequest { + namespace, + cookie, + limit, + } => Message::Discover { + namespace, + cookie, + limit, + }, + }; + + Self(FutureSubstream::new(async move { + use Message::*; + use OutEvent::*; + + stream + .send(sent_message.clone()) + .map_err(Error::WriteMessage) + .await?; + let received_message = stream.try_next().map_err(Error::ReadMessage).await?; + let received_message = received_message.ok_or(Error::UnexpectedEndOfStream)?; + + let event = match (sent_message, received_message) { + (Register(registration), RegisterResponse(Ok(ttl))) => Registered { + namespace: registration.namespace, + ttl, + }, + (Register(registration), RegisterResponse(Err(error))) => { + RegisterFailed(registration.namespace, error) + } + (Discover { .. }, DiscoverResponse(Ok((registrations, cookie)))) => Discovered { + registrations, + cookie, + }, + (Discover { namespace, .. }, DiscoverResponse(Err(error))) => { + DiscoverFailed { namespace, error } + } + (.., other) => return Err(Error::BadMessage(other)), + }; + + stream.close().map_err(Error::WriteMessage).await?; + + Ok(event) + })) + } + + fn inject_event(self, event: Self::InEvent) -> Self { + void::unreachable(event) + } + + fn advance(self, cx: &mut Context<'_>) -> Result, Self::Error> { + Ok(self.0.advance(cx)?.map_state(Stream)) + } +} + +#[derive(Debug, Clone)] +pub enum OutEvent { + Registered { + namespace: Namespace, + ttl: Ttl, + }, + RegisterFailed(Namespace, ErrorCode), + Discovered { + registrations: Vec, + cookie: Cookie, + }, + DiscoverFailed { + namespace: Option, + error: ErrorCode, + }, +} + +#[derive(Debug)] +pub enum OpenInfo { + RegisterRequest(NewRegistration), + UnregisterRequest(Namespace), + DiscoverRequest { + namespace: Option, + cookie: Option, + limit: Option, + }, +} diff --git a/protocols/rendezvous/src/lib.rs b/protocols/rendezvous/src/lib.rs new file mode 100644 index 00000000000..87c88434db3 --- /dev/null +++ b/protocols/rendezvous/src/lib.rs @@ -0,0 +1,43 @@ +// Copyright 2021 COMIT Network. +// +// 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. + +pub use self::codec::{ErrorCode, Namespace, NamespaceTooLong, Registration, Ttl}; + +mod codec; +mod handler; +mod substream_handler; + +/// If unspecified, rendezvous nodes should assume a TTL of 2h. +/// +/// See . +pub const DEFAULT_TTL: Ttl = 60 * 60 * 2; + +/// By default, nodes should require a minimum TTL of 2h +/// +/// . +pub const MIN_TTL: Ttl = 60 * 60 * 2; + +/// By default, nodes should allow a maximum TTL of 72h +/// +/// . +pub const MAX_TTL: Ttl = 60 * 60 * 72; + +pub mod client; +pub mod server; diff --git a/protocols/rendezvous/src/rpc.proto b/protocols/rendezvous/src/rpc.proto new file mode 100644 index 00000000000..d4e388ca96e --- /dev/null +++ b/protocols/rendezvous/src/rpc.proto @@ -0,0 +1,61 @@ +syntax = "proto2"; + +package rendezvous.pb; + +message Message { + enum MessageType { + REGISTER = 0; + REGISTER_RESPONSE = 1; + UNREGISTER = 2; + DISCOVER = 3; + DISCOVER_RESPONSE = 4; + } + + enum ResponseStatus { + OK = 0; + E_INVALID_NAMESPACE = 100; + E_INVALID_SIGNED_PEER_RECORD = 101; + E_INVALID_TTL = 102; + E_INVALID_COOKIE = 103; + E_NOT_AUTHORIZED = 200; + E_INTERNAL_ERROR = 300; + E_UNAVAILABLE = 400; + } + + message Register { + optional string ns = 1; + optional bytes signedPeerRecord = 2; + optional uint64 ttl = 3; // in seconds + } + + message RegisterResponse { + optional ResponseStatus status = 1; + optional string statusText = 2; + optional uint64 ttl = 3; // in seconds + } + + message Unregister { + optional string ns = 1; + optional bytes id = 2; + } + + message Discover { + optional string ns = 1; + optional uint64 limit = 2; + optional bytes cookie = 3; + } + + message DiscoverResponse { + repeated Register registrations = 1; + optional bytes cookie = 2; + optional ResponseStatus status = 3; + optional string statusText = 4; + } + + optional MessageType type = 1; + optional Register register = 2; + optional RegisterResponse registerResponse = 3; + optional Unregister unregister = 4; + optional Discover discover = 5; + optional DiscoverResponse discoverResponse = 6; +} diff --git a/protocols/rendezvous/src/server.rs b/protocols/rendezvous/src/server.rs new file mode 100644 index 00000000000..93682dda422 --- /dev/null +++ b/protocols/rendezvous/src/server.rs @@ -0,0 +1,764 @@ +// Copyright 2021 COMIT Network. +// +// 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::codec::{Cookie, ErrorCode, Namespace, NewRegistration, Registration, Ttl}; +use crate::handler::inbound; +use crate::substream_handler::{InboundSubstreamId, SubstreamProtocolsHandler}; +use crate::{handler, MAX_TTL, MIN_TTL}; +use bimap::BiMap; +use futures::future::BoxFuture; +use futures::ready; +use futures::stream::FuturesUnordered; +use futures::{FutureExt, StreamExt}; +use libp2p_core::connection::ConnectionId; +use libp2p_core::PeerId; +use libp2p_swarm::{ + CloseConnection, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, +}; +use std::collections::{HashMap, HashSet, VecDeque}; +use std::iter::FromIterator; +use std::task::{Context, Poll}; +use std::time::Duration; +use void::Void; + +pub struct Behaviour { + events: VecDeque< + NetworkBehaviourAction>, + >, + registrations: Registrations, +} + +pub struct Config { + min_ttl: Ttl, + max_ttl: Ttl, +} + +impl Config { + pub fn with_min_ttl(mut self, min_ttl: Ttl) -> Self { + self.min_ttl = min_ttl; + self + } + + pub fn with_max_ttl(mut self, max_ttl: Ttl) -> Self { + self.max_ttl = max_ttl; + self + } +} + +impl Default for Config { + fn default() -> Self { + Self { + min_ttl: MIN_TTL, + max_ttl: MAX_TTL, + } + } +} + +impl Behaviour { + /// Create a new instance of the rendezvous [`NetworkBehaviour`]. + pub fn new(config: Config) -> Self { + Self { + events: Default::default(), + registrations: Registrations::with_config(config), + } + } +} + +#[derive(Debug)] +#[allow(clippy::large_enum_variant)] +pub enum Event { + /// We successfully served a discover request from a peer. + DiscoverServed { + enquirer: PeerId, + registrations: Vec, + }, + /// We failed to serve a discover request for a peer. + DiscoverNotServed { enquirer: PeerId, error: ErrorCode }, + /// A peer successfully registered with us. + PeerRegistered { + peer: PeerId, + registration: Registration, + }, + /// We declined a registration from a peer. + PeerNotRegistered { + peer: PeerId, + namespace: Namespace, + error: ErrorCode, + }, + /// A peer successfully unregistered with us. + PeerUnregistered { peer: PeerId, namespace: Namespace }, + /// A registration from a peer expired. + RegistrationExpired(Registration), +} + +impl NetworkBehaviour for Behaviour { + type ProtocolsHandler = SubstreamProtocolsHandler; + type OutEvent = Event; + + fn new_handler(&mut self) -> Self::ProtocolsHandler { + let initial_keep_alive = Duration::from_secs(30); + + SubstreamProtocolsHandler::new_inbound_only(initial_keep_alive) + } + + fn inject_event( + &mut self, + peer_id: PeerId, + connection: ConnectionId, + event: handler::InboundOutEvent, + ) { + let new_events = match event { + handler::InboundOutEvent::InboundEvent { id, message } => { + handle_inbound_event(message, peer_id, connection, id, &mut self.registrations) + } + handler::InboundOutEvent::OutboundEvent { message, .. } => void::unreachable(message), + handler::InboundOutEvent::InboundError { error, .. } => { + log::warn!("Connection with peer {} failed: {}", peer_id, error); + + vec![NetworkBehaviourAction::CloseConnection { + peer_id, + connection: CloseConnection::One(connection), + }] + } + handler::InboundOutEvent::OutboundError { error, .. } => void::unreachable(error), + }; + + self.events.extend(new_events); + } + + fn poll( + &mut self, + cx: &mut Context<'_>, + _: &mut impl PollParameters, + ) -> Poll> { + if let Poll::Ready(ExpiredRegistration(registration)) = self.registrations.poll(cx) { + return Poll::Ready(NetworkBehaviourAction::GenerateEvent( + Event::RegistrationExpired(registration), + )); + } + + if let Some(event) = self.events.pop_front() { + return Poll::Ready(event); + } + + Poll::Pending + } +} + +fn handle_inbound_event( + event: inbound::OutEvent, + peer_id: PeerId, + connection: ConnectionId, + id: InboundSubstreamId, + registrations: &mut Registrations, +) -> Vec>> { + match event { + // bad registration + inbound::OutEvent::RegistrationRequested(registration) + if registration.record.peer_id() != peer_id => + { + let error = ErrorCode::NotAuthorized; + + vec![ + NetworkBehaviourAction::NotifyHandler { + peer_id, + handler: NotifyHandler::One(connection), + event: handler::InboundInEvent::NotifyInboundSubstream { + id, + message: inbound::InEvent::DeclineRegisterRequest(error), + }, + }, + NetworkBehaviourAction::GenerateEvent(Event::PeerNotRegistered { + peer: peer_id, + namespace: registration.namespace, + error, + }), + ] + } + inbound::OutEvent::RegistrationRequested(registration) => { + let namespace = registration.namespace.clone(); + + match registrations.add(registration) { + Ok(registration) => { + vec![ + NetworkBehaviourAction::NotifyHandler { + peer_id, + handler: NotifyHandler::One(connection), + event: handler::InboundInEvent::NotifyInboundSubstream { + id, + message: inbound::InEvent::RegisterResponse { + ttl: registration.ttl, + }, + }, + }, + NetworkBehaviourAction::GenerateEvent(Event::PeerRegistered { + peer: peer_id, + registration, + }), + ] + } + Err(TtlOutOfRange::TooLong { .. }) | Err(TtlOutOfRange::TooShort { .. }) => { + let error = ErrorCode::InvalidTtl; + + vec![ + NetworkBehaviourAction::NotifyHandler { + peer_id, + handler: NotifyHandler::One(connection), + event: handler::InboundInEvent::NotifyInboundSubstream { + id, + message: inbound::InEvent::DeclineRegisterRequest(error), + }, + }, + NetworkBehaviourAction::GenerateEvent(Event::PeerNotRegistered { + peer: peer_id, + namespace, + error, + }), + ] + } + } + } + inbound::OutEvent::DiscoverRequested { + namespace, + cookie, + limit, + } => match registrations.get(namespace, cookie, limit) { + Ok((registrations, cookie)) => { + let discovered = registrations.cloned().collect::>(); + + vec![ + NetworkBehaviourAction::NotifyHandler { + peer_id, + handler: NotifyHandler::One(connection), + event: handler::InboundInEvent::NotifyInboundSubstream { + id, + message: inbound::InEvent::DiscoverResponse { + discovered: discovered.clone(), + cookie, + }, + }, + }, + NetworkBehaviourAction::GenerateEvent(Event::DiscoverServed { + enquirer: peer_id, + registrations: discovered, + }), + ] + } + Err(_) => { + let error = ErrorCode::InvalidCookie; + + vec![ + NetworkBehaviourAction::NotifyHandler { + peer_id, + handler: NotifyHandler::One(connection), + event: handler::InboundInEvent::NotifyInboundSubstream { + id, + message: inbound::InEvent::DeclineDiscoverRequest(error), + }, + }, + NetworkBehaviourAction::GenerateEvent(Event::DiscoverNotServed { + enquirer: peer_id, + error, + }), + ] + } + }, + inbound::OutEvent::UnregisterRequested(namespace) => { + registrations.remove(namespace.clone(), peer_id); + + vec![NetworkBehaviourAction::GenerateEvent( + Event::PeerUnregistered { + peer: peer_id, + namespace, + }, + )] + } + } +} + +#[derive(Debug, Eq, PartialEq, Hash, Copy, Clone)] +struct RegistrationId(u64); + +impl RegistrationId { + fn new() -> Self { + Self(rand::random()) + } +} + +#[derive(Debug, PartialEq)] +struct ExpiredRegistration(Registration); + +pub struct Registrations { + registrations_for_peer: BiMap<(PeerId, Namespace), RegistrationId>, + registrations: HashMap, + cookies: HashMap>, + min_ttl: Ttl, + max_ttl: Ttl, + next_expiry: FuturesUnordered>, +} + +#[derive(Debug, thiserror::Error)] +pub enum TtlOutOfRange { + #[error("Requested TTL ({requested}s) is too long; max {bound}s")] + TooLong { bound: Ttl, requested: Ttl }, + #[error("Requested TTL ({requested}s) is too short; min {bound}s")] + TooShort { bound: Ttl, requested: Ttl }, +} + +impl Default for Registrations { + fn default() -> Self { + Registrations::with_config(Config::default()) + } +} + +impl Registrations { + pub fn with_config(config: Config) -> Self { + Self { + registrations_for_peer: Default::default(), + registrations: Default::default(), + min_ttl: config.min_ttl, + max_ttl: config.max_ttl, + cookies: Default::default(), + next_expiry: FuturesUnordered::from_iter(vec![futures::future::pending().boxed()]), + } + } + + pub fn add( + &mut self, + new_registration: NewRegistration, + ) -> Result { + let ttl = new_registration.effective_ttl(); + if ttl > self.max_ttl { + return Err(TtlOutOfRange::TooLong { + bound: self.max_ttl, + requested: ttl, + }); + } + if ttl < self.min_ttl { + return Err(TtlOutOfRange::TooShort { + bound: self.min_ttl, + requested: ttl, + }); + } + + let namespace = new_registration.namespace; + let registration_id = RegistrationId::new(); + + if let Some(old_registration) = self + .registrations_for_peer + .get_by_left(&(new_registration.record.peer_id(), namespace.clone())) + { + self.registrations.remove(old_registration); + } + + self.registrations_for_peer.insert( + (new_registration.record.peer_id(), namespace.clone()), + registration_id, + ); + + let registration = Registration { + namespace, + record: new_registration.record, + ttl, + }; + self.registrations + .insert(registration_id, registration.clone()); + + let next_expiry = wasm_timer::Delay::new(Duration::from_secs(ttl as u64)) + .map(move |result| { + if result.is_err() { + log::warn!("Timer for registration {} has unexpectedly errored, treating it as expired", registration_id.0); + } + + registration_id + }) + .boxed(); + + self.next_expiry.push(next_expiry); + + Ok(registration) + } + + pub fn remove(&mut self, namespace: Namespace, peer_id: PeerId) { + let reggo_to_remove = self + .registrations_for_peer + .remove_by_left(&(peer_id, namespace)); + + if let Some((_, reggo_to_remove)) = reggo_to_remove { + self.registrations.remove(®go_to_remove); + } + } + + pub fn get( + &mut self, + discover_namespace: Option, + cookie: Option, + limit: Option, + ) -> Result<(impl Iterator + '_, Cookie), CookieNamespaceMismatch> { + let cookie_namespace = cookie.as_ref().and_then(|cookie| cookie.namespace()); + + match (discover_namespace.as_ref(), cookie_namespace) { + // discover all namespace but cookie is specific to a namespace? => bad + (None, Some(_)) => return Err(CookieNamespaceMismatch), + // discover for a namespace but cookie is for a different namesapce? => bad + (Some(namespace), Some(cookie_namespace)) if namespace != cookie_namespace => { + return Err(CookieNamespaceMismatch) + } + // every other combination is fine + _ => {} + } + + let mut reggos_of_last_discover = cookie + .and_then(|cookie| self.cookies.get(&cookie)) + .cloned() + .unwrap_or_default(); + + let ids = self + .registrations_for_peer + .iter() + .filter_map({ + |((_, namespace), registration_id)| { + if reggos_of_last_discover.contains(registration_id) { + return None; + } + + match discover_namespace.as_ref() { + Some(discover_namespace) if discover_namespace == namespace => { + Some(registration_id) + } + Some(_) => None, + None => Some(registration_id), + } + } + }) + .take(limit.unwrap_or(u64::MAX) as usize) + .cloned() + .collect::>(); + + reggos_of_last_discover.extend(&ids); + + let new_cookie = discover_namespace + .map(Cookie::for_namespace) + .unwrap_or_else(Cookie::for_all_namespaces); + self.cookies + .insert(new_cookie.clone(), reggos_of_last_discover); + + let reggos = &self.registrations; + let registrations = ids + .into_iter() + .map(move |id| reggos.get(&id).expect("bad internal datastructure")); + + Ok((registrations, new_cookie)) + } + + fn poll(&mut self, cx: &mut Context<'_>) -> Poll { + let expired_registration = ready!(self.next_expiry.poll_next_unpin(cx)).expect( + "This stream should never finish because it is initialised with a pending future", + ); + + // clean up our cookies + self.cookies.retain(|_, registrations| { + registrations.remove(&expired_registration); + + // retain all cookies where there are still registrations left + !registrations.is_empty() + }); + + self.registrations_for_peer + .remove_by_right(&expired_registration); + match self.registrations.remove(&expired_registration) { + None => self.poll(cx), + Some(registration) => Poll::Ready(ExpiredRegistration(registration)), + } + } +} + +#[derive(Debug, thiserror::Error, Eq, PartialEq)] +#[error("The provided cookie is not valid for a DISCOVER request for the given namespace")] +pub struct CookieNamespaceMismatch; + +#[cfg(test)] +mod tests { + use std::option::Option::None; + use std::time::SystemTime; + + use libp2p_core::{identity, PeerRecord}; + + use super::*; + + #[test] + fn given_cookie_from_discover_when_discover_again_then_only_get_diff() { + let mut registrations = Registrations::default(); + registrations.add(new_dummy_registration("foo")).unwrap(); + registrations.add(new_dummy_registration("foo")).unwrap(); + + let (initial_discover, cookie) = registrations.get(None, None, None).unwrap(); + assert_eq!(initial_discover.count(), 2); + + let (subsequent_discover, _) = registrations.get(None, Some(cookie), None).unwrap(); + assert_eq!(subsequent_discover.count(), 0); + } + + #[test] + fn given_registrations_when_discover_all_then_all_are_returned() { + let mut registrations = Registrations::default(); + registrations.add(new_dummy_registration("foo")).unwrap(); + registrations.add(new_dummy_registration("foo")).unwrap(); + + let (discover, _) = registrations.get(None, None, None).unwrap(); + + assert_eq!(discover.count(), 2); + } + + #[test] + fn given_registrations_when_discover_only_for_specific_namespace_then_only_those_are_returned() + { + let mut registrations = Registrations::default(); + registrations.add(new_dummy_registration("foo")).unwrap(); + registrations.add(new_dummy_registration("bar")).unwrap(); + + let (discover, _) = registrations + .get(Some(Namespace::from_static("foo")), None, None) + .unwrap(); + + assert_eq!( + discover.map(|r| &r.namespace).collect::>(), + vec!["foo"] + ); + } + + #[test] + fn given_reregistration_old_registration_is_discarded() { + let alice = identity::Keypair::generate_ed25519(); + let mut registrations = Registrations::default(); + registrations + .add(new_registration("foo", alice.clone(), None)) + .unwrap(); + registrations + .add(new_registration("foo", alice, None)) + .unwrap(); + + let (discover, _) = registrations + .get(Some(Namespace::from_static("foo")), None, None) + .unwrap(); + + assert_eq!( + discover.map(|r| &r.namespace).collect::>(), + vec!["foo"] + ); + } + + #[test] + fn given_cookie_from_2nd_discover_does_not_return_nodes_from_first_discover() { + let mut registrations = Registrations::default(); + registrations.add(new_dummy_registration("foo")).unwrap(); + registrations.add(new_dummy_registration("foo")).unwrap(); + + let (initial_discover, cookie1) = registrations.get(None, None, None).unwrap(); + assert_eq!(initial_discover.count(), 2); + + let (subsequent_discover, cookie2) = registrations.get(None, Some(cookie1), None).unwrap(); + assert_eq!(subsequent_discover.count(), 0); + + let (subsequent_discover, _) = registrations.get(None, Some(cookie2), None).unwrap(); + assert_eq!(subsequent_discover.count(), 0); + } + + #[test] + fn cookie_from_different_discover_request_is_not_valid() { + let mut registrations = Registrations::default(); + registrations.add(new_dummy_registration("foo")).unwrap(); + registrations.add(new_dummy_registration("bar")).unwrap(); + + let (_, foo_discover_cookie) = registrations + .get(Some(Namespace::from_static("foo")), None, None) + .unwrap(); + let result = registrations.get( + Some(Namespace::from_static("bar")), + Some(foo_discover_cookie), + None, + ); + + assert!(matches!(result, Err(CookieNamespaceMismatch))) + } + + #[tokio::test] + async fn given_two_registration_ttls_one_expires_one_lives() { + let mut registrations = Registrations::with_config(Config { + min_ttl: 0, + max_ttl: 4, + }); + + let start_time = SystemTime::now(); + + registrations + .add(new_dummy_registration_with_ttl("foo", 1)) + .unwrap(); + registrations + .add(new_dummy_registration_with_ttl("bar", 4)) + .unwrap(); + + let event = registrations.next_event().await; + + let elapsed = start_time.elapsed().unwrap(); + assert!(elapsed.as_secs() >= 1); + assert!(elapsed.as_secs() < 2); + + assert_eq!(event.0.namespace, Namespace::from_static("foo")); + + { + let (mut discovered_foo, _) = registrations + .get(Some(Namespace::from_static("foo")), None, None) + .unwrap(); + assert!(discovered_foo.next().is_none()); + } + let (mut discovered_bar, _) = registrations + .get(Some(Namespace::from_static("bar")), None, None) + .unwrap(); + assert!(discovered_bar.next().is_some()); + } + + #[tokio::test] + async fn given_peer_unregisters_before_expiry_do_not_emit_registration_expired() { + let mut registrations = Registrations::with_config(Config { + min_ttl: 1, + max_ttl: 10, + }); + let dummy_registration = new_dummy_registration_with_ttl("foo", 2); + let namespace = dummy_registration.namespace.clone(); + let peer_id = dummy_registration.record.peer_id(); + + registrations.add(dummy_registration).unwrap(); + registrations.no_event_for(1).await; + registrations.remove(namespace, peer_id); + + registrations.no_event_for(3).await + } + + /// FuturesUnordered stop polling for ready futures when poll_next() is called until a None + /// value is returned. To prevent the next_expiry future from going to "sleep", next_expiry + /// is initialised with a future that always returns pending. This test ensures that + /// FuturesUnordered does not stop polling for ready futures. + #[tokio::test] + async fn given_all_registrations_expired_then_successfully_handle_new_registration_and_expiry() + { + let mut registrations = Registrations::with_config(Config { + min_ttl: 0, + max_ttl: 10, + }); + let dummy_registration = new_dummy_registration_with_ttl("foo", 1); + + registrations.add(dummy_registration.clone()).unwrap(); + let _ = registrations.next_event_in_at_most(2).await; + + registrations.no_event_for(1).await; + + registrations.add(dummy_registration).unwrap(); + let _ = registrations.next_event_in_at_most(2).await; + } + + #[tokio::test] + async fn cookies_are_cleaned_up_if_registrations_expire() { + let mut registrations = Registrations::with_config(Config { + min_ttl: 1, + max_ttl: 10, + }); + + registrations + .add(new_dummy_registration_with_ttl("foo", 2)) + .unwrap(); + let (_, _) = registrations.get(None, None, None).unwrap(); + + assert_eq!(registrations.cookies.len(), 1); + + let _ = registrations.next_event_in_at_most(3).await; + + assert_eq!(registrations.cookies.len(), 0); + } + + #[test] + fn given_limit_discover_only_returns_n_results() { + let mut registrations = Registrations::default(); + registrations.add(new_dummy_registration("foo")).unwrap(); + registrations.add(new_dummy_registration("foo")).unwrap(); + + let (registrations, _) = registrations.get(None, None, Some(1)).unwrap(); + + assert_eq!(registrations.count(), 1); + } + + #[test] + fn given_limit_cookie_can_be_used_for_pagination() { + let mut registrations = Registrations::default(); + registrations.add(new_dummy_registration("foo")).unwrap(); + registrations.add(new_dummy_registration("foo")).unwrap(); + + let (discover1, cookie) = registrations.get(None, None, Some(1)).unwrap(); + assert_eq!(discover1.count(), 1); + + let (discover2, _) = registrations.get(None, Some(cookie), None).unwrap(); + assert_eq!(discover2.count(), 1); + } + + fn new_dummy_registration(namespace: &'static str) -> NewRegistration { + let identity = identity::Keypair::generate_ed25519(); + + new_registration(namespace, identity, None) + } + + fn new_dummy_registration_with_ttl(namespace: &'static str, ttl: Ttl) -> NewRegistration { + let identity = identity::Keypair::generate_ed25519(); + + new_registration(namespace, identity, Some(ttl)) + } + + fn new_registration( + namespace: &'static str, + identity: identity::Keypair, + ttl: Option, + ) -> NewRegistration { + NewRegistration::new( + Namespace::from_static(namespace), + PeerRecord::new(identity, vec!["/ip4/127.0.0.1/tcp/1234".parse().unwrap()]).unwrap(), + ttl, + ) + } + + /// Defines utility functions that make the tests more readable. + impl Registrations { + async fn next_event(&mut self) -> ExpiredRegistration { + futures::future::poll_fn(|cx| self.poll(cx)).await + } + + /// Polls [`Registrations`] for `seconds` and panics if it returns a event during this time. + async fn no_event_for(&mut self, seconds: u64) { + tokio::time::timeout(Duration::from_secs(seconds), self.next_event()) + .await + .unwrap_err(); + } + + /// Polls [`Registrations`] for at most `seconds` and panics if doesn't return an event within that time. + async fn next_event_in_at_most(&mut self, seconds: u64) -> ExpiredRegistration { + tokio::time::timeout(Duration::from_secs(seconds), self.next_event()) + .await + .unwrap() + } + } +} diff --git a/protocols/rendezvous/src/substream_handler.rs b/protocols/rendezvous/src/substream_handler.rs new file mode 100644 index 00000000000..efd7956b13c --- /dev/null +++ b/protocols/rendezvous/src/substream_handler.rs @@ -0,0 +1,551 @@ +// Copyright 2021 COMIT Network. +// +// 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. + +//! A generic [`ProtocolsHandler`] that delegates the handling of substreams to [`SubstreamHandler`]s. +//! +//! This module is an attempt to simplify the implementation of protocols by freeing implementations from dealing with aspects such as concurrent substreams. +//! Particularly for outbound substreams, it greatly simplifies the definition of protocols through the [`FutureSubstream`] helper. +//! +//! At the moment, this module is an implementation detail of the rendezvous protocol but the intent is for it to be provided as a generic module that is accessible to other protocols as well. + +use futures::future::{self, BoxFuture, Fuse, FusedFuture}; +use futures::FutureExt; +use libp2p_core::{InboundUpgrade, OutboundUpgrade, UpgradeInfo}; +use libp2p_swarm::protocols_handler::{InboundUpgradeSend, OutboundUpgradeSend}; +use libp2p_swarm::{ + KeepAlive, NegotiatedSubstream, ProtocolsHandler, ProtocolsHandlerEvent, + ProtocolsHandlerUpgrErr, SubstreamProtocol, +}; +use std::collections::{HashMap, VecDeque}; +use std::fmt; +use std::future::Future; +use std::hash::Hash; +use std::task::{Context, Poll}; +use std::time::{Duration, Instant}; +use void::Void; + +/// Handles a substream throughout its lifetime. +pub trait SubstreamHandler: Sized { + type InEvent; + type OutEvent; + type Error; + type OpenInfo; + + fn upgrade(open_info: Self::OpenInfo) + -> SubstreamProtocol; + fn new(substream: NegotiatedSubstream, info: Self::OpenInfo) -> Self; + fn inject_event(self, event: Self::InEvent) -> Self; + fn advance(self, cx: &mut Context<'_>) -> Result, Self::Error>; +} + +/// The result of advancing a [`SubstreamHandler`]. +pub enum Next { + /// Return the given event and set the handler into `next_state`. + EmitEvent { event: TEvent, next_state: TState }, + /// The handler currently cannot do any more work, set its state back into `next_state`. + Pending { next_state: TState }, + /// The handler performed some work and wants to continue in the given state. + /// + /// This variant is useful because it frees the handler from implementing a loop internally. + Continue { next_state: TState }, + /// The handler finished. + Done, +} + +impl Next { + pub fn map_state( + self, + map: impl FnOnce(TState) -> TNextState, + ) -> Next { + match self { + Next::EmitEvent { event, next_state } => Next::EmitEvent { + event, + next_state: map(next_state), + }, + Next::Pending { next_state } => Next::Pending { + next_state: map(next_state), + }, + Next::Continue { next_state } => Next::Pending { + next_state: map(next_state), + }, + Next::Done => Next::Done, + } + } +} + +#[derive(Debug, Hash, Eq, PartialEq, Clone, Copy)] +pub struct InboundSubstreamId(u64); + +impl InboundSubstreamId { + fn fetch_and_increment(&mut self) -> Self { + let next_id = *self; + self.0 += 1; + + next_id + } +} + +impl fmt::Display for InboundSubstreamId { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.0) + } +} + +#[derive(Debug, Hash, Eq, PartialEq, Clone, Copy)] +pub struct OutboundSubstreamId(u64); + +impl OutboundSubstreamId { + fn fetch_and_increment(&mut self) -> Self { + let next_id = *self; + self.0 += 1; + + next_id + } +} + +impl fmt::Display for OutboundSubstreamId { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.0) + } +} + +pub struct PassthroughProtocol { + ident: Option<&'static [u8]>, +} + +impl PassthroughProtocol { + pub fn new(ident: &'static [u8]) -> Self { + Self { ident: Some(ident) } + } +} + +impl UpgradeInfo for PassthroughProtocol { + type Info = &'static [u8]; + type InfoIter = std::option::IntoIter; + + fn protocol_info(&self) -> Self::InfoIter { + self.ident.into_iter() + } +} + +impl InboundUpgrade for PassthroughProtocol { + type Output = C; + type Error = Void; + type Future = BoxFuture<'static, Result>; + + fn upgrade_inbound(self, socket: C, _: Self::Info) -> Self::Future { + match self.ident { + Some(_) => future::ready(Ok(socket)).boxed(), + None => future::pending().boxed(), + } + } +} + +impl OutboundUpgrade for PassthroughProtocol { + type Output = C; + type Error = Void; + type Future = BoxFuture<'static, Result>; + + fn upgrade_outbound(self, socket: C, _: Self::Info) -> Self::Future { + match self.ident { + Some(_) => future::ready(Ok(socket)).boxed(), + None => future::pending().boxed(), + } + } +} + +/// An implementation of [`ProtocolsHandler`] that delegates to individual [`SubstreamHandler`]s. +pub struct SubstreamProtocolsHandler { + inbound_substreams: HashMap, + outbound_substreams: HashMap, + next_inbound_substream_id: InboundSubstreamId, + next_outbound_substream_id: OutboundSubstreamId, + + new_substreams: VecDeque, + + initial_keep_alive_deadline: Instant, +} + +impl + SubstreamProtocolsHandler +{ + pub fn new(initial_keep_alive: Duration) -> Self { + Self { + inbound_substreams: Default::default(), + outbound_substreams: Default::default(), + next_inbound_substream_id: InboundSubstreamId(0), + next_outbound_substream_id: OutboundSubstreamId(0), + new_substreams: Default::default(), + initial_keep_alive_deadline: Instant::now() + initial_keep_alive, + } + } +} + +impl + SubstreamProtocolsHandler +{ + pub fn new_outbound_only(initial_keep_alive: Duration) -> Self { + Self { + inbound_substreams: Default::default(), + outbound_substreams: Default::default(), + next_inbound_substream_id: InboundSubstreamId(0), + next_outbound_substream_id: OutboundSubstreamId(0), + new_substreams: Default::default(), + initial_keep_alive_deadline: Instant::now() + initial_keep_alive, + } + } +} + +impl + SubstreamProtocolsHandler +{ + pub fn new_inbound_only(initial_keep_alive: Duration) -> Self { + Self { + inbound_substreams: Default::default(), + outbound_substreams: Default::default(), + next_inbound_substream_id: InboundSubstreamId(0), + next_outbound_substream_id: OutboundSubstreamId(0), + new_substreams: Default::default(), + initial_keep_alive_deadline: Instant::now() + initial_keep_alive, + } + } +} + +/// Poll all substreams within the given HashMap. +/// +/// This is defined as a separate function because we call it with two different fields stored within [`SubstreamProtocolsHandler`]. +fn poll_substreams( + substreams: &mut HashMap, + cx: &mut Context<'_>, +) -> Poll> +where + TSubstream: SubstreamHandler, + TId: Copy + Eq + Hash + fmt::Display, +{ + let substream_ids = substreams.keys().copied().collect::>(); + + 'loop_substreams: for id in substream_ids { + let mut handler = substreams + .remove(&id) + .expect("we just got the key out of the map"); + + let (next_state, poll) = 'loop_handler: loop { + match handler.advance(cx) { + Ok(Next::EmitEvent { next_state, event }) => { + break (next_state, Poll::Ready(Ok((id, event)))) + } + Ok(Next::Pending { next_state }) => break (next_state, Poll::Pending), + Ok(Next::Continue { next_state }) => { + handler = next_state; + continue 'loop_handler; + } + Ok(Next::Done) => { + log::debug!("Substream handler {} finished", id); + continue 'loop_substreams; + } + Err(e) => return Poll::Ready(Err((id, e))), + } + }; + + substreams.insert(id, next_state); + + return poll; + } + + Poll::Pending +} + +/// Event sent from the [`libp2p_swarm::NetworkBehaviour`] to the [`SubstreamProtocolsHandler`]. +#[derive(Debug)] +pub enum InEvent { + /// Open a new substream using the provided `open_info`. + /// + /// For "client-server" protocols, this is typically the initial message to be sent to the other party. + NewSubstream { open_info: I }, + NotifyInboundSubstream { + id: InboundSubstreamId, + message: TInboundEvent, + }, + NotifyOutboundSubstream { + id: OutboundSubstreamId, + message: TOutboundEvent, + }, +} + +/// Event produced by the [`SubstreamProtocolsHandler`] for the corresponding [`libp2p_swarm::NetworkBehaviour`]. +#[derive(Debug)] +pub enum OutEvent { + /// An inbound substream produced an event. + InboundEvent { + id: InboundSubstreamId, + message: TInbound, + }, + /// An outbound substream produced an event. + OutboundEvent { + id: OutboundSubstreamId, + message: TOutbound, + }, + /// An inbound substream errored irrecoverably. + InboundError { + id: InboundSubstreamId, + error: TInboundError, + }, + /// An outbound substream errored irrecoverably. + OutboundError { + id: OutboundSubstreamId, + error: TOutboundError, + }, +} + +impl< + TInboundInEvent, + TInboundOutEvent, + TOutboundInEvent, + TOutboundOutEvent, + TOutboundOpenInfo, + TInboundError, + TOutboundError, + TInboundSubstreamHandler, + TOutboundSubstreamHandler, + > ProtocolsHandler + for SubstreamProtocolsHandler< + TInboundSubstreamHandler, + TOutboundSubstreamHandler, + TOutboundOpenInfo, + > +where + TInboundSubstreamHandler: SubstreamHandler< + InEvent = TInboundInEvent, + OutEvent = TInboundOutEvent, + Error = TInboundError, + OpenInfo = (), + >, + TOutboundSubstreamHandler: SubstreamHandler< + InEvent = TOutboundInEvent, + OutEvent = TOutboundOutEvent, + Error = TOutboundError, + OpenInfo = TOutboundOpenInfo, + >, + TInboundInEvent: fmt::Debug + Send + 'static, + TInboundOutEvent: fmt::Debug + Send + 'static, + TOutboundInEvent: fmt::Debug + Send + 'static, + TOutboundOutEvent: fmt::Debug + Send + 'static, + TOutboundOpenInfo: fmt::Debug + Send + 'static, + TInboundError: fmt::Debug + Send + 'static, + TOutboundError: fmt::Debug + Send + 'static, + TInboundSubstreamHandler: Send + 'static, + TOutboundSubstreamHandler: Send + 'static, +{ + type InEvent = InEvent; + type OutEvent = OutEvent; + type Error = Void; + type InboundProtocol = PassthroughProtocol; + type OutboundProtocol = PassthroughProtocol; + type InboundOpenInfo = (); + type OutboundOpenInfo = TOutboundOpenInfo; + + fn listen_protocol(&self) -> SubstreamProtocol { + TInboundSubstreamHandler::upgrade(()) + } + + fn inject_fully_negotiated_inbound( + &mut self, + protocol: ::Output, + _: Self::InboundOpenInfo, + ) { + self.inbound_substreams.insert( + self.next_inbound_substream_id.fetch_and_increment(), + TInboundSubstreamHandler::new(protocol, ()), + ); + } + + fn inject_fully_negotiated_outbound( + &mut self, + protocol: ::Output, + info: Self::OutboundOpenInfo, + ) { + self.outbound_substreams.insert( + self.next_outbound_substream_id.fetch_and_increment(), + TOutboundSubstreamHandler::new(protocol, info), + ); + } + + fn inject_event(&mut self, event: Self::InEvent) { + match event { + InEvent::NewSubstream { open_info } => self.new_substreams.push_back(open_info), + InEvent::NotifyInboundSubstream { id, message } => { + match self.inbound_substreams.remove(&id) { + Some(handler) => { + let new_handler = handler.inject_event(message); + + self.inbound_substreams.insert(id, new_handler); + } + None => { + log::debug!("Substream with ID {} not found", id); + } + } + } + InEvent::NotifyOutboundSubstream { id, message } => { + match self.outbound_substreams.remove(&id) { + Some(handler) => { + let new_handler = handler.inject_event(message); + + self.outbound_substreams.insert(id, new_handler); + } + None => { + log::debug!("Substream with ID {} not found", id); + } + } + } + } + } + + fn inject_dial_upgrade_error( + &mut self, + _: Self::OutboundOpenInfo, + _: ProtocolsHandlerUpgrErr, + ) { + // TODO: Handle upgrade errors properly + } + + fn connection_keep_alive(&self) -> KeepAlive { + // Rudimentary keep-alive handling, to be extended as needed as this abstraction is used more by other protocols. + + if Instant::now() < self.initial_keep_alive_deadline { + return KeepAlive::Yes; + } + + if self.inbound_substreams.is_empty() + && self.outbound_substreams.is_empty() + && self.new_substreams.is_empty() + { + return KeepAlive::No; + } + + KeepAlive::Yes + } + + fn poll( + &mut self, + cx: &mut Context<'_>, + ) -> Poll< + ProtocolsHandlerEvent< + Self::OutboundProtocol, + Self::OutboundOpenInfo, + Self::OutEvent, + Self::Error, + >, + > { + if let Some(open_info) = self.new_substreams.pop_front() { + return Poll::Ready(ProtocolsHandlerEvent::OutboundSubstreamRequest { + protocol: TOutboundSubstreamHandler::upgrade(open_info), + }); + } + + match poll_substreams(&mut self.inbound_substreams, cx) { + Poll::Ready(Ok((id, message))) => { + return Poll::Ready(ProtocolsHandlerEvent::Custom(OutEvent::InboundEvent { + id, + message, + })) + } + Poll::Ready(Err((id, error))) => { + return Poll::Ready(ProtocolsHandlerEvent::Custom(OutEvent::InboundError { + id, + error, + })) + } + Poll::Pending => {} + } + + match poll_substreams(&mut self.outbound_substreams, cx) { + Poll::Ready(Ok((id, message))) => { + return Poll::Ready(ProtocolsHandlerEvent::Custom(OutEvent::OutboundEvent { + id, + message, + })) + } + Poll::Ready(Err((id, error))) => { + return Poll::Ready(ProtocolsHandlerEvent::Custom(OutEvent::OutboundError { + id, + error, + })) + } + Poll::Pending => {} + } + + Poll::Pending + } +} + +/// A helper struct for substream handlers that can be implemented as async functions. +/// +/// This only works for substreams without an `InEvent` because - once constructed - the state of an inner future is opaque. +pub struct FutureSubstream { + future: Fuse>>, +} + +impl FutureSubstream { + pub fn new(future: impl Future> + Send + 'static) -> Self { + Self { + future: future.boxed().fuse(), + } + } + + pub fn advance(mut self, cx: &mut Context<'_>) -> Result, TError> { + if self.future.is_terminated() { + return Ok(Next::Done); + } + + match self.future.poll_unpin(cx) { + Poll::Ready(Ok(event)) => Ok(Next::EmitEvent { + event, + next_state: self, + }), + Poll::Ready(Err(error)) => Err(error), + Poll::Pending => Ok(Next::Pending { next_state: self }), + } + } +} + +impl SubstreamHandler for void::Void { + type InEvent = void::Void; + type OutEvent = void::Void; + type Error = void::Void; + type OpenInfo = (); + + fn new(_: NegotiatedSubstream, _: Self::OpenInfo) -> Self { + unreachable!("we should never yield a substream") + } + + fn inject_event(self, event: Self::InEvent) -> Self { + void::unreachable(event) + } + + fn advance(self, _: &mut Context<'_>) -> Result, Self::Error> { + void::unreachable(self) + } + + fn upgrade( + open_info: Self::OpenInfo, + ) -> SubstreamProtocol { + SubstreamProtocol::new(PassthroughProtocol { ident: None }, open_info) + } +} diff --git a/protocols/rendezvous/tests/harness/mod.rs b/protocols/rendezvous/tests/harness/mod.rs new file mode 100644 index 00000000000..5747f7d19a6 --- /dev/null +++ b/protocols/rendezvous/tests/harness/mod.rs @@ -0,0 +1,221 @@ +// Copyright 2021 COMIT Network. +// +// 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 async_trait::async_trait; +use futures::stream::FusedStream; +use futures::StreamExt; +use futures::{future, Stream}; +use libp2p::core::muxing::StreamMuxerBox; +use libp2p::core::transport::upgrade::Version; +use libp2p::core::transport::MemoryTransport; +use libp2p::core::upgrade::SelectUpgrade; +use libp2p::core::{identity, Multiaddr, PeerId, Transport}; +use libp2p::mplex::MplexConfig; +use libp2p::noise::{Keypair, NoiseConfig, X25519Spec}; +use libp2p::swarm::{AddressScore, NetworkBehaviour, Swarm, SwarmBuilder, SwarmEvent}; +use libp2p::yamux::YamuxConfig; +use std::fmt::Debug; +use std::time::Duration; + +pub fn new_swarm(behaviour_fn: F) -> Swarm +where + B: NetworkBehaviour, + ::OutEvent: Debug, + B: NetworkBehaviour, + F: FnOnce(PeerId, identity::Keypair) -> B, +{ + let identity = identity::Keypair::generate_ed25519(); + let peer_id = PeerId::from(identity.public()); + + let dh_keys = Keypair::::new() + .into_authentic(&identity) + .expect("failed to create dh_keys"); + let noise = NoiseConfig::xx(dh_keys).into_authenticated(); + + let transport = MemoryTransport::default() + .upgrade(Version::V1) + .authenticate(noise) + .multiplex(SelectUpgrade::new( + YamuxConfig::default(), + MplexConfig::new(), + )) + .timeout(Duration::from_secs(5)) + .map(|(peer, muxer), _| (peer, StreamMuxerBox::new(muxer))) + .boxed(); + + SwarmBuilder::new(transport, behaviour_fn(peer_id, identity), peer_id) + .executor(Box::new(|future| { + let _ = tokio::spawn(future); + })) + .build() +} + +fn get_rand_memory_address() -> Multiaddr { + let address_port = rand::random::(); + let addr = format!("/memory/{}", address_port) + .parse::() + .unwrap(); + + addr +} + +pub async fn await_events_or_timeout( + swarm_1: &mut (impl Stream> + FusedStream + Unpin), + swarm_2: &mut (impl Stream> + FusedStream + Unpin), +) -> (SwarmEvent, SwarmEvent) +where + SwarmEvent: Debug, + SwarmEvent: Debug, +{ + tokio::time::timeout( + Duration::from_secs(30), + future::join( + swarm_1 + .inspect(|event| log::debug!("Swarm1 emitted {:?}", event)) + .select_next_some(), + swarm_2 + .inspect(|event| log::debug!("Swarm2 emitted {:?}", event)) + .select_next_some(), + ), + ) + .await + .expect("network behaviours to emit an event within 10 seconds") +} + +#[macro_export] +macro_rules! assert_behaviour_events { + ($swarm1: ident: $pat1: pat, $swarm2: ident: $pat2: pat, || $body: block) => { + match await_events_or_timeout(&mut $swarm1, &mut $swarm2).await { + ( + libp2p::swarm::SwarmEvent::Behaviour($pat1), + libp2p::swarm::SwarmEvent::Behaviour($pat2), + ) => $body, + _ => panic!("Unexpected combination of events emitted, check logs for details"), + } + }; +} + +/// An extension trait for [`Swarm`] that makes it easier to set up a network of [`Swarm`]s for tests. +#[async_trait] +pub trait SwarmExt { + /// Establishes a connection to the given [`Swarm`], polling both of them until the connection is established. + async fn block_on_connection(&mut self, other: &mut Swarm) + where + T: NetworkBehaviour, + ::OutEvent: Debug; + + /// Listens on a random memory address, polling the [`Swarm`] until the transport is ready to accept connections. + async fn listen_on_random_memory_address(&mut self) -> Multiaddr; + + /// Spawns the given [`Swarm`] into a runtime, polling it endlessly. + fn spawn_into_runtime(self); +} + +#[async_trait] +impl SwarmExt for Swarm +where + B: NetworkBehaviour, + ::OutEvent: Debug, +{ + async fn block_on_connection(&mut self, other: &mut Swarm) + where + T: NetworkBehaviour, + ::OutEvent: Debug, + { + let addr_to_dial = other.external_addresses().next().unwrap().addr.clone(); + + self.dial_addr(addr_to_dial.clone()).unwrap(); + + let mut dialer_done = false; + let mut listener_done = false; + + loop { + let dialer_event_fut = self.select_next_some(); + + tokio::select! { + dialer_event = dialer_event_fut => { + match dialer_event { + SwarmEvent::ConnectionEstablished { .. } => { + dialer_done = true; + } + SwarmEvent::UnknownPeerUnreachableAddr { address, error } if address == addr_to_dial => { + panic!("Failed to dial address {}: {}", addr_to_dial, error) + } + other => { + log::debug!("Ignoring {:?}", other); + } + } + }, + listener_event = other.select_next_some() => { + match listener_event { + SwarmEvent::ConnectionEstablished { .. } => { + listener_done = true; + } + SwarmEvent::IncomingConnectionError { error, .. } => { + panic!("Failure in incoming connection {}", error); + } + other => { + log::debug!("Ignoring {:?}", other); + } + } + } + } + + if dialer_done && listener_done { + return; + } + } + } + + async fn listen_on_random_memory_address(&mut self) -> Multiaddr { + let memory_addr_listener_id = self.listen_on(get_rand_memory_address()).unwrap(); + + // block until we are actually listening + let multiaddr = loop { + match self.select_next_some().await { + SwarmEvent::NewListenAddr { + address, + listener_id, + } if listener_id == memory_addr_listener_id => { + break address; + } + other => { + log::debug!( + "Ignoring {:?} while waiting for listening to succeed", + other + ); + } + } + }; + + // Memory addresses are externally reachable because they all share the same memory-space. + self.add_external_address(multiaddr.clone(), AddressScore::Infinite); + + multiaddr + } + + fn spawn_into_runtime(mut self) { + tokio::spawn(async move { + loop { + self.next().await; + } + }); + } +} diff --git a/protocols/rendezvous/tests/rendezvous.rs b/protocols/rendezvous/tests/rendezvous.rs new file mode 100644 index 00000000000..e6ba5fcc39a --- /dev/null +++ b/protocols/rendezvous/tests/rendezvous.rs @@ -0,0 +1,384 @@ +// Copyright 2021 COMIT Network. +// +// 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. + +#[macro_use] +pub mod harness; + +use crate::harness::{await_events_or_timeout, new_swarm, SwarmExt}; +use futures::stream::FuturesUnordered; +use futures::StreamExt; +use libp2p_core::identity; +use libp2p_rendezvous as rendezvous; +use libp2p_swarm::DialError; +use libp2p_swarm::{Swarm, SwarmEvent}; +use std::convert::TryInto; +use std::time::Duration; + +#[tokio::test] +async fn given_successful_registration_then_successful_discovery() { + let _ = env_logger::try_init(); + let namespace = rendezvous::Namespace::from_static("some-namespace"); + let ([mut alice, mut bob], mut robert) = + new_server_with_connected_clients(rendezvous::server::Config::default()).await; + + let _ = alice + .behaviour_mut() + .register(namespace.clone(), *robert.local_peer_id(), None); + + assert_behaviour_events! { + alice: rendezvous::client::Event::Registered { rendezvous_node, ttl, namespace: register_node_namespace }, + robert: rendezvous::server::Event::PeerRegistered { peer, registration }, + || { + assert_eq!(&peer, alice.local_peer_id()); + assert_eq!(&rendezvous_node, robert.local_peer_id()); + assert_eq!(registration.namespace, namespace); + assert_eq!(register_node_namespace, namespace); + assert_eq!(ttl, rendezvous::DEFAULT_TTL); + } + }; + + bob.behaviour_mut() + .discover(Some(namespace.clone()), None, None, *robert.local_peer_id()); + + assert_behaviour_events! { + bob: rendezvous::client::Event::Discovered { registrations, .. }, + robert: rendezvous::server::Event::DiscoverServed { .. }, + || { + match registrations.as_slice() { + [rendezvous::Registration { + namespace: registered_namespace, + record, + ttl, + }] => { + assert_eq!(*ttl, rendezvous::DEFAULT_TTL); + assert_eq!(record.peer_id(), *alice.local_peer_id()); + assert_eq!(*registered_namespace, namespace); + } + _ => panic!("Expected exactly one registration to be returned from discover"), + } + } + }; +} + +#[tokio::test] +async fn given_successful_registration_then_refresh_ttl() { + let _ = env_logger::try_init(); + let namespace = rendezvous::Namespace::from_static("some-namespace"); + let ([mut alice, mut bob], mut robert) = + new_server_with_connected_clients(rendezvous::server::Config::default()).await; + + let roberts_peer_id = *robert.local_peer_id(); + let refresh_ttl = 10_000; + + let _ = alice + .behaviour_mut() + .register(namespace.clone(), roberts_peer_id, None); + + assert_behaviour_events! { + alice: rendezvous::client::Event::Registered { .. }, + robert: rendezvous::server::Event::PeerRegistered { .. }, + || { } + }; + + bob.behaviour_mut() + .discover(Some(namespace.clone()), None, None, roberts_peer_id); + + assert_behaviour_events! { + bob: rendezvous::client::Event::Discovered { .. }, + robert: rendezvous::server::Event::DiscoverServed { .. }, + || { } + }; + + alice + .behaviour_mut() + .register(namespace.clone(), roberts_peer_id, Some(refresh_ttl)); + + assert_behaviour_events! { + alice: rendezvous::client::Event::Registered { ttl, .. }, + robert: rendezvous::server::Event::PeerRegistered { .. }, + || { + assert_eq!(ttl, refresh_ttl); + } + }; + + bob.behaviour_mut() + .discover(Some(namespace.clone()), None, None, *robert.local_peer_id()); + + assert_behaviour_events! { + bob: rendezvous::client::Event::Discovered { registrations, .. }, + robert: rendezvous::server::Event::DiscoverServed { .. }, + || { + match registrations.as_slice() { + [rendezvous::Registration { ttl, .. }] => { + assert_eq!(*ttl, refresh_ttl); + } + _ => panic!("Expected exactly one registration to be returned from discover"), + } + } + }; +} + +#[tokio::test] +async fn given_invalid_ttl_then_unsuccessful_registration() { + let _ = env_logger::try_init(); + let namespace = rendezvous::Namespace::from_static("some-namespace"); + let ([mut alice], mut robert) = + new_server_with_connected_clients(rendezvous::server::Config::default()).await; + + alice.behaviour_mut().register( + namespace.clone(), + *robert.local_peer_id(), + Some(100_000_000), + ); + + assert_behaviour_events! { + alice: rendezvous::client::Event::RegisterFailed(rendezvous::client::RegisterError::Remote {error , ..}), + robert: rendezvous::server::Event::PeerNotRegistered { .. }, + || { + assert_eq!(error, rendezvous::ErrorCode::InvalidTtl); + } + }; +} + +#[tokio::test] +async fn discover_allows_for_dial_by_peer_id() { + let _ = env_logger::try_init(); + let namespace = rendezvous::Namespace::from_static("some-namespace"); + let ([mut alice, mut bob], robert) = + new_server_with_connected_clients(rendezvous::server::Config::default()).await; + + let roberts_peer_id = *robert.local_peer_id(); + robert.spawn_into_runtime(); + + alice + .behaviour_mut() + .register(namespace.clone(), roberts_peer_id, None); + bob.behaviour_mut() + .discover(Some(namespace.clone()), None, None, roberts_peer_id); + + assert_behaviour_events! { + alice: rendezvous::client::Event::Registered { .. }, + bob: rendezvous::client::Event::Discovered { .. }, + || { } + }; + + let alices_peer_id = *alice.local_peer_id(); + let bobs_peer_id = *bob.local_peer_id(); + + bob.dial(&alices_peer_id).unwrap(); + + let alice_connected_to = tokio::spawn(async move { + loop { + if let SwarmEvent::ConnectionEstablished { peer_id, .. } = + alice.select_next_some().await + { + break peer_id; + } + } + }); + let bob_connected_to = tokio::spawn(async move { + loop { + if let SwarmEvent::ConnectionEstablished { peer_id, .. } = bob.select_next_some().await + { + break peer_id; + } + } + }); + + assert_eq!(alice_connected_to.await.unwrap(), bobs_peer_id); + assert_eq!(bob_connected_to.await.unwrap(), alices_peer_id); +} + +#[tokio::test] +async fn eve_cannot_register() { + let _ = env_logger::try_init(); + let namespace = rendezvous::Namespace::from_static("some-namespace"); + let mut robert = new_server(rendezvous::server::Config::default()).await; + let mut eve = new_impersonating_client().await; + eve.block_on_connection(&mut robert).await; + + eve.behaviour_mut() + .register(namespace.clone(), *robert.local_peer_id(), None); + + assert_behaviour_events! { + eve: rendezvous::client::Event::RegisterFailed(rendezvous::client::RegisterError::Remote { error: err_code , ..}), + robert: rendezvous::server::Event::PeerNotRegistered { .. }, + || { + assert_eq!(err_code, rendezvous::ErrorCode::NotAuthorized); + } + }; +} + +// test if charlie can operate as client and server simultaneously +#[tokio::test] +async fn can_combine_client_and_server() { + let _ = env_logger::try_init(); + let namespace = rendezvous::Namespace::from_static("some-namespace"); + let ([mut alice], mut robert) = + new_server_with_connected_clients(rendezvous::server::Config::default()).await; + let mut charlie = new_combined_node().await; + charlie.block_on_connection(&mut robert).await; + alice.block_on_connection(&mut charlie).await; + + charlie + .behaviour_mut() + .client + .register(namespace.clone(), *robert.local_peer_id(), None); + + assert_behaviour_events! { + charlie: CombinedEvent::Client(rendezvous::client::Event::Registered { .. }), + robert: rendezvous::server::Event::PeerRegistered { .. }, + || { } + }; + + alice + .behaviour_mut() + .register(namespace, *charlie.local_peer_id(), None); + + assert_behaviour_events! { + charlie: CombinedEvent::Server(rendezvous::server::Event::PeerRegistered { .. }), + alice: rendezvous::client::Event::Registered { .. }, + || { } + }; +} + +#[tokio::test] +async fn registration_on_clients_expire() { + let _ = env_logger::try_init(); + let namespace = rendezvous::Namespace::from_static("some-namespace"); + let ([mut alice, mut bob], robert) = + new_server_with_connected_clients(rendezvous::server::Config::default().with_min_ttl(1)) + .await; + + let roberts_peer_id = *robert.local_peer_id(); + robert.spawn_into_runtime(); + + let registration_ttl = 3; + + alice + .behaviour_mut() + .register(namespace.clone(), roberts_peer_id, Some(registration_ttl)); + bob.behaviour_mut() + .discover(Some(namespace), None, None, roberts_peer_id); + + assert_behaviour_events! { + alice: rendezvous::client::Event::Registered { .. }, + bob: rendezvous::client::Event::Discovered { .. }, + || { } + }; + + tokio::time::sleep(Duration::from_secs(registration_ttl + 5)).await; + + let event = bob.select_next_some().await; + let error = bob.dial(alice.local_peer_id()).unwrap_err(); + + assert!(matches!( + event, + SwarmEvent::Behaviour(rendezvous::client::Event::Expired { .. }) + )); + assert!(matches!(error, DialError::NoAddresses)); +} + +async fn new_server_with_connected_clients( + config: rendezvous::server::Config, +) -> ( + [Swarm; N], + Swarm, +) { + let mut server = new_server(config).await; + + let mut clients: [Swarm<_>; N] = match (0usize..N) + .map(|_| new_client()) + .collect::>() + .collect::>() + .await + .try_into() + { + Ok(clients) => clients, + Err(_) => panic!("Vec is of size N"), + }; + + for client in &mut clients { + client.block_on_connection(&mut server).await; + } + + (clients, server) +} + +async fn new_client() -> Swarm { + let mut client = new_swarm(|_, identity| rendezvous::client::Behaviour::new(identity)); + client.listen_on_random_memory_address().await; // we need to listen otherwise we don't have addresses to register + + client +} + +async fn new_server(config: rendezvous::server::Config) -> Swarm { + let mut server = new_swarm(|_, _| rendezvous::server::Behaviour::new(config)); + + server.listen_on_random_memory_address().await; + + server +} + +async fn new_combined_node() -> Swarm { + let mut node = new_swarm(|_, identity| CombinedBehaviour { + client: rendezvous::client::Behaviour::new(identity), + server: rendezvous::server::Behaviour::new(rendezvous::server::Config::default()), + }); + node.listen_on_random_memory_address().await; + + node +} + +async fn new_impersonating_client() -> Swarm { + // In reality, if Eve were to try and fake someones identity, she would obviously only know the public key. + // Due to the type-safe API of the `Rendezvous` behaviour and `PeerRecord`, we actually cannot construct a bad `PeerRecord` (i.e. one that is claims to be someone else). + // As such, the best we can do is hand eve a completely different keypair from what she is using to authenticate her connection. + let someone_else = identity::Keypair::generate_ed25519(); + let mut eve = new_swarm(move |_, _| rendezvous::client::Behaviour::new(someone_else)); + eve.listen_on_random_memory_address().await; + + eve +} + +#[derive(libp2p::NetworkBehaviour)] +#[behaviour(event_process = false, out_event = "CombinedEvent")] +struct CombinedBehaviour { + client: rendezvous::client::Behaviour, + server: rendezvous::server::Behaviour, +} + +#[derive(Debug)] +enum CombinedEvent { + Client(rendezvous::client::Event), + Server(rendezvous::server::Event), +} + +impl From for CombinedEvent { + fn from(v: rendezvous::server::Event) -> Self { + Self::Server(v) + } +} + +impl From for CombinedEvent { + fn from(v: rendezvous::client::Event) -> Self { + Self::Client(v) + } +} diff --git a/protocols/request-response/CHANGELOG.md b/protocols/request-response/CHANGELOG.md index 973fbe3f1ed..08be7686cbf 100644 --- a/protocols/request-response/CHANGELOG.md +++ b/protocols/request-response/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.13.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. - Manually implement `Debug` for `RequestResponseHandlerEvent` and diff --git a/protocols/request-response/Cargo.toml b/protocols/request-response/Cargo.toml index 40bc39000f3..6f71ec9f408 100644 --- a/protocols/request-response/Cargo.toml +++ b/protocols/request-response/Cargo.toml @@ -13,11 +13,11 @@ categories = ["network-programming", "asynchronous"] async-trait = "0.1" bytes = "1" futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } libp2p-swarm = { version = "0.31.0", path = "../../swarm" } log = "0.4.11" lru = "0.6" -minicbor = { version = "0.10", features = ["std", "derive"] } +minicbor = { version = "0.11", features = ["std", "derive"] } rand = "0.7" smallvec = "1.6.1" unsigned-varint = { version = "0.7", features = ["std", "futures"] } @@ -25,6 +25,7 @@ wasm-timer = "0.2" [dev-dependencies] async-std = "1.6.2" +env_logger = "0.9.0" libp2p-noise = { path = "../../transports/noise" } libp2p-tcp = { path = "../../transports/tcp" } libp2p-yamux = { path = "../../muxers/yamux" } diff --git a/protocols/request-response/src/lib.rs b/protocols/request-response/src/lib.rs index a2277e4c8df..ef3913c1efb 100644 --- a/protocols/request-response/src/lib.rs +++ b/protocols/request-response/src/lib.rs @@ -68,7 +68,8 @@ use futures::channel::oneshot; use handler::{RequestProtocol, RequestResponseHandler, RequestResponseHandlerEvent}; use libp2p_core::{connection::ConnectionId, ConnectedPoint, Multiaddr, PeerId}; use libp2p_swarm::{ - DialPeerCondition, NetworkBehaviour, NetworkBehaviourAction, NotifyHandler, PollParameters, + DialError, DialPeerCondition, IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, + NotifyHandler, PollParameters, }; use smallvec::SmallVec; use std::{ @@ -303,7 +304,7 @@ impl RequestResponseConfig { /// A request/response protocol for some message codec. pub struct RequestResponse where - TCodec: RequestResponseCodec, + TCodec: RequestResponseCodec + Clone + Send + 'static, { /// The supported inbound protocols. inbound_protocols: SmallVec<[TCodec::Protocol; 2]>, @@ -320,8 +321,8 @@ where /// Pending events to return from `poll`. pending_events: VecDeque< NetworkBehaviourAction< - RequestProtocol, RequestResponseEvent, + RequestResponseHandler, >, >, /// The currently connected peers, their pending outbound and inbound responses and their known, @@ -336,7 +337,7 @@ where impl RequestResponse where - TCodec: RequestResponseCodec + Clone, + TCodec: RequestResponseCodec + Clone + Send + 'static, { /// Creates a new `RequestResponse` behaviour for the given /// protocols, codec and configuration. @@ -403,10 +404,12 @@ where }; if let Some(request) = self.try_send_request(peer, request) { + let handler = self.new_handler(); self.pending_events .push_back(NetworkBehaviourAction::DialPeer { peer_id: *peer, condition: DialPeerCondition::Disconnected, + handler, }); self.pending_outbound_requests .entry(*peer) @@ -639,6 +642,7 @@ where peer_id: &PeerId, conn: &ConnectionId, _: &ConnectedPoint, + _: ::Handler, ) { let connections = self .connected @@ -682,7 +686,7 @@ where self.connected.remove(peer); } - fn inject_dial_failure(&mut self, peer: &PeerId) { + fn inject_dial_failure(&mut self, peer: &PeerId, _: Self::ProtocolsHandler, _: DialError) { // If there are pending outgoing requests when a dial failure occurs, // it is implied that we are not connected to the peer, since pending // outgoing requests are drained when a connection is established and @@ -863,12 +867,7 @@ where &mut self, _: &mut Context<'_>, _: &mut impl PollParameters, - ) -> Poll< - NetworkBehaviourAction< - RequestProtocol, - RequestResponseEvent, - >, - > { + ) -> Poll> { if let Some(ev) = self.pending_events.pop_front() { return Poll::Ready(ev); } else if self.pending_events.capacity() > EMPTY_QUEUE_SHRINK_THRESHOLD { diff --git a/protocols/request-response/src/throttled.rs b/protocols/request-response/src/throttled.rs index c882f41b211..2b8693bc437 100644 --- a/protocols/request-response/src/throttled.rs +++ b/protocols/request-response/src/throttled.rs @@ -40,11 +40,13 @@ use super::{ ProtocolSupport, RequestId, RequestResponse, RequestResponseCodec, RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, }; -use crate::handler::{RequestProtocol, RequestResponseHandler, RequestResponseHandlerEvent}; +use crate::handler::{RequestResponseHandler, RequestResponseHandlerEvent}; use codec::{Codec, Message, ProtocolWrapper, Type}; use futures::ready; use libp2p_core::{connection::ConnectionId, ConnectedPoint, Multiaddr, PeerId}; -use libp2p_swarm::{NetworkBehaviour, NetworkBehaviourAction, PollParameters}; +use libp2p_swarm::{ + DialError, IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, PollParameters, +}; use lru::LruCache; use std::{cmp::max, num::NonZeroU16}; use std::{ @@ -57,7 +59,7 @@ pub type ResponseChannel = super::ResponseChannel>; /// A wrapper around [`RequestResponse`] which adds request limits per peer. pub struct Throttled where - C: RequestResponseCodec + Send, + C: RequestResponseCodec + Clone + Send + 'static, C::Protocol: Sync, { /// A random id used for logging. @@ -439,8 +441,15 @@ where self.behaviour.inject_connection_established(p, id, end) } - fn inject_connection_closed(&mut self, peer: &PeerId, id: &ConnectionId, end: &ConnectedPoint) { - self.behaviour.inject_connection_closed(peer, id, end); + fn inject_connection_closed( + &mut self, + peer: &PeerId, + id: &ConnectionId, + end: &ConnectedPoint, + handler: ::Handler, + ) { + self.behaviour + .inject_connection_closed(peer, id, end, handler); if let Some(info) = self.peer_info.get_mut(peer) { if let Some(grant) = &mut info.recv_budget.grant { log::debug! { "{:08x}: resending credit grant {} to {} after connection closed", @@ -484,8 +493,13 @@ where self.behaviour.inject_disconnected(p) } - fn inject_dial_failure(&mut self, p: &PeerId) { - self.behaviour.inject_dial_failure(p) + fn inject_dial_failure( + &mut self, + p: &PeerId, + handler: Self::ProtocolsHandler, + error: DialError, + ) { + self.behaviour.inject_dial_failure(p, handler, error) } fn inject_event( @@ -501,7 +515,7 @@ where &mut self, cx: &mut Context<'_>, params: &mut impl PollParameters, - ) -> Poll>, Self::OutEvent>> { + ) -> Poll> { loop { if let Some(ev) = self.events.pop_front() { return Poll::Ready(NetworkBehaviourAction::GenerateEvent(ev)); @@ -737,12 +751,18 @@ where RequestResponseEvent::ResponseSent { peer, request_id }, )) } - NetworkBehaviourAction::DialAddress { address } => { - NetworkBehaviourAction::DialAddress { address } - } - NetworkBehaviourAction::DialPeer { peer_id, condition } => { - NetworkBehaviourAction::DialPeer { peer_id, condition } + NetworkBehaviourAction::DialAddress { address, handler } => { + NetworkBehaviourAction::DialAddress { address, handler } } + NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + } => NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + }, NetworkBehaviourAction::NotifyHandler { peer_id, handler, diff --git a/protocols/request-response/tests/ping.rs b/protocols/request-response/tests/ping.rs index 626f4effef3..884a378bcce 100644 --- a/protocols/request-response/tests/ping.rs +++ b/protocols/request-response/tests/ping.rs @@ -39,6 +39,7 @@ use std::{io, iter}; #[test] fn is_response_outbound() { + let _ = env_logger::try_init(); let ping = Ping("ping".to_string().into_bytes()); let offline_peer = PeerId::random(); diff --git a/src/lib.rs b/src/lib.rs index 52a5629d6cb..e75c306e245 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -76,6 +76,10 @@ pub use libp2p_kad as kad; #[cfg(not(any(target_os = "emscripten", target_os = "wasi", target_os = "unknown")))] #[doc(inline)] pub use libp2p_mdns as mdns; +#[cfg(feature = "metrics")] +#[cfg_attr(docsrs, doc(cfg(feature = "metrics")))] +#[doc(inline)] +pub use libp2p_metrics as metrics; #[cfg(feature = "mplex")] #[cfg_attr(docsrs, doc(cfg(feature = "mplex")))] #[doc(inline)] @@ -105,6 +109,10 @@ pub use libp2p_quic as quic; #[cfg_attr(docsrs, doc(cfg(feature = "relay")))] #[doc(inline)] pub use libp2p_relay as relay; +#[cfg(feature = "rendezvous")] +#[cfg_attr(docsrs, doc(cfg(feature = "rendezvous")))] +#[doc(inline)] +pub use libp2p_rendezvous as rendezvous; #[cfg(feature = "request-response")] #[cfg_attr(docsrs, doc(cfg(feature = "request-response")))] #[doc(inline)] diff --git a/swarm-derive/CHANGELOG.md b/swarm-derive/CHANGELOG.md index 94ea25f6b10..dc31bd83453 100644 --- a/swarm-derive/CHANGELOG.md +++ b/swarm-derive/CHANGELOG.md @@ -1,3 +1,11 @@ +# 0.25.0 [unreleased] + +- Update to latest `libp2p-swarm` changes (see [PR 2191]). + +- Make `event_process = false` the default. + +[PR 2191]: https://github.com/libp2p/rust-libp2p/pull/2191 + # 0.24.0 [2021-07-12] - Handle `NetworkBehaviourAction::CloseConnection`. See [PR 2110] for details. diff --git a/swarm-derive/Cargo.toml b/swarm-derive/Cargo.toml index e63b7a71dd8..2da7fc8a34b 100644 --- a/swarm-derive/Cargo.toml +++ b/swarm-derive/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-swarm-derive" edition = "2018" description = "Procedural macros of libp2p-core" -version = "0.24.0" +version = "0.25.0" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" diff --git a/swarm-derive/src/lib.rs b/swarm-derive/src/lib.rs index 3f92e549fa9..ded16461146 100644 --- a/swarm-derive/src/lib.rs +++ b/swarm-derive/src/lib.rs @@ -57,6 +57,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { let connection_id = quote! {::libp2p::core::connection::ConnectionId}; let connected_point = quote! {::libp2p::core::ConnectedPoint}; let listener_id = quote! {::libp2p::core::connection::ListenerId}; + let dial_error = quote! {::libp2p::swarm::DialError}; let poll_parameters = quote! {::libp2p::swarm::PollParameters}; @@ -70,7 +71,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // Whether or not we require the `NetworkBehaviourEventProcess` trait to be implemented. let event_process = { - let mut event_process = true; // Default to true for backwards compatibility + let mut event_process = false; for meta_items in ast.attrs.iter().filter_map(get_meta_items) { for meta_item in meta_items { @@ -150,7 +151,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } @@ -168,7 +169,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } Some(match field.ident { @@ -185,7 +186,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } Some(match field.ident { @@ -198,7 +199,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // Build the list of statements to put in the body of `inject_connection_established()`. let inject_connection_established_stmts = { data_struct.fields.iter().enumerate().filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } Some(match field.ident { @@ -211,7 +212,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // Build the list of statements to put in the body of `inject_address_change()`. let inject_address_change_stmts = { data_struct.fields.iter().enumerate().filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } Some(match field.ident { @@ -223,22 +224,40 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // Build the list of statements to put in the body of `inject_connection_closed()`. let inject_connection_closed_stmts = { - data_struct.fields.iter().enumerate().filter_map(move |(field_n, field)| { - if is_ignored(&field) { - return None; - } - Some(match field.ident { - Some(ref i) => quote!{ self.#i.inject_connection_closed(peer_id, connection_id, endpoint); }, - None => quote!{ self.#field_n.inject_connection_closed(peer_id, connection_id, endpoint); }, + data_struct + .fields + .iter() + .enumerate() + // The outmost handler belongs to the last behaviour. + .rev() + .filter(|f| !is_ignored(f.1)) + .enumerate() + .map(move |(enum_n, (field_n, field))| { + let handler = if field_n == 0 { + // Given that the iterator is reversed, this is the innermost handler only. + quote! { let handler = handlers } + } else { + quote! { + let (handlers, handler) = handlers.into_inner() + } + }; + let inject = match field.ident { + Some(ref i) => quote!{ self.#i.inject_connection_closed(peer_id, connection_id, endpoint, handler) }, + None => quote!{ self.#enum_n.inject_connection_closed(peer_id, connection_id, endpoint, handler) }, + }; + + quote! { + #handler; + #inject; + } }) - }) }; // Build the list of statements to put in the body of `inject_addr_reach_failure()`. let inject_addr_reach_failure_stmts = { data_struct.fields.iter().enumerate().filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } @@ -255,15 +274,63 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .fields .iter() .enumerate() - .filter_map(move |(field_n, field)| { - if is_ignored(&field) { - return None; + // The outmost handler belongs to the last behaviour. + .rev() + .filter(|f| !is_ignored(f.1)) + .enumerate() + .map(move |(enum_n, (field_n, field))| { + let handler = if field_n == 0 { + // Given that the iterator is reversed, this is the innermost handler only. + quote! { let handler = handlers } + } else { + quote! { + let (handlers, handler) = handlers.into_inner() + } + }; + + let inject = match field.ident { + Some(ref i) => { + quote! { self.#i.inject_dial_failure(peer_id, handler, error.clone()) } + } + None => { + quote! { self.#enum_n.inject_dial_failure(peer_id, handler, error.clone()) } + } + }; + + quote! { + #handler; + #inject; } + }) + }; - Some(match field.ident { - Some(ref i) => quote! { self.#i.inject_dial_failure(peer_id); }, - None => quote! { self.#field_n.inject_dial_failure(peer_id); }, - }) + // Build the list of statements to put in the body of `inject_listen_failure()`. + let inject_listen_failure_stmts = { + data_struct + .fields + .iter() + .enumerate() + .rev() + .filter(|f| !is_ignored(f.1)) + .enumerate() + .map(move |(enum_n, (field_n, field))| { + let handler = if field_n == 0 { + quote! { let handler = handlers } + } else { + quote! { + let (handlers, handler) = handlers.into_inner() + } + }; + + let inject = match field.ident { + Some(ref i) => quote! { self.#i.inject_listen_failure(local_addr, send_back_addr, handler) }, + None => quote! { self.#enum_n.inject_listen_failure(local_addr, send_back_addr, handler) }, + }; + + quote! { + #handler; + #inject; + } }) }; @@ -274,7 +341,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } @@ -292,7 +359,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } @@ -310,7 +377,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } @@ -328,7 +395,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } @@ -346,7 +413,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } @@ -364,7 +431,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } Some(match field.ident { @@ -381,7 +448,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { .iter() .enumerate() .filter_map(move |(field_n, field)| { - if is_ignored(&field) { + if is_ignored(field) { return None; } Some(match field.ident { @@ -395,7 +462,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // // The event type is a construction of nested `#either_ident`s of the events of the children. // We call `inject_event` on the corresponding child. - let inject_node_event_stmts = data_struct.fields.iter().enumerate().filter(|f| !is_ignored(&f.1)).enumerate().map(|(enum_n, (field_n, field))| { + let inject_node_event_stmts = data_struct.fields.iter().enumerate().filter(|f| !is_ignored(f.1)).enumerate().map(|(enum_n, (field_n, field))| { let mut elem = if enum_n != 0 { quote!{ #either_ident::Second(ev) } } else { @@ -416,7 +483,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { let protocols_handler_ty = { let mut ph_ty = None; for field in data_struct.fields.iter() { - if is_ignored(&field) { + if is_ignored(field) { continue; } let ty = &field.ty; @@ -426,6 +493,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { ref mut ev @ None => *ev = Some(field_info), } } + // ph_ty = Some(quote! ) ph_ty.unwrap_or(quote! {()}) // TODO: `!` instead }; @@ -435,7 +503,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { let mut out_handler = None; for (field_n, field) in data_struct.fields.iter().enumerate() { - if is_ignored(&field) { + if is_ignored(field) { continue; } @@ -456,7 +524,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { } } - out_handler.unwrap_or(quote! {()}) // TODO: incorrect + out_handler.unwrap_or(quote! {()}) // TODO: See test `empty`. }; // The method to use to poll. @@ -485,7 +553,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { // List of statements to put in `poll()`. // // We poll each child one by one and wrap around the output. - let poll_stmts = data_struct.fields.iter().enumerate().filter(|f| !is_ignored(&f.1)).enumerate().map(|(enum_n, (field_n, field))| { + let poll_stmts = data_struct.fields.iter().enumerate().filter(|f| !is_ignored(f.1)).enumerate().map(|(enum_n, (field_n, field))| { let field_name = match field.ident { Some(ref i) => quote!{ self.#i }, None => quote!{ self.#field_n }, @@ -500,6 +568,42 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { wrapped_event = quote!{ #either_ident::First(#wrapped_event) }; } + // `DialPeer` and `DialAddress` each provide a handler of the specific + // behaviour triggering the event. Though in order for the final handler + // to be able to handle protocols of all behaviours, the provided + // handler needs to be combined with handlers of all other behaviours. + let provided_handler_and_new_handlers = { + let mut out_handler = None; + + for (f_n, f) in data_struct.fields.iter().enumerate() { + if is_ignored(f) { + continue; + } + + let f_name = match f.ident { + Some(ref i) => quote! { self.#i }, + None => quote! { self.#f_n }, + }; + + let builder = if field_n == f_n { + // The behaviour that triggered the event. Thus, instead of + // creating a new handler, use the provided handler. + quote! { provided_handler } + } else { + quote! { #f_name.new_handler() } + }; + + match out_handler { + Some(h) => { + out_handler = Some(quote! { #into_protocols_handler::select(#h, #builder) }) + } + ref mut h @ None => *h = Some(builder), + } + } + + out_handler.unwrap_or(quote! {()}) // TODO: See test `empty`. + }; + let generate_event_match_arm = if event_process { quote! { std::task::Poll::Ready(#network_behaviour_action::GenerateEvent(event)) => { @@ -518,11 +622,11 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { loop { match #trait_to_impl::poll(&mut #field_name, cx, poll_params) { #generate_event_match_arm - std::task::Poll::Ready(#network_behaviour_action::DialAddress { address }) => { - return std::task::Poll::Ready(#network_behaviour_action::DialAddress { address }); + std::task::Poll::Ready(#network_behaviour_action::DialAddress { address, handler: provided_handler }) => { + return std::task::Poll::Ready(#network_behaviour_action::DialAddress { address, handler: #provided_handler_and_new_handlers }); } - std::task::Poll::Ready(#network_behaviour_action::DialPeer { peer_id, condition }) => { - return std::task::Poll::Ready(#network_behaviour_action::DialPeer { peer_id, condition }); + std::task::Poll::Ready(#network_behaviour_action::DialPeer { peer_id, condition, handler: provided_handler }) => { + return std::task::Poll::Ready(#network_behaviour_action::DialPeer { peer_id, condition, handler: #provided_handler_and_new_handlers }); } std::task::Poll::Ready(#network_behaviour_action::NotifyHandler { peer_id, handler, event }) => { return std::task::Poll::Ready(#network_behaviour_action::NotifyHandler { @@ -578,7 +682,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { #(#inject_address_change_stmts);* } - fn inject_connection_closed(&mut self, peer_id: &#peer_id, connection_id: &#connection_id, endpoint: &#connected_point) { + fn inject_connection_closed(&mut self, peer_id: &#peer_id, connection_id: &#connection_id, endpoint: &#connected_point, handlers: ::Handler) { #(#inject_connection_closed_stmts);* } @@ -586,10 +690,14 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { #(#inject_addr_reach_failure_stmts);* } - fn inject_dial_failure(&mut self, peer_id: &#peer_id) { + fn inject_dial_failure(&mut self, peer_id: &#peer_id, handlers: Self::ProtocolsHandler, error: #dial_error) { #(#inject_dial_failure_stmts);* } + fn inject_listen_failure(&mut self, local_addr: &#multiaddr, send_back_addr: &#multiaddr, handlers: Self::ProtocolsHandler) { + #(#inject_listen_failure_stmts);* + } + fn inject_new_listener(&mut self, id: #listener_id) { #(#inject_new_listener_stmts);* } @@ -629,10 +737,10 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { } } - fn poll(&mut self, cx: &mut std::task::Context, poll_params: &mut impl #poll_parameters) -> std::task::Poll<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> { + fn poll(&mut self, cx: &mut std::task::Context, poll_params: &mut impl #poll_parameters) -> std::task::Poll<#network_behaviour_action> { use libp2p::futures::prelude::*; #(#poll_stmts)* - let f: std::task::Poll<#network_behaviour_action<<::Handler as #protocols_handler>::InEvent, Self::OutEvent>> = #poll_method; + let f: std::task::Poll<#network_behaviour_action> = #poll_method; f } } diff --git a/swarm-derive/tests/test.rs b/swarm-derive/tests/test.rs index 78a9ed985f9..829c9b71e34 100644 --- a/swarm-derive/tests/test.rs +++ b/swarm-derive/tests/test.rs @@ -19,7 +19,7 @@ // DEALINGS IN THE SOFTWARE. use futures::prelude::*; -use libp2p::swarm::SwarmEvent; +use libp2p::swarm::{NetworkBehaviour, SwarmEvent}; use libp2p_swarm_derive::*; /// Small utility to check that a type implements `NetworkBehaviour`. @@ -38,6 +38,7 @@ fn empty() { fn one_field() { #[allow(dead_code)] #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct Foo { ping: libp2p::ping::Ping, } @@ -56,6 +57,7 @@ fn one_field() { fn two_fields() { #[allow(dead_code)] #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct Foo { ping: libp2p::ping::Ping, identify: libp2p::identify::Identify, @@ -79,6 +81,7 @@ fn two_fields() { fn three_fields() { #[allow(dead_code)] #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct Foo { ping: libp2p::ping::Ping, identify: libp2p::identify::Identify, @@ -109,6 +112,7 @@ fn three_fields() { fn three_fields_non_last_ignored() { #[allow(dead_code)] #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct Foo { ping: libp2p::ping::Ping, #[behaviour(ignore)] @@ -134,7 +138,7 @@ fn three_fields_non_last_ignored() { fn custom_polling() { #[allow(dead_code)] #[derive(NetworkBehaviour)] - #[behaviour(poll_method = "foo")] + #[behaviour(poll_method = "foo", event_process = true)] struct Foo { ping: libp2p::ping::Ping, identify: libp2p::identify::Identify, @@ -149,11 +153,16 @@ fn custom_polling() { } impl Foo { - fn foo( + fn foo( &mut self, _: &mut std::task::Context, _: &mut impl libp2p::swarm::PollParameters, - ) -> std::task::Poll> { + ) -> std::task::Poll< + libp2p::swarm::NetworkBehaviourAction< + ::OutEvent, + ::ProtocolsHandler, + >, + > { std::task::Poll::Pending } } @@ -168,7 +177,7 @@ fn custom_polling() { fn custom_event_no_polling() { #[allow(dead_code)] #[derive(NetworkBehaviour)] - #[behaviour(out_event = "Vec")] + #[behaviour(out_event = "Vec", event_process = true)] struct Foo { ping: libp2p::ping::Ping, identify: libp2p::identify::Identify, @@ -192,7 +201,7 @@ fn custom_event_no_polling() { fn custom_event_and_polling() { #[allow(dead_code)] #[derive(NetworkBehaviour)] - #[behaviour(poll_method = "foo", out_event = "String")] + #[behaviour(poll_method = "foo", out_event = "String", event_process = true)] struct Foo { ping: libp2p::ping::Ping, identify: libp2p::identify::Identify, @@ -207,11 +216,16 @@ fn custom_event_and_polling() { } impl Foo { - fn foo( + fn foo( &mut self, _: &mut std::task::Context, _: &mut impl libp2p::swarm::PollParameters, - ) -> std::task::Poll> { + ) -> std::task::Poll< + libp2p::swarm::NetworkBehaviourAction< + ::OutEvent, + ::ProtocolsHandler, + >, + > { std::task::Poll::Pending } } @@ -226,6 +240,7 @@ fn custom_event_and_polling() { fn where_clause() { #[allow(dead_code)] #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct Foo { ping: libp2p::ping::Ping, bar: T, @@ -238,12 +253,14 @@ fn nested_derives_with_import() { #[allow(dead_code)] #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct Foo { ping: libp2p::ping::Ping, } #[allow(dead_code)] #[derive(NetworkBehaviour)] + #[behaviour(event_process = true)] struct Bar { foo: Foo, } @@ -283,7 +300,7 @@ fn event_process_false() { #[allow(dead_code)] #[derive(NetworkBehaviour)] - #[behaviour(out_event = "BehaviourOutEvent", event_process = false)] + #[behaviour(out_event = "BehaviourOutEvent")] struct Foo { ping: libp2p::ping::Ping, identify: libp2p::identify::Identify, diff --git a/swarm/CHANGELOG.md b/swarm/CHANGELOG.md index 281f6ca3bc7..5894591c3bd 100644 --- a/swarm/CHANGELOG.md +++ b/swarm/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.31.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. - Provide default implementations for all functions of `NetworkBehaviour`, @@ -13,9 +16,37 @@ - Require `ProtocolsHandler::{InEvent,OutEvent,Error}` to implement `Debug` (see [PR 2183]). -[PR 2150]: https://github.com/libp2p/rust-libp2p/pull/2150/ +- Implement `ProtocolsHandler` on `either::Either`representing either of two + `ProtocolsHandler` implementations (see [PR 2192]). + +- Require implementation to provide handler in + `NetworkBehaviourAction::DialPeer` and `NetworkBehaviourAction::DialAddress`. + Note that the handler is returned to the `NetworkBehaviour` on connection + failure and connection closing. Thus it can be used to carry state, which + otherwise would have to be tracked in the `NetworkBehaviour` itself. E.g. a + message destined to an unconnected peer can be included in the handler, and + thus directly send on connection success or extracted by the + `NetworkBehaviour` on connection failure (see [PR 2191]). + +- Include handler in `NetworkBehaviour::inject_dial_failure`, + `NetworkBehaviour::inject_connection_closed`, + `NetworkBehaviour::inject_listen_failure` (see [PR 2191]). + +- Include error in `NetworkBehaviour::inject_dial_failure` and call + `NetworkBehaviour::inject_dial_failure` on `DialPeerCondition` evaluating to + false. To emulate the previous behaviour, return early within + `inject_dial_failure` on `DialError::DialPeerConditionFalse`. See [PR 2191]. + +- Make `NetworkBehaviourAction` generic over `NetworkBehaviour::OutEvent` and + `NetworkBehaviour::ProtocolsHandler`. In most cases, change your generic type + parameters to `NetworkBehaviourAction`. See [PR 2191]. + +[PR 2150]: https://github.com/libp2p/rust-libp2p/pull/2150 [PR 2182]: https://github.com/libp2p/rust-libp2p/pull/2182 [PR 2183]: https://github.com/libp2p/rust-libp2p/pull/2183 +[PR 2192]: https://github.com/libp2p/rust-libp2p/pull/2192 +[PR 2191]: https://github.com/libp2p/rust-libp2p/pull/2191 # 0.30.0 [2021-07-12] diff --git a/swarm/Cargo.toml b/swarm/Cargo.toml index b7ea6525447..b8e5d77a242 100644 --- a/swarm/Cargo.toml +++ b/swarm/Cargo.toml @@ -12,7 +12,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] either = "1.6.0" futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../core" } +libp2p-core = { version = "0.30.0", path = "../core", default-features = false } log = "0.4" rand = "0.7" smallvec = "1.6.1" @@ -20,7 +20,6 @@ wasm-timer = "0.2" void = "1" [dev-dependencies] -libp2p-mplex = { path = "../muxers/mplex" } -libp2p-noise = { path = "../transports/noise" } +libp2p = { path = "../", default-features = false, features = ["yamux", "plaintext"] } quickcheck = "0.9.0" rand = "0.7.2" diff --git a/swarm/src/behaviour.rs b/swarm/src/behaviour.rs index a21c7a023b8..c75049e6a9b 100644 --- a/swarm/src/behaviour.rs +++ b/swarm/src/behaviour.rs @@ -19,13 +19,17 @@ // DEALINGS IN THE SOFTWARE. use crate::protocols_handler::{IntoProtocolsHandler, ProtocolsHandler}; -use crate::{AddressRecord, AddressScore}; +use crate::{AddressRecord, AddressScore, DialError}; use libp2p_core::{ connection::{ConnectionId, ListenerId}, ConnectedPoint, Multiaddr, PeerId, }; use std::{error, task::Context, task::Poll}; +/// Custom event that can be received by the [`ProtocolsHandler`]. +type THandlerInEvent = + <::Handler as ProtocolsHandler>::InEvent; + /// A behaviour for the network. Allows customizing the swarm. /// /// This trait has been designed to be composable. Multiple implementations can be combined into @@ -65,16 +69,20 @@ pub trait NetworkBehaviour: Send + 'static { /// Creates a new `ProtocolsHandler` for a connection with a peer. /// - /// Every time an incoming connection is opened, and every time we start dialing a node, this - /// method is called. + /// Every time an incoming connection is opened, and every time another [`NetworkBehaviour`] + /// emitted a dial request, this method is called. /// /// The returned object is a handler for that specific connection, and will be moved to a /// background task dedicated to that connection. /// - /// The network behaviour (ie. the implementation of this trait) and the handlers it has - /// spawned (ie. the objects returned by `new_handler`) can communicate by passing messages. - /// Messages sent from the handler to the behaviour are injected with `inject_event`, and - /// the behaviour can send a message to the handler by making `poll` return `SendEvent`. + /// The network behaviour (ie. the implementation of this trait) and the handlers it has spawned + /// (ie. the objects returned by `new_handler`) can communicate by passing messages. Messages + /// sent from the handler to the behaviour are injected with [`NetworkBehaviour::inject_event`], + /// and the behaviour can send a message to the handler by making [`NetworkBehaviour::poll`] + /// return [`NetworkBehaviourAction::NotifyHandler`]. + /// + /// Note that the handler is returned to the [`NetworkBehaviour`] on connection failure and + /// connection closing. fn new_handler(&mut self) -> Self::ProtocolsHandler; /// Addresses that this behaviour is aware of for this specific peer, and that may allow @@ -112,7 +120,14 @@ pub trait NetworkBehaviour: Send + 'static { /// A call to this method is always paired with an earlier call to /// `inject_connection_established` with the same peer ID, connection ID and /// endpoint. - fn inject_connection_closed(&mut self, _: &PeerId, _: &ConnectionId, _: &ConnectedPoint) {} + fn inject_connection_closed( + &mut self, + _: &PeerId, + _: &ConnectionId, + _: &ConnectedPoint, + _: ::Handler, + ) { + } /// Informs the behaviour that the [`ConnectedPoint`] of an existing connection has changed. fn inject_address_change( @@ -153,7 +168,26 @@ pub trait NetworkBehaviour: Send + 'static { /// /// The `peer_id` is guaranteed to be in a disconnected state. In other words, /// `inject_connected` has not been called, or `inject_disconnected` has been called since then. - fn inject_dial_failure(&mut self, _peer_id: &PeerId) {} + fn inject_dial_failure( + &mut self, + _peer_id: &PeerId, + _handler: Self::ProtocolsHandler, + _error: DialError, + ) { + } + + /// Indicates to the behaviour that an error happened on an incoming connection during its + /// initial handshake. + /// + /// This can include, for example, an error during the handshake of the encryption layer, or the + /// connection unexpectedly closed. + fn inject_listen_failure( + &mut self, + _local_addr: &Multiaddr, + _send_back_addr: &Multiaddr, + _handler: Self::ProtocolsHandler, + ) { + } /// Indicates to the behaviour that a new listener was created. fn inject_new_listener(&mut self, _id: ListenerId) {} @@ -182,8 +216,11 @@ pub trait NetworkBehaviour: Send + 'static { /// /// This API mimics the API of the `Stream` trait. The method may register the current task in /// order to wake it up at a later point in time. - fn poll(&mut self, cx: &mut Context<'_>, params: &mut impl PollParameters) - -> Poll::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>>; + fn poll( + &mut self, + cx: &mut Context<'_>, + params: &mut impl PollParameters, + ) -> Poll>; } /// Parameters passed to `poll()`, that the `NetworkBehaviour` has access to. @@ -228,15 +265,35 @@ pub trait NetworkBehaviourEventProcess { /// in whose context it is executing. /// /// [`Swarm`]: super::Swarm +// +// Note: `TInEvent` is needed to be able to implement +// [`NetworkBehaviourAction::map_in`], mapping the handler `InEvent` leaving the +// handler itself untouched. #[derive(Debug)] -pub enum NetworkBehaviourAction { +pub enum NetworkBehaviourAction< + TOutEvent, + THandler: IntoProtocolsHandler, + TInEvent = THandlerInEvent, +> { /// Instructs the `Swarm` to return an event when it is being polled. GenerateEvent(TOutEvent), /// Instructs the swarm to dial the given multiaddress optionally including a [`PeerId`]. + /// + /// On success, [`NetworkBehaviour::inject_connection_established`] is invoked. + /// On failure, [`NetworkBehaviour::inject_dial_failure`] is invoked. + /// + /// Note that the provided handler is returned to the [`NetworkBehaviour`] on connection failure + /// and connection closing. Thus it can be used to carry state, which otherwise would have to be + /// tracked in the [`NetworkBehaviour`] itself. E.g. a message destined to an unconnected peer + /// can be included in the handler, and thus directly send on connection success or extracted by + /// the [`NetworkBehaviour`] on connection failure. See [`NetworkBehaviourAction::DialPeer`] for + /// example. DialAddress { /// The address to dial. address: Multiaddr, + /// The handler to be used to handle the connection to the peer. + handler: THandler, }, /// Instructs the swarm to dial a known `PeerId`. @@ -247,13 +304,194 @@ pub enum NetworkBehaviourAction { /// If we were already trying to dial this node, the addresses that are not yet in the queue of /// addresses to try are added back to this queue. /// - /// On success, [`NetworkBehaviour::inject_connected`] is invoked. + /// On success, [`NetworkBehaviour::inject_connection_established`] is invoked. /// On failure, [`NetworkBehaviour::inject_dial_failure`] is invoked. + /// + /// Note that the provided handler is returned to the [`NetworkBehaviour`] on connection failure + /// and connection closing. Thus it can be used to carry state, which otherwise would have to be + /// tracked in the [`NetworkBehaviour`] itself. E.g. a message destined to an unconnected peer + /// can be included in the handler, and thus directly send on connection success or extracted by + /// the [`NetworkBehaviour`] on connection failure. + /// + /// # Example + /// + /// ```rust + /// # use futures::executor::block_on; + /// # use futures::stream::StreamExt; + /// # use libp2p::core::connection::ConnectionId; + /// # use libp2p::core::identity; + /// # use libp2p::core::transport::{MemoryTransport, Transport}; + /// # use libp2p::core::upgrade::{self, DeniedUpgrade, InboundUpgrade, OutboundUpgrade}; + /// # use libp2p::core::PeerId; + /// # use libp2p::plaintext::PlainText2Config; + /// # use libp2p::swarm::{ + /// # DialError, DialPeerCondition, IntoProtocolsHandler, KeepAlive, NegotiatedSubstream, + /// # NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler, + /// # ProtocolsHandlerEvent, ProtocolsHandlerUpgrErr, SubstreamProtocol, Swarm, SwarmEvent, + /// # }; + /// # use libp2p::yamux; + /// # use std::collections::VecDeque; + /// # use std::task::{Context, Poll}; + /// # use void::Void; + /// # + /// # let local_key = identity::Keypair::generate_ed25519(); + /// # let local_public_key = local_key.public(); + /// # let local_peer_id = PeerId::from(local_public_key.clone()); + /// # + /// # let transport = MemoryTransport::default() + /// # .upgrade(upgrade::Version::V1) + /// # .authenticate(PlainText2Config { local_public_key }) + /// # .multiplex(yamux::YamuxConfig::default()) + /// # .boxed(); + /// # + /// # let mut swarm = Swarm::new(transport, MyBehaviour::default(), local_peer_id); + /// # + /// // Super precious message that we should better not lose. + /// let message = PreciousMessage("My precious message".to_string()); + /// + /// // Unfortunately this peer is offline, thus sending our message to it will fail. + /// let offline_peer = PeerId::random(); + /// + /// // Let's send it anyways. We should get it back in case connecting to the peer fails. + /// swarm.behaviour_mut().send(offline_peer, message); + /// + /// block_on(async { + /// // As expected, sending failed. But great news, we got our message back. + /// matches!( + /// swarm.next().await.expect("Infinite stream"), + /// SwarmEvent::Behaviour(PreciousMessage(_)) + /// ); + /// }); + /// + /// # #[derive(Default)] + /// # struct MyBehaviour { + /// # outbox_to_swarm: VecDeque>, + /// # } + /// # + /// # impl MyBehaviour { + /// # fn send(&mut self, peer_id: PeerId, msg: PreciousMessage) { + /// # self.outbox_to_swarm + /// # .push_back(NetworkBehaviourAction::DialPeer { + /// # peer_id, + /// # condition: DialPeerCondition::Always, + /// # handler: MyHandler { message: Some(msg) }, + /// # }); + /// # } + /// # } + /// # + /// impl NetworkBehaviour for MyBehaviour { + /// # type ProtocolsHandler = MyHandler; + /// # type OutEvent = PreciousMessage; + /// # + /// # fn new_handler(&mut self) -> Self::ProtocolsHandler { + /// # MyHandler { message: None } + /// # } + /// # + /// # + /// # fn inject_event( + /// # &mut self, + /// # _: PeerId, + /// # _: ConnectionId, + /// # _: <::Handler as ProtocolsHandler>::OutEvent, + /// # ) { + /// # unreachable!(); + /// # } + /// # + /// fn inject_dial_failure( + /// &mut self, + /// _: &PeerId, + /// handler: Self::ProtocolsHandler, + /// _: DialError, + /// ) { + /// // As expected, sending the message failed. But lucky us, we got the handler back, thus + /// // the precious message is not lost and we can return it back to the user. + /// let msg = handler.message.unwrap(); + /// self.outbox_to_swarm + /// .push_back(NetworkBehaviourAction::GenerateEvent(msg)) + /// } + /// # + /// # fn poll( + /// # &mut self, + /// # _: &mut Context<'_>, + /// # _: &mut impl PollParameters, + /// # ) -> Poll> { + /// # if let Some(action) = self.outbox_to_swarm.pop_front() { + /// # return Poll::Ready(action); + /// # } + /// # Poll::Pending + /// # } + /// } + /// + /// # struct MyHandler { + /// # message: Option, + /// # } + /// # + /// # impl ProtocolsHandler for MyHandler { + /// # type InEvent = Void; + /// # type OutEvent = Void; + /// # type Error = Void; + /// # type InboundProtocol = DeniedUpgrade; + /// # type OutboundProtocol = DeniedUpgrade; + /// # type InboundOpenInfo = (); + /// # type OutboundOpenInfo = Void; + /// # + /// # fn listen_protocol( + /// # &self, + /// # ) -> SubstreamProtocol { + /// # SubstreamProtocol::new(DeniedUpgrade, ()) + /// # } + /// # + /// # fn inject_fully_negotiated_inbound( + /// # &mut self, + /// # _: >::Output, + /// # _: Self::InboundOpenInfo, + /// # ) { + /// # } + /// # + /// # fn inject_fully_negotiated_outbound( + /// # &mut self, + /// # _: >::Output, + /// # _: Self::OutboundOpenInfo, + /// # ) { + /// # } + /// # + /// # fn inject_event(&mut self, _event: Self::InEvent) {} + /// # + /// # fn inject_dial_upgrade_error( + /// # &mut self, + /// # _: Self::OutboundOpenInfo, + /// # _: ProtocolsHandlerUpgrErr, + /// # ) { + /// # } + /// # + /// # fn connection_keep_alive(&self) -> KeepAlive { + /// # KeepAlive::Yes + /// # } + /// # + /// # fn poll( + /// # &mut self, + /// # _: &mut Context<'_>, + /// # ) -> Poll< + /// # ProtocolsHandlerEvent< + /// # Self::OutboundProtocol, + /// # Self::OutboundOpenInfo, + /// # Self::OutEvent, + /// # Self::Error, + /// # >, + /// # > { + /// # todo!("If `Self::message.is_some()` send the message to the remote.") + /// # } + /// # } + /// # #[derive(Debug, PartialEq, Eq)] + /// # struct PreciousMessage(String); + /// ``` DialPeer { /// The peer to try reach. peer_id: PeerId, /// The condition for initiating a new dialing attempt. condition: DialPeerCondition, + /// The handler to be used to handle the connection to the peer. + handler: THandler, }, /// Instructs the `Swarm` to send an event to the handler dedicated to a @@ -314,17 +552,28 @@ pub enum NetworkBehaviourAction { }, } -impl NetworkBehaviourAction { +impl + NetworkBehaviourAction +{ /// Map the handler event. - pub fn map_in(self, f: impl FnOnce(TInEvent) -> E) -> NetworkBehaviourAction { + pub fn map_in( + self, + f: impl FnOnce(TInEventOld) -> TInEventNew, + ) -> NetworkBehaviourAction { match self { NetworkBehaviourAction::GenerateEvent(e) => NetworkBehaviourAction::GenerateEvent(e), - NetworkBehaviourAction::DialAddress { address } => { - NetworkBehaviourAction::DialAddress { address } - } - NetworkBehaviourAction::DialPeer { peer_id, condition } => { - NetworkBehaviourAction::DialPeer { peer_id, condition } + NetworkBehaviourAction::DialAddress { address, handler } => { + NetworkBehaviourAction::DialAddress { address, handler } } + NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + } => NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + }, NetworkBehaviourAction::NotifyHandler { peer_id, handler, @@ -346,17 +595,79 @@ impl NetworkBehaviourAction { }, } } +} +impl NetworkBehaviourAction { /// Map the event the swarm will return. - pub fn map_out(self, f: impl FnOnce(TOutEvent) -> E) -> NetworkBehaviourAction { + pub fn map_out(self, f: impl FnOnce(TOutEvent) -> E) -> NetworkBehaviourAction { match self { NetworkBehaviourAction::GenerateEvent(e) => NetworkBehaviourAction::GenerateEvent(f(e)), - NetworkBehaviourAction::DialAddress { address } => { - NetworkBehaviourAction::DialAddress { address } + NetworkBehaviourAction::DialAddress { address, handler } => { + NetworkBehaviourAction::DialAddress { address, handler } } - NetworkBehaviourAction::DialPeer { peer_id, condition } => { - NetworkBehaviourAction::DialPeer { peer_id, condition } + NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + } => NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + }, + NetworkBehaviourAction::NotifyHandler { + peer_id, + handler, + event, + } => NetworkBehaviourAction::NotifyHandler { + peer_id, + handler, + event, + }, + NetworkBehaviourAction::ReportObservedAddr { address, score } => { + NetworkBehaviourAction::ReportObservedAddr { address, score } + } + NetworkBehaviourAction::CloseConnection { + peer_id, + connection, + } => NetworkBehaviourAction::CloseConnection { + peer_id, + connection, + }, + } + } +} + +impl NetworkBehaviourAction +where + THandlerOld: IntoProtocolsHandler, + ::Handler: ProtocolsHandler, +{ + /// Map the handler. + pub fn map_handler( + self, + f: impl FnOnce(THandlerOld) -> THandlerNew, + ) -> NetworkBehaviourAction + where + THandlerNew: IntoProtocolsHandler, + ::Handler: ProtocolsHandler, + { + match self { + NetworkBehaviourAction::GenerateEvent(e) => NetworkBehaviourAction::GenerateEvent(e), + NetworkBehaviourAction::DialAddress { address, handler } => { + NetworkBehaviourAction::DialAddress { + address, + handler: f(handler), + } } + NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + } => NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler: f(handler), + }, NetworkBehaviourAction::NotifyHandler { peer_id, handler, @@ -392,7 +703,6 @@ pub enum NotifyHandler { /// The available conditions under which a new dialing attempt to /// a peer is initiated when requested by [`NetworkBehaviourAction::DialPeer`]. #[derive(Debug, Copy, Clone)] -#[non_exhaustive] pub enum DialPeerCondition { /// A new dialing attempt is initiated _only if_ the peer is currently /// considered disconnected, i.e. there is no established connection diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 0bc718690be..a903113c2a9 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -82,8 +82,8 @@ use libp2p_core::{ }, muxing::StreamMuxerBox, network::{ - self, peer::ConnectedPeer, ConnectionLimits, Network, NetworkConfig, NetworkEvent, - NetworkInfo, + self, peer::ConnectedPeer, ConnectionLimits, DialAttemptsRemaining, Network, NetworkConfig, + NetworkEvent, NetworkInfo, }, transport::{self, TransportError}, upgrade::ProtocolName, @@ -331,19 +331,40 @@ where /// Initiates a new dialing attempt to the given address. pub fn dial_addr(&mut self, addr: Multiaddr) -> Result<(), DialError> { - let handler = self - .behaviour - .new_handler() + let handler = self.behaviour.new_handler(); + self.dial_addr_with_handler(addr, handler) + .map_err(DialError::from_network_dial_error) + .map_err(|(e, _)| e) + } + + fn dial_addr_with_handler( + &mut self, + addr: Multiaddr, + handler: ::ProtocolsHandler, + ) -> Result<(), network::DialError>>> { + let handler = handler .into_node_handler_builder() .with_substream_upgrade_protocol_override(self.substream_upgrade_protocol_override); - Ok(self.network.dial(&addr, handler).map(|_id| ())?) + + self.network.dial(&addr, handler).map(|_id| ()) } /// Initiates a new dialing attempt to the given peer. pub fn dial(&mut self, peer_id: &PeerId) -> Result<(), DialError> { + let handler = self.behaviour.new_handler(); + self.dial_with_handler(peer_id, handler) + } + + fn dial_with_handler( + &mut self, + peer_id: &PeerId, + handler: ::ProtocolsHandler, + ) -> Result<(), DialError> { if self.banned_peers.contains(peer_id) { - self.behaviour.inject_dial_failure(peer_id); - return Err(DialError::Banned); + let error = DialError::Banned; + self.behaviour + .inject_dial_failure(peer_id, handler, error.clone()); + return Err(error); } let self_listening = &self.listened_addrs; @@ -353,31 +374,31 @@ where .into_iter() .filter(|a| !self_listening.contains(a)); - let result = if let Some(first) = addrs.next() { - let handler = self - .behaviour - .new_handler() - .into_node_handler_builder() - .with_substream_upgrade_protocol_override(self.substream_upgrade_protocol_override); - self.network - .peer(*peer_id) - .dial(first, addrs, handler) - .map(|_| ()) - .map_err(DialError::from) - } else { - Err(DialError::NoAddresses) + let first = match addrs.next() { + Some(first) => first, + None => { + let error = DialError::NoAddresses; + self.behaviour + .inject_dial_failure(peer_id, handler, error.clone()); + return Err(error); + } }; - if let Err(error) = &result { - log::debug!( - "New dialing attempt to peer {:?} failed: {:?}.", - peer_id, - error - ); - self.behaviour.inject_dial_failure(&peer_id); + let handler = handler + .into_node_handler_builder() + .with_substream_upgrade_protocol_override(self.substream_upgrade_protocol_override); + match self.network.peer(*peer_id).dial(first, addrs, handler) { + Ok(_connection_id) => Ok(()), + Err(error) => { + let (error, handler) = DialError::from_network_dial_error(error); + self.behaviour.inject_dial_failure( + peer_id, + handler.into_protocols_handler(), + error.clone(), + ); + Err(error) + } } - - result } /// Returns an iterator that produces the list of addresses we're listening on. @@ -568,6 +589,7 @@ where connected, error, num_established, + handler, }) => { if let Some(error) = error.as_ref() { log::debug!("Connection {:?} closed: {:?}", connected, error); @@ -576,8 +598,12 @@ where } let peer_id = connected.peer_id; let endpoint = connected.endpoint; - this.behaviour - .inject_connection_closed(&peer_id, &id, &endpoint); + this.behaviour.inject_connection_closed( + &peer_id, + &id, + &endpoint, + handler.into_protocols_handler(), + ); if num_established == 0 { this.behaviour.inject_disconnected(&peer_id); } @@ -668,8 +694,14 @@ where local_addr, send_back_addr, error, + handler, }) => { log::debug!("Incoming connection failed: {:?}", error); + this.behaviour.inject_listen_failure( + &local_addr, + &send_back_addr, + handler.into_protocols_handler(), + ); return Poll::Ready(SwarmEvent::IncomingConnectionError { local_addr, send_back_addr, @@ -682,19 +714,34 @@ where error, attempts_remaining, }) => { - log::debug!( - "Connection attempt to {:?} via {:?} failed with {:?}. Attempts remaining: {}.", - peer_id, multiaddr, error, attempts_remaining); this.behaviour .inject_addr_reach_failure(Some(&peer_id), &multiaddr, &error); - if attempts_remaining == 0 { - this.behaviour.inject_dial_failure(&peer_id); + + let num_remaining: u32; + match attempts_remaining { + DialAttemptsRemaining::Some(n) => { + num_remaining = n.into(); + } + DialAttemptsRemaining::None(handler) => { + num_remaining = 0; + this.behaviour.inject_dial_failure( + &peer_id, + handler.into_protocols_handler(), + DialError::UnreachableAddr(multiaddr.clone()), + ); + } } + + log::debug!( + "Connection attempt to {:?} via {:?} failed with {:?}. Attempts remaining: {}.", + peer_id, multiaddr, error, num_remaining, + ); + return Poll::Ready(SwarmEvent::UnreachableAddr { peer_id, address: multiaddr, error, - attempts_remaining, + attempts_remaining: num_remaining, }); } Poll::Ready(NetworkEvent::UnknownPeerDialError { @@ -761,44 +808,48 @@ where Poll::Ready(NetworkBehaviourAction::GenerateEvent(event)) => { return Poll::Ready(SwarmEvent::Behaviour(event)) } - Poll::Ready(NetworkBehaviourAction::DialAddress { address }) => { - let _ = Swarm::dial_addr(&mut *this, address); + Poll::Ready(NetworkBehaviourAction::DialAddress { address, handler }) => { + let _ = Swarm::dial_addr_with_handler(&mut *this, address, handler); } - Poll::Ready(NetworkBehaviourAction::DialPeer { peer_id, condition }) => { - if this.banned_peers.contains(&peer_id) { - this.behaviour.inject_dial_failure(&peer_id); + Poll::Ready(NetworkBehaviourAction::DialPeer { + peer_id, + condition, + handler, + }) => { + let condition_matched = match condition { + DialPeerCondition::Disconnected => this.network.is_disconnected(&peer_id), + DialPeerCondition::NotDialing => !this.network.is_dialing(&peer_id), + DialPeerCondition::Always => true, + }; + if condition_matched { + if Swarm::dial_with_handler(this, &peer_id, handler).is_ok() { + return Poll::Ready(SwarmEvent::Dialing(peer_id)); + } } else { - let condition_matched = match condition { - DialPeerCondition::Disconnected => { - this.network.is_disconnected(&peer_id) - } - DialPeerCondition::NotDialing => !this.network.is_dialing(&peer_id), - DialPeerCondition::Always => true, - }; - if condition_matched { - if Swarm::dial(this, &peer_id).is_ok() { - return Poll::Ready(SwarmEvent::Dialing(peer_id)); - } - } else { - // Even if the condition for a _new_ dialing attempt is not met, - // we always add any potentially new addresses of the peer to an - // ongoing dialing attempt, if there is one. - log::trace!( - "Condition for new dialing attempt to {:?} not met: {:?}", - peer_id, - condition - ); - let self_listening = &this.listened_addrs; - if let Some(mut peer) = this.network.peer(peer_id).into_dialing() { - let addrs = this.behaviour.addresses_of_peer(peer.id()); - let mut attempt = peer.some_attempt(); - for a in addrs { - if !self_listening.contains(&a) { - attempt.add_address(a); - } + // Even if the condition for a _new_ dialing attempt is not met, + // we always add any potentially new addresses of the peer to an + // ongoing dialing attempt, if there is one. + log::trace!( + "Condition for new dialing attempt to {:?} not met: {:?}", + peer_id, + condition + ); + let self_listening = &this.listened_addrs; + if let Some(mut peer) = this.network.peer(peer_id).into_dialing() { + let addrs = this.behaviour.addresses_of_peer(peer.id()); + let mut attempt = peer.some_attempt(); + for a in addrs { + if !self_listening.contains(&a) { + attempt.add_address(a); } } } + + this.behaviour.inject_dial_failure( + &peer_id, + handler, + DialError::DialPeerConditionFalse(condition), + ); } } Poll::Ready(NetworkBehaviourAction::NotifyHandler { @@ -1000,7 +1051,7 @@ impl<'a> PollParameters for SwarmPollParameters<'a> { } fn local_peer_id(&self) -> &PeerId { - &self.local_peer_id + self.local_peer_id } } @@ -1148,8 +1199,8 @@ where } } -/// The possible failures of [`Swarm::dial`]. -#[derive(Debug)] +/// The possible failures of dialing. +#[derive(Debug, Clone)] pub enum DialError { /// The peer is currently banned. Banned, @@ -1158,16 +1209,27 @@ pub enum DialError { ConnectionLimit(ConnectionLimit), /// The address given for dialing is invalid. InvalidAddress(Multiaddr), + /// Tried to dial an address but it ended up being unreachaable. + UnreachableAddr(Multiaddr), + /// The peer being dialed is the local peer and thus the dial was aborted. + LocalPeerId, /// [`NetworkBehaviour::addresses_of_peer`] returned no addresses /// for the peer to dial. NoAddresses, + /// The provided [`DialPeerCondition`] evaluated to false and thus the dial was aborted. + DialPeerConditionFalse(DialPeerCondition), } -impl From for DialError { - fn from(err: network::DialError) -> DialError { - match err { - network::DialError::ConnectionLimit(l) => DialError::ConnectionLimit(l), - network::DialError::InvalidAddress(a) => DialError::InvalidAddress(a), +impl DialError { + fn from_network_dial_error(error: network::DialError) -> (Self, THandler) { + match error { + network::DialError::ConnectionLimit { limit, handler } => { + (DialError::ConnectionLimit(limit), handler) + } + network::DialError::InvalidAddress { address, handler } => { + (DialError::InvalidAddress(address), handler) + } + network::DialError::LocalPeerId { handler } => (DialError::LocalPeerId, handler), } } } @@ -1177,8 +1239,17 @@ impl fmt::Display for DialError { match self { DialError::ConnectionLimit(err) => write!(f, "Dial error: {}", err), DialError::NoAddresses => write!(f, "Dial error: no addresses for peer."), + DialError::LocalPeerId => write!(f, "Dial error: tried to dial local peer id."), DialError::InvalidAddress(a) => write!(f, "Dial error: invalid address: {}", a), + DialError::UnreachableAddr(a) => write!(f, "Dial error: unreachable address: {}", a), DialError::Banned => write!(f, "Dial error: peer is banned."), + DialError::DialPeerConditionFalse(c) => { + write!( + f, + "Dial error: condition {:?} for dialing peer was false.", + c + ) + } } } } @@ -1188,8 +1259,11 @@ impl error::Error for DialError { match self { DialError::ConnectionLimit(err) => Some(err), DialError::InvalidAddress(_) => None, + DialError::UnreachableAddr(_) => None, + DialError::LocalPeerId => None, DialError::NoAddresses => None, DialError::Banned => None, + DialError::DialPeerConditionFalse(_) => None, } } } @@ -1241,12 +1315,7 @@ impl NetworkBehaviour for DummyBehaviour { &mut self, _: &mut Context<'_>, _: &mut impl PollParameters, - ) -> Poll< - NetworkBehaviourAction< - ::InEvent, - Self::OutEvent, - >, - > { + ) -> Poll> { Poll::Pending } } @@ -1257,8 +1326,9 @@ mod tests { use crate::protocols_handler::DummyProtocolsHandler; use crate::test::{CallTraceBehaviour, MockBehaviour}; use futures::{executor, future}; - use libp2p_core::{identity, multiaddr, transport, upgrade}; - use libp2p_noise as noise; + use libp2p::core::{identity, multiaddr, transport, upgrade}; + use libp2p::plaintext; + use libp2p::yamux; // Test execution state. // Connection => Disconnecting => Connecting. @@ -1274,17 +1344,16 @@ mod tests { O: Send + 'static, { let id_keys = identity::Keypair::generate_ed25519(); - let pubkey = id_keys.public(); - let noise_keys = noise::Keypair::::new() - .into_authentic(&id_keys) - .unwrap(); + let local_public_key = id_keys.public(); let transport = transport::MemoryTransport::default() .upgrade(upgrade::Version::V1) - .authenticate(noise::NoiseConfig::xx(noise_keys).into_authenticated()) - .multiplex(libp2p_mplex::MplexConfig::new()) + .authenticate(plaintext::PlainText2Config { + local_public_key: local_public_key.clone(), + }) + .multiplex(yamux::YamuxConfig::default()) .boxed(); let behaviour = CallTraceBehaviour::new(MockBehaviour::new(handler_proto)); - SwarmBuilder::new(transport, behaviour, pubkey.into()).build() + SwarmBuilder::new(transport, behaviour, local_public_key.into()).build() } fn swarms_connected( @@ -1320,17 +1389,15 @@ mod tests { <::Handler as ProtocolsHandler>::OutEvent: Clone { for s in &[swarm1, swarm2] { - if s.behaviour.inject_connection_closed.len() < num_connections { - assert_eq!(s.behaviour.inject_disconnected.len(), 0); - } else { - assert_eq!(s.behaviour.inject_disconnected.len(), 1); - } assert_eq!(s.behaviour.inject_connection_established.len(), 0); assert_eq!(s.behaviour.inject_connected.len(), 0); } [swarm1, swarm2] .iter() .all(|s| s.behaviour.inject_connection_closed.len() == num_connections) + && [swarm1, swarm2] + .iter() + .all(|s| s.behaviour.inject_disconnected.len() == 1) } /// Establishes multiple connections between two peers, diff --git a/swarm/src/protocols_handler.rs b/swarm/src/protocols_handler.rs index 911693f32af..2000dbc3fb0 100644 --- a/swarm/src/protocols_handler.rs +++ b/swarm/src/protocols_handler.rs @@ -38,6 +38,7 @@ //! > the network as a whole, see the `NetworkBehaviour` trait. mod dummy; +pub mod either; mod map_in; mod map_out; pub mod multi; @@ -45,7 +46,7 @@ mod node_handler; mod one_shot; mod select; -pub use crate::upgrade::{InboundUpgradeSend, OutboundUpgradeSend, UpgradeInfoSend}; +pub use crate::upgrade::{InboundUpgradeSend, OutboundUpgradeSend, SendWrapper, UpgradeInfoSend}; use libp2p_core::{upgrade::UpgradeError, ConnectedPoint, Multiaddr, PeerId}; use std::{cmp::Ordering, error, fmt, task::Context, task::Poll, time::Duration}; @@ -522,10 +523,7 @@ pub enum KeepAlive { impl KeepAlive { /// Returns true for `Yes`, false otherwise. pub fn is_yes(&self) -> bool { - match *self { - KeepAlive::Yes => true, - _ => false, - } + matches!(*self, KeepAlive::Yes) } } diff --git a/swarm/src/protocols_handler/either.rs b/swarm/src/protocols_handler/either.rs new file mode 100644 index 00000000000..c51baadd2d7 --- /dev/null +++ b/swarm/src/protocols_handler/either.rs @@ -0,0 +1,305 @@ +// Copyright 2021 Protocol Labs. +// +// 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::protocols_handler::{ + IntoProtocolsHandler, KeepAlive, ProtocolsHandler, ProtocolsHandlerEvent, + ProtocolsHandlerUpgrErr, SubstreamProtocol, +}; +use crate::upgrade::{InboundUpgradeSend, OutboundUpgradeSend, SendWrapper}; +use either::Either; +use libp2p_core::either::{EitherError, EitherOutput}; +use libp2p_core::upgrade::{EitherUpgrade, UpgradeError}; +use libp2p_core::{ConnectedPoint, Multiaddr, PeerId}; +use std::task::{Context, Poll}; + +pub enum IntoEitherHandler { + Left(L), + Right(R), +} + +/// Implementation of a [`IntoProtocolsHandler`] that represents either of two [`IntoProtocolsHandler`] +/// implementations. +impl IntoProtocolsHandler for IntoEitherHandler +where + L: IntoProtocolsHandler, + R: IntoProtocolsHandler, +{ + type Handler = Either; + + fn into_handler(self, p: &PeerId, c: &ConnectedPoint) -> Self::Handler { + match self { + IntoEitherHandler::Left(into_handler) => Either::Left(into_handler.into_handler(p, c)), + IntoEitherHandler::Right(into_handler) => { + Either::Right(into_handler.into_handler(p, c)) + } + } + } + + fn inbound_protocol(&self) -> ::InboundProtocol { + match self { + IntoEitherHandler::Left(into_handler) => { + EitherUpgrade::A(SendWrapper(into_handler.inbound_protocol())) + } + IntoEitherHandler::Right(into_handler) => { + EitherUpgrade::B(SendWrapper(into_handler.inbound_protocol())) + } + } + } +} + +/// Implementation of a [`ProtocolsHandler`] that represents either of two [`ProtocolsHandler`] +/// implementations. +impl ProtocolsHandler for Either +where + L: ProtocolsHandler, + R: ProtocolsHandler, +{ + type InEvent = Either; + type OutEvent = Either; + type Error = Either; + type InboundProtocol = + EitherUpgrade, SendWrapper>; + type OutboundProtocol = + EitherUpgrade, SendWrapper>; + type InboundOpenInfo = Either; + type OutboundOpenInfo = Either; + + fn listen_protocol(&self) -> SubstreamProtocol { + match self { + Either::Left(a) => a + .listen_protocol() + .map_upgrade(|u| EitherUpgrade::A(SendWrapper(u))) + .map_info(Either::Left), + Either::Right(b) => b + .listen_protocol() + .map_upgrade(|u| EitherUpgrade::B(SendWrapper(u))) + .map_info(Either::Right), + } + } + + fn inject_fully_negotiated_outbound( + &mut self, + output: ::Output, + info: Self::OutboundOpenInfo, + ) { + match (self, output, info) { + (Either::Left(handler), EitherOutput::First(output), Either::Left(info)) => { + handler.inject_fully_negotiated_outbound(output, info) + } + (Either::Right(handler), EitherOutput::Second(output), Either::Right(info)) => { + handler.inject_fully_negotiated_outbound(output, info) + } + _ => unreachable!(), + } + } + + fn inject_fully_negotiated_inbound( + &mut self, + output: ::Output, + info: Self::InboundOpenInfo, + ) { + match (self, output, info) { + (Either::Left(handler), EitherOutput::First(output), Either::Left(info)) => { + handler.inject_fully_negotiated_inbound(output, info) + } + (Either::Right(handler), EitherOutput::Second(output), Either::Right(info)) => { + handler.inject_fully_negotiated_inbound(output, info) + } + _ => unreachable!(), + } + } + + fn inject_event(&mut self, event: Self::InEvent) { + match (self, event) { + (Either::Left(handler), Either::Left(event)) => handler.inject_event(event), + (Either::Right(handler), Either::Right(event)) => handler.inject_event(event), + _ => unreachable!(), + } + } + + fn inject_address_change(&mut self, addr: &Multiaddr) { + match self { + Either::Left(handler) => handler.inject_address_change(addr), + Either::Right(handler) => handler.inject_address_change(addr), + } + } + + fn inject_dial_upgrade_error( + &mut self, + info: Self::OutboundOpenInfo, + error: ProtocolsHandlerUpgrErr<::Error>, + ) { + match error { + ProtocolsHandlerUpgrErr::Timer => match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_dial_upgrade_error(info, ProtocolsHandlerUpgrErr::Timer); + } + (Either::Right(handler), Either::Right(info)) => { + handler.inject_dial_upgrade_error(info, ProtocolsHandlerUpgrErr::Timer); + } + _ => unreachable!(), + }, + ProtocolsHandlerUpgrErr::Timeout => match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_dial_upgrade_error(info, ProtocolsHandlerUpgrErr::Timeout); + } + (Either::Right(handler), Either::Right(info)) => { + handler.inject_dial_upgrade_error(info, ProtocolsHandlerUpgrErr::Timeout); + } + _ => unreachable!(), + }, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(error)) => match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_dial_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(error)), + ); + } + (Either::Right(handler), Either::Right(info)) => { + handler.inject_dial_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(error)), + ); + } + _ => unreachable!(), + }, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(EitherError::A(e))) => { + match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_dial_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(e)), + ); + } + _ => unreachable!(), + } + } + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(EitherError::B(e))) => { + match (self, info) { + (Either::Right(handler), Either::Right(info)) => { + handler.inject_dial_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(e)), + ); + } + _ => unreachable!(), + } + } + } + } + + fn inject_listen_upgrade_error( + &mut self, + info: Self::InboundOpenInfo, + error: ProtocolsHandlerUpgrErr<::Error>, + ) { + match error { + ProtocolsHandlerUpgrErr::Timer => match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_listen_upgrade_error(info, ProtocolsHandlerUpgrErr::Timer); + } + (Either::Right(handler), Either::Right(info)) => { + handler.inject_listen_upgrade_error(info, ProtocolsHandlerUpgrErr::Timer); + } + _ => unreachable!(), + }, + ProtocolsHandlerUpgrErr::Timeout => match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_listen_upgrade_error(info, ProtocolsHandlerUpgrErr::Timeout); + } + (Either::Right(handler), Either::Right(info)) => { + handler.inject_listen_upgrade_error(info, ProtocolsHandlerUpgrErr::Timeout); + } + _ => unreachable!(), + }, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(error)) => match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_listen_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(error)), + ); + } + (Either::Right(handler), Either::Right(info)) => { + handler.inject_listen_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Select(error)), + ); + } + _ => unreachable!(), + }, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(EitherError::A(e))) => { + match (self, info) { + (Either::Left(handler), Either::Left(info)) => { + handler.inject_listen_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(e)), + ); + } + _ => unreachable!(), + } + } + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(EitherError::B(e))) => { + match (self, info) { + (Either::Right(handler), Either::Right(info)) => { + handler.inject_listen_upgrade_error( + info, + ProtocolsHandlerUpgrErr::Upgrade(UpgradeError::Apply(e)), + ); + } + _ => unreachable!(), + } + } + } + } + + fn connection_keep_alive(&self) -> KeepAlive { + match self { + Either::Left(handler) => handler.connection_keep_alive(), + Either::Right(handler) => handler.connection_keep_alive(), + } + } + + fn poll( + &mut self, + cx: &mut Context<'_>, + ) -> Poll< + ProtocolsHandlerEvent< + Self::OutboundProtocol, + Self::OutboundOpenInfo, + Self::OutEvent, + Self::Error, + >, + > { + let event = match self { + Either::Left(handler) => futures::ready!(handler.poll(cx)) + .map_custom(Either::Left) + .map_close(Either::Left) + .map_protocol(|p| EitherUpgrade::A(SendWrapper(p))) + .map_outbound_open_info(Either::Left), + Either::Right(handler) => futures::ready!(handler.poll(cx)) + .map_custom(Either::Right) + .map_close(Either::Right) + .map_protocol(|p| EitherUpgrade::B(SendWrapper(p))) + .map_outbound_open_info(Either::Right), + }; + + Poll::Ready(event) + } +} diff --git a/swarm/src/protocols_handler/node_handler.rs b/swarm/src/protocols_handler/node_handler.rs index edb383282cd..8254968c6e8 100644 --- a/swarm/src/protocols_handler/node_handler.rs +++ b/swarm/src/protocols_handler/node_handler.rs @@ -65,6 +65,10 @@ where self.substream_upgrade_protocol_override = version; self } + + pub(crate) fn into_protocols_handler(self) -> TIntoProtoHandler { + self.handler + } } impl IntoConnectionHandler @@ -130,6 +134,12 @@ where substream_upgrade_protocol_override: Option, } +impl NodeHandlerWrapper { + pub(crate) fn into_protocols_handler(self) -> TProtoHandler { + self.handler + } +} + struct SubstreamUpgrade { user_data: Option, timeout: Delay, diff --git a/swarm/src/protocols_handler/select.rs b/swarm/src/protocols_handler/select.rs index b5891c25d1f..cce84928b5e 100644 --- a/swarm/src/protocols_handler/select.rs +++ b/swarm/src/protocols_handler/select.rs @@ -45,6 +45,10 @@ impl IntoProtocolsHandlerSelect { pub(crate) fn new(proto1: TProto1, proto2: TProto2) -> Self { IntoProtocolsHandlerSelect { proto1, proto2 } } + + pub fn into_inner(self) -> (TProto1, TProto2) { + (self.proto1, self.proto2) + } } impl IntoProtocolsHandler for IntoProtocolsHandlerSelect @@ -87,6 +91,10 @@ impl ProtocolsHandlerSelect { pub(crate) fn new(proto1: TProto1, proto2: TProto2) -> Self { ProtocolsHandlerSelect { proto1, proto2 } } + + pub fn into_inner(self) -> (TProto1, TProto2) { + (self.proto1, self.proto2) + } } impl ProtocolsHandler for ProtocolsHandlerSelect diff --git a/swarm/src/registry.rs b/swarm/src/registry.rs index 5819ecf1e4e..76892edf981 100644 --- a/swarm/src/registry.rs +++ b/swarm/src/registry.rs @@ -285,7 +285,7 @@ impl<'a> Iterator for AddressIter<'a> { } let item = &self.items[self.offset]; self.offset += 1; - Some(&item) + Some(item) } fn size_hint(&self) -> (usize, Option) { diff --git a/swarm/src/test.rs b/swarm/src/test.rs index 5cb05d7baf3..457701899a8 100644 --- a/swarm/src/test.rs +++ b/swarm/src/test.rs @@ -19,7 +19,7 @@ // DEALINGS IN THE SOFTWARE. use crate::{ - IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, PollParameters, + DialError, IntoProtocolsHandler, NetworkBehaviour, NetworkBehaviourAction, PollParameters, ProtocolsHandler, }; use libp2p_core::{ @@ -45,7 +45,7 @@ where /// The next action to return from `poll`. /// /// An action is only returned once. - pub next_action: Option>, + pub next_action: Option>, } impl MockBehaviour @@ -84,7 +84,7 @@ where &mut self, _: &mut Context, _: &mut impl PollParameters, - ) -> Poll> { + ) -> Poll> { self.next_action.take().map_or(Poll::Pending, Poll::Ready) } } @@ -202,10 +202,16 @@ where self.inner.inject_disconnected(peer); } - fn inject_connection_closed(&mut self, p: &PeerId, c: &ConnectionId, e: &ConnectedPoint) { + fn inject_connection_closed( + &mut self, + p: &PeerId, + c: &ConnectionId, + e: &ConnectedPoint, + handler: ::Handler, + ) { self.inject_connection_closed .push((p.clone(), c.clone(), e.clone())); - self.inner.inject_connection_closed(p, c, e); + self.inner.inject_connection_closed(p, c, e, handler); } fn inject_event( @@ -228,9 +234,14 @@ where self.inner.inject_addr_reach_failure(p, a, e); } - fn inject_dial_failure(&mut self, p: &PeerId) { + fn inject_dial_failure( + &mut self, + p: &PeerId, + handler: Self::ProtocolsHandler, + error: DialError, + ) { self.inject_dial_failure.push(p.clone()); - self.inner.inject_dial_failure(p); + self.inner.inject_dial_failure(p, handler, error); } fn inject_new_listener(&mut self, id: ListenerId) { @@ -268,12 +279,11 @@ where self.inner.inject_listener_closed(l, r); } - fn poll(&mut self, cx: &mut Context, args: &mut impl PollParameters) -> - Poll::Handler as ProtocolsHandler>::InEvent, - Self::OutEvent - >> - { + fn poll( + &mut self, + cx: &mut Context, + args: &mut impl PollParameters, + ) -> Poll> { self.poll += 1; self.inner.poll(cx, args) } diff --git a/swarm/src/toggle.rs b/swarm/src/toggle.rs index 5a86a4824ed..575d4e46809 100644 --- a/swarm/src/toggle.rs +++ b/swarm/src/toggle.rs @@ -24,7 +24,8 @@ use crate::protocols_handler::{ }; use crate::upgrade::{InboundUpgradeSend, OutboundUpgradeSend, SendWrapper}; use crate::{ - NetworkBehaviour, NetworkBehaviourAction, NetworkBehaviourEventProcess, PollParameters, + DialError, NetworkBehaviour, NetworkBehaviourAction, NetworkBehaviourEventProcess, + PollParameters, }; use either::Either; use libp2p_core::{ @@ -113,9 +114,12 @@ where peer_id: &PeerId, connection: &ConnectionId, endpoint: &ConnectedPoint, + handler: ::Handler, ) { if let Some(inner) = self.inner.as_mut() { - inner.inject_connection_closed(peer_id, connection, endpoint) + if let Some(handler) = handler.inner { + inner.inject_connection_closed(peer_id, connection, endpoint, handler) + } } } @@ -153,9 +157,29 @@ where } } - fn inject_dial_failure(&mut self, peer_id: &PeerId) { + fn inject_dial_failure( + &mut self, + peer_id: &PeerId, + handler: Self::ProtocolsHandler, + error: DialError, + ) { + if let Some(inner) = self.inner.as_mut() { + if let Some(handler) = handler.inner { + inner.inject_dial_failure(peer_id, handler, error) + } + } + } + + fn inject_listen_failure( + &mut self, + local_addr: &Multiaddr, + send_back_addr: &Multiaddr, + handler: Self::ProtocolsHandler, + ) { if let Some(inner) = self.inner.as_mut() { - inner.inject_dial_failure(peer_id) + if let Some(handler) = handler.inner { + inner.inject_listen_failure(local_addr, send_back_addr, handler) + } } } @@ -201,11 +225,15 @@ where } } - fn poll(&mut self, cx: &mut Context<'_>, params: &mut impl PollParameters) - -> Poll::Handler as ProtocolsHandler>::InEvent, Self::OutEvent>> - { + fn poll( + &mut self, + cx: &mut Context<'_>, + params: &mut impl PollParameters, + ) -> Poll> { if let Some(inner) = self.inner.as_mut() { - inner.poll(cx, params) + inner + .poll(cx, params) + .map(|action| action.map_handler(|h| ToggleIntoProtoHandler { inner: Some(h) })) } else { Poll::Pending } diff --git a/transports/deflate/CHANGELOG.md b/transports/deflate/CHANGELOG.md index 4a43c23b83b..72bdaba8828 100644 --- a/transports/deflate/CHANGELOG.md +++ b/transports/deflate/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.30.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.29.0 [2021-07-12] diff --git a/transports/deflate/Cargo.toml b/transports/deflate/Cargo.toml index 6ab23c1cef5..3ce579f0dea 100644 --- a/transports/deflate/Cargo.toml +++ b/transports/deflate/Cargo.toml @@ -11,7 +11,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } flate2 = "1.0" [dev-dependencies] diff --git a/transports/dns/CHANGELOG.md b/transports/dns/CHANGELOG.md index 96b4f41babe..41c0cc06b5e 100644 --- a/transports/dns/CHANGELOG.md +++ b/transports/dns/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.30.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.29.0 [2021-07-12] diff --git a/transports/dns/Cargo.toml b/transports/dns/Cargo.toml index df388345486..51c9a8fadd1 100644 --- a/transports/dns/Cargo.toml +++ b/transports/dns/Cargo.toml @@ -10,7 +10,7 @@ keywords = ["peer-to-peer", "libp2p", "networking"] categories = ["network-programming", "asynchronous"] [dependencies] -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } log = "0.4.1" futures = "0.3.1" trust-dns-resolver = { version = "0.20", default-features = false, features = ["system-config"] } diff --git a/transports/dns/src/lib.rs b/transports/dns/src/lib.rs index 6174c1e362c..2cfc27f93a6 100644 --- a/transports/dns/src/lib.rs +++ b/transports/dns/src/lib.rs @@ -76,7 +76,7 @@ pub use trust_dns_resolver::config::{ResolverConfig, ResolverOpts}; pub use trust_dns_resolver::error::{ResolveError, ResolveErrorKind}; /// The prefix for `dnsaddr` protocol TXT record lookups. -const DNSADDR_PREFIX: &'static str = "_dnsaddr."; +const DNSADDR_PREFIX: &str = "_dnsaddr."; /// The maximum number of dialing attempts to resolved addresses. const MAX_DIAL_ATTEMPTS: usize = 16; diff --git a/transports/noise/CHANGELOG.md b/transports/noise/CHANGELOG.md index 1df94010538..67830fb01fc 100644 --- a/transports/noise/CHANGELOG.md +++ b/transports/noise/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.33.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.32.0 [2021-07-12] diff --git a/transports/noise/Cargo.toml b/transports/noise/Cargo.toml index cf3880ec84f..201d7293aa4 100644 --- a/transports/noise/Cargo.toml +++ b/transports/noise/Cargo.toml @@ -12,7 +12,7 @@ bytes = "1" curve25519-dalek = "3.0.0" futures = "0.3.1" lazy_static = "1.2" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } log = "0.4" prost = "0.8" rand = "0.8.3" diff --git a/transports/noise/src/io.rs b/transports/noise/src/io.rs index 37e35ecbeee..afeee65363a 100644 --- a/transports/noise/src/io.rs +++ b/transports/noise/src/io.rs @@ -114,7 +114,7 @@ impl AsyncWrite for NoiseOutput { if this.send_offset == MAX_FRAME_LEN { trace!("write: sending {} bytes", MAX_FRAME_LEN); ready!(io.as_mut().poll_ready(cx))?; - io.as_mut().start_send(&frame_buf)?; + io.as_mut().start_send(frame_buf)?; this.send_offset = 0; } @@ -138,7 +138,7 @@ impl AsyncWrite for NoiseOutput { if this.send_offset > 0 { ready!(io.as_mut().poll_ready(cx))?; trace!("flush: sending {} bytes", this.send_offset); - io.as_mut().start_send(&frame_buf)?; + io.as_mut().start_send(frame_buf)?; this.send_offset = 0; } diff --git a/transports/noise/src/protocol/x25519.rs b/transports/noise/src/protocol/x25519.rs index c0d3936ee36..fb216909806 100644 --- a/transports/noise/src/protocol/x25519.rs +++ b/transports/noise/src/protocol/x25519.rs @@ -246,7 +246,7 @@ impl snow::types::Dh for Keypair { fn set(&mut self, sk: &[u8]) { let mut secret = [0u8; 32]; - secret.copy_from_slice(&sk); + secret.copy_from_slice(sk); self.secret = SecretKey(X25519(secret)); // Copy self.public = PublicKey(X25519(x25519(secret, X25519_BASEPOINT_BYTES))); secret.zeroize(); diff --git a/transports/noise/src/protocol/x25519_spec.rs b/transports/noise/src/protocol/x25519_spec.rs index 2f2c24237a6..839a5b77253 100644 --- a/transports/noise/src/protocol/x25519_spec.rs +++ b/transports/noise/src/protocol/x25519_spec.rs @@ -162,7 +162,7 @@ impl snow::types::Dh for Keypair { fn set(&mut self, sk: &[u8]) { let mut secret = [0u8; 32]; - secret.copy_from_slice(&sk); + secret.copy_from_slice(sk); self.secret = SecretKey(X25519Spec(secret)); // Copy self.public = PublicKey(X25519Spec(x25519(secret, X25519_BASEPOINT_BYTES))); secret.zeroize(); diff --git a/transports/plaintext/CHANGELOG.md b/transports/plaintext/CHANGELOG.md index 88f9e8d4766..dfbf8d07bf9 100644 --- a/transports/plaintext/CHANGELOG.md +++ b/transports/plaintext/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.30.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.29.0 [2021-07-12] diff --git a/transports/plaintext/Cargo.toml b/transports/plaintext/Cargo.toml index f9b206fb663..e0f7d6778da 100644 --- a/transports/plaintext/Cargo.toml +++ b/transports/plaintext/Cargo.toml @@ -13,7 +13,7 @@ categories = ["network-programming", "asynchronous"] bytes = "1" futures = "0.3.1" asynchronous-codec = "0.6" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } log = "0.4.8" prost = "0.8" unsigned-varint = { version = "0.7", features = ["asynchronous_codec"] } diff --git a/transports/pnet/CHANGELOG.md b/transports/pnet/CHANGELOG.md index b70e30c660e..b32b95f8739 100644 --- a/transports/pnet/CHANGELOG.md +++ b/transports/pnet/CHANGELOG.md @@ -1,3 +1,7 @@ +# 0.22.0 [unreleased] + +- Update dependencies. + # 0.21.0 [2021-05-17] - Update dependencies. diff --git a/transports/pnet/Cargo.toml b/transports/pnet/Cargo.toml index 3ed9acfba31..7b354c1e853 100644 --- a/transports/pnet/Cargo.toml +++ b/transports/pnet/Cargo.toml @@ -2,7 +2,7 @@ name = "libp2p-pnet" edition = "2018" description = "Private swarm support for libp2p" -version = "0.21.0" +version = "0.22.0" authors = ["Parity Technologies "] license = "MIT" repository = "https://github.com/libp2p/rust-libp2p" @@ -12,7 +12,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] futures = "0.3.1" log = "0.4.8" -salsa20 = "0.8" +salsa20 = "0.9" sha3 = "0.9" rand = "0.7" pin-project = "1.0.2" diff --git a/transports/tcp/CHANGELOG.md b/transports/tcp/CHANGELOG.md index 6972aa2a513..325672e194f 100644 --- a/transports/tcp/CHANGELOG.md +++ b/transports/tcp/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.30.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.29.0 [2021-07-12] diff --git a/transports/tcp/Cargo.toml b/transports/tcp/Cargo.toml index a4572cbcf70..7e56026a628 100644 --- a/transports/tcp/Cargo.toml +++ b/transports/tcp/Cargo.toml @@ -17,7 +17,7 @@ if-watch = { version = "0.2.0", optional = true } if-addrs = { version = "0.6.4", optional = true } ipnet = "2.0.0" libc = "0.2.80" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } log = "0.4.11" socket2 = { version = "0.4.0", features = ["all"] } tokio-crate = { package = "tokio", version = "1.0.1", default-features = false, features = ["net"], optional = true } diff --git a/transports/uds/CHANGELOG.md b/transports/uds/CHANGELOG.md index ce66f13cd8f..fe900a22918 100644 --- a/transports/uds/CHANGELOG.md +++ b/transports/uds/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.30.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.29.0 [2021-07-12] diff --git a/transports/uds/Cargo.toml b/transports/uds/Cargo.toml index 1a079ab8ac8..84aca6450a1 100644 --- a/transports/uds/Cargo.toml +++ b/transports/uds/Cargo.toml @@ -11,7 +11,7 @@ categories = ["network-programming", "asynchronous"] [target.'cfg(all(unix, not(target_os = "emscripten")))'.dependencies] async-std = { version = "1.6.2", optional = true } -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } log = "0.4.1" futures = "0.3.1" tokio = { version = "1.0.1", default-features = false, features = ["net"], optional = true } diff --git a/transports/wasm-ext/CHANGELOG.md b/transports/wasm-ext/CHANGELOG.md index a5047f28c94..278aaa01791 100644 --- a/transports/wasm-ext/CHANGELOG.md +++ b/transports/wasm-ext/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.30.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.29.0 [2021-07-12] diff --git a/transports/wasm-ext/Cargo.toml b/transports/wasm-ext/Cargo.toml index bc34c37472f..d5d0139ba03 100644 --- a/transports/wasm-ext/Cargo.toml +++ b/transports/wasm-ext/Cargo.toml @@ -12,7 +12,7 @@ categories = ["network-programming", "asynchronous"] [dependencies] futures = "0.3.1" js-sys = "0.3.50" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } parity-send-wrapper = "0.1.0" wasm-bindgen = "0.2.42" wasm-bindgen-futures = "0.4.4" diff --git a/transports/websocket/CHANGELOG.md b/transports/websocket/CHANGELOG.md index eddc2a8f2ac..c3cfd407eda 100644 --- a/transports/websocket/CHANGELOG.md +++ b/transports/websocket/CHANGELOG.md @@ -1,5 +1,8 @@ # 0.31.0 [unreleased] +- Make default features of `libp2p-core` optional. + [PR 2181](https://github.com/libp2p/rust-libp2p/pull/2181) + - Update dependencies. # 0.30.0 [2021-07-12] diff --git a/transports/websocket/Cargo.toml b/transports/websocket/Cargo.toml index 5fce54bc80a..a7feeac7c86 100644 --- a/transports/websocket/Cargo.toml +++ b/transports/websocket/Cargo.toml @@ -13,7 +13,7 @@ categories = ["network-programming", "asynchronous"] futures-rustls = "0.21" either = "1.5.3" futures = "0.3.1" -libp2p-core = { version = "0.30.0", path = "../../core" } +libp2p-core = { version = "0.30.0", path = "../../core", default-features = false } log = "0.4.8" quicksink = "0.1" rw-stream-sink = "0.2.0" diff --git a/transports/websocket/src/framed.rs b/transports/websocket/src/framed.rs index dc57cb8e220..52635ec5ba0 100644 --- a/transports/websocket/src/framed.rs +++ b/transports/websocket/src/framed.rs @@ -494,27 +494,15 @@ impl IncomingData { } pub fn is_binary(&self) -> bool { - if let IncomingData::Binary(_) = self { - true - } else { - false - } + matches!(self, IncomingData::Binary(_)) } pub fn is_text(&self) -> bool { - if let IncomingData::Text(_) = self { - true - } else { - false - } + matches!(self, IncomingData::Text(_)) } pub fn is_pong(&self) -> bool { - if let IncomingData::Pong(_) = self { - true - } else { - false - } + matches!(self, IncomingData::Pong(_)) } pub fn into_bytes(self) -> Vec { From 9b36bc1c87942c9781d8a73368c850b35aa0f83e Mon Sep 17 00:00:00 2001 From: David Craven Date: Sat, 2 Oct 2021 11:25:18 +0200 Subject: [PATCH 14/14] Update transports/quic/src/transport.rs Co-authored-by: Max Inden --- transports/quic/src/transport.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index ae2ca5c4ca6..0cd77026634 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -233,7 +233,7 @@ where } Poll::Ready(Err(err)) => Poll::Ready(Err(err.into())), Poll::Ready(Ok(_)) => { - panic!("muxer.incoming is set to false so no events can be produced"); + unreachable!("muxer.incoming is set to false so no events can be produced"); } } }