From 7431703b7e2617b731d4d63d2228018601337939 Mon Sep 17 00:00:00 2001 From: David Craven Date: Fri, 16 Jul 2021 16:01:46 +0200 Subject: [PATCH 001/218] 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 002/218] 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 003/218] 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 004/218] 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 005/218] 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 006/218] 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 007/218] 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 008/218] 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 009/218] 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 010/218] 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 011/218] 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 012/218] 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 013/218] 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 014/218] 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"); } } } From 7c33cdcbf765d7058bca175e70c3b73ea57801e9 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 14 Oct 2021 14:47:35 +0300 Subject: [PATCH 015/218] Add David's credentials to quic/src headers --- transports/quic/src/crypto.rs | 20 ++++++++++++++++++++ transports/quic/src/endpoint.rs | 20 ++++++++++++++++++++ transports/quic/src/lib.rs | 20 ++++++++++++++++++++ transports/quic/src/muxer.rs | 20 ++++++++++++++++++++ transports/quic/src/transport.rs | 20 ++++++++++++++++++++ 5 files changed, 100 insertions(+) diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index 81d1c8193b3..1af369095f2 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -1,3 +1,23 @@ +// Copyright 2021 David Craven. +// +// 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::PeerId; use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 3295b5bbf58..cb5cfc78f97 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -1,3 +1,23 @@ +// Copyright 2021 David Craven. +// +// 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::crypto::{Crypto, CryptoConfig}; use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 9185922c6cb..74b4abafc76 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -1,3 +1,23 @@ +// Copyright 2021 David Craven. +// +// 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. + mod crypto; mod endpoint; mod muxer; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 29f6b2a3ced..561213c12d2 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -1,3 +1,23 @@ +// Copyright 2021 David Craven. +// +// 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::crypto::Crypto; use crate::endpoint::ConnectionChannel; use async_io::Timer; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 0cd77026634..de038593be8 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -1,3 +1,23 @@ +// Copyright 2021 David Craven. +// +// 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::crypto::Crypto; use crate::endpoint::{EndpointConfig, TransportChannel}; use crate::muxer::QuicMuxer; From af73d9868c10bd3d61523a13e3b715083604234e Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 21 Oct 2021 15:09:50 +0300 Subject: [PATCH 016/218] Mirror quic features in the root Cargo.toml --- Cargo.toml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/Cargo.toml b/Cargo.toml index eca381b7825..3ea1f4632d9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,6 +49,8 @@ ping = ["libp2p-ping", "libp2p-metrics/ping"] plaintext = ["libp2p-plaintext"] pnet = ["libp2p-pnet"] quic = ["libp2p-quic"] +quic-noise = ["libp2p-quic", "libp2p-quic/noise"] +quic-tls = ["libp2p-quic", "libp2p-quic/tls"] relay = ["libp2p-relay"] request-response = ["libp2p-request-response"] rendezvous = ["libp2p-rendezvous"] From 1e0078c5011e7b95dc6724ae0df9516edb0ecd24 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 21 Oct 2021 16:07:19 +0300 Subject: [PATCH 017/218] Fix tests after merging master --- transports/quic/tests/smoke.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 3ee821e956a..36729b2327b 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -376,7 +376,7 @@ async fn dial_failure_tls() -> Result<()> { } match b.next().await { - Some(SwarmEvent::UnreachableAddr { .. }) => {} + Some(SwarmEvent::OutgoingConnectionError { .. }) => {} e => panic!("{:?}", e), }; From b30ae08f1dcb4472dc13c34ef169dc3e7aa101df Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Mon, 25 Oct 2021 15:52:01 +0300 Subject: [PATCH 018/218] Fix TLS 1.2 error msg --- transports/quic/src/tls/verifier.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 9dcf2ed2298..3427aa4275a 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -63,7 +63,7 @@ impl rustls::ServerCertVerifier for Libp2pServerCertificateVerifier { _dss: &DigitallySignedStruct, ) -> Result { Err(TLSError::PeerIncompatibleError( - "Only TLS 1.3 certificates are supported".to_string(), + "Only TLS 1.3 is supported".to_string(), )) } @@ -116,7 +116,7 @@ impl rustls::ClientCertVerifier for Libp2pClientCertificateVerifier { _dss: &DigitallySignedStruct, ) -> Result { Err(TLSError::PeerIncompatibleError( - "Only TLS 1.3 certificates are supported".to_string(), + "Only TLS 1.3 is supported".to_string(), )) } From 9f11c3002133b6340006cbc551f3e485c339565d Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 26 Oct 2021 12:09:45 +0300 Subject: [PATCH 019/218] Expect substreams when accessing by id in the inner struct --- transports/quic/src/muxer.rs | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 561213c12d2..423cb2cec9a 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -187,18 +187,16 @@ impl StreamMuxer for QuicMuxer { } 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(); - } + let substream = inner.substreams.get_mut(&id).expect("known substream; qed"); + 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(); - } + let substream = inner.substreams.get_mut(&id).expect("known substream; qed"); + if let Some(waker) = substream.write_waker.take() { + waker.wake(); } } Event::Stream(StreamEvent::Finished { id }) => { @@ -218,7 +216,7 @@ impl StreamMuxer for QuicMuxer { return Poll::Ready(Err(QuicMuxerError::StreamStopped { id, error_code })); } Event::Stream(StreamEvent::Available { dir: Dir::Bi }) => { - tracing::trace!("stream available"); + tracing::trace!("substream available"); if let Some(waker) = inner.pending_substreams.pop_front() { waker.wake(); } @@ -324,7 +322,7 @@ impl StreamMuxer for QuicMuxer { waker.wake(); } } - let substream = inner.substreams.get_mut(id).unwrap(); + let substream = inner.substreams.get_mut(id).expect("known substream; qed"); if pending && bytes == 0 { substream.read_waker = Some(cx.waker().clone()); Poll::Pending @@ -343,7 +341,7 @@ impl StreamMuxer for QuicMuxer { 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(); + let mut substream = inner.substreams.get_mut(id).expect("known substream; qed"); substream.write_waker = Some(cx.waker().clone()); Poll::Pending } From 9425cc5bb59b2195393f67e4b5f7f9e8990c8303 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 26 Oct 2021 14:29:10 +0300 Subject: [PATCH 020/218] Comment out _max_datagrams --- transports/quic/src/muxer.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 423cb2cec9a..9b279d8e29a 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -142,7 +142,8 @@ impl StreamMuxer for QuicMuxer { inner.connection.handle_event(event); } - let _max_datagrams = inner.endpoint.max_datagrams(); + // TODO: `quinn` got plans to add max_datagrams as an arg to `poll_transmit` + // let _max_datagrams = inner.endpoint.max_datagrams(); while let Some(transmit) = inner.connection.poll_transmit(now) { inner.endpoint.send_transmit(transmit); } From 25ee1fea8dcd767ab0a23ce0b18ec3dbc171db92 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 9 Nov 2021 17:52:49 +0300 Subject: [PATCH 021/218] Cut off noise crypto --- Cargo.toml | 2 - transports/quic/Cargo.toml | 24 ++++--- transports/quic/src/crypto.rs | 105 ++----------------------------- transports/quic/src/endpoint.rs | 1 - transports/quic/src/lib.rs | 7 --- transports/quic/src/transport.rs | 11 ---- transports/quic/tests/smoke.rs | 64 ------------------- 7 files changed, 15 insertions(+), 199 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 3ea1f4632d9..eca381b7825 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,8 +49,6 @@ ping = ["libp2p-ping", "libp2p-metrics/ping"] plaintext = ["libp2p-plaintext"] pnet = ["libp2p-pnet"] quic = ["libp2p-quic"] -quic-noise = ["libp2p-quic", "libp2p-quic/noise"] -quic-tls = ["libp2p-quic", "libp2p-quic/tls"] relay = ["libp2p-relay"] request-response = ["libp2p-request-response"] rendezvous = ["libp2p-rendezvous"] diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 669e9d63230..9683d2be3ae 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -3,36 +3,34 @@ name = "libp2p-quic" version = "0.6.0" authors = ["David Craven ", "Parity Technologies "] edition = "2018" -description = "TLS and Noise based QUIC transport implementation for libp2p" +description = "TLS based QUIC transport implementation for libp2p" repository = "https://github.com/libp2p/rust-libp2p" license = "MIT" [features] -noise = ["quinn-noise", "ed25519-dalek"] -tls = ["barebones-x509", "quinn-proto/tls-rustls", "rcgen", "ring", "rustls", "untrusted", "webpki", "yasna"] +tls = [] +default = ["tls"] [dependencies] async-global-executor = "2.0.2" async-io = "1.6.0" -barebones-x509 = { version = "0.5.0", optional = true, features = ["webpki", "rustls", "std"] } +barebones-x509 = { version = "0.5.0", features = ["webpki", "rustls", "std"] } bytes = "1.0.1" -ed25519-dalek = { version = "1.0.1", optional = true } futures = "0.3.15" if-watch = "0.2.2" libp2p-core = { version = "0.30.0-rc.1", 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 } -rcgen = { version = "0.8.11", optional = true } -ring = { version = "0.16.20", optional = true } -rustls = { version = "0.19.1", optional = true, features = ["dangerous_configuration"] } +quinn-proto = { version = "0.7.3", default-features = false, features = ["tls-rustls"] } +rcgen = { version = "0.8.11" } +ring = { version = "0.16.20" } +rustls = { version = "0.19.1", 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 } +untrusted = { version = "0.7.1" } +webpki = { version = "0.21.4", features = ["std"] } +yasna = { version = "0.4.0" } [dev-dependencies] anyhow = "1.0.41" diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index 1af369095f2..3313d294d5e 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -25,23 +25,10 @@ use std::sync::Arc; pub struct CryptoConfig { pub keypair: C::Keypair, - pub psk: Option<[u8; 32]>, pub keylogger: Option, pub transport: Arc, } -#[cfg(feature = "noise")] -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_public(&self) -> libp2p_core::identity::PublicKey; fn to_peer_id(&self) -> PeerId { @@ -49,23 +36,6 @@ pub trait ToLibp2p { } } -#[cfg(feature = "noise")] -impl ToLibp2p for ed25519_dalek::Keypair { - fn to_public(&self) -> libp2p_core::identity::PublicKey { - self.public.to_public() - } -} - -#[cfg(feature = "noise")] -impl ToLibp2p for ed25519_dalek::PublicKey { - 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_public(&self) -> libp2p_core::identity::PublicKey { @@ -80,10 +50,10 @@ pub trait Crypto: std::fmt::Debug + Clone + 'static { type PublicKey: Send + std::fmt::Debug + PartialEq; fn new_server_config( - config: &Arc>, + config: &CryptoConfig, ) -> ::ServerConfig; fn new_client_config( - config: &Arc>, + config: &CryptoConfig, remote_public: Self::PublicKey, ) -> ::ClientConfig; fn supported_quic_versions() -> Vec; @@ -93,71 +63,6 @@ pub trait Crypto: std::fmt::Debug + Clone + 'static { 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; - type Keypair = ed25519_dalek::Keypair; - type PublicKey = ed25519_dalek::PublicKey; - - fn new_server_config( - config: &Arc>, - ) -> ::ServerConfig { - Arc::new( - quinn_noise::NoiseServerConfig { - keypair: config.keypair.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: Self::PublicKey, - ) -> ::ClientConfig { - quinn_noise::NoiseClientConfig { - keypair: config.keypair.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 { - 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()) - } -} - #[cfg(feature = "tls")] #[derive(Clone, Copy, Debug)] pub struct TlsCrypto; @@ -170,9 +75,8 @@ impl Crypto for TlsCrypto { type PublicKey = libp2p_core::identity::PublicKey; fn new_server_config( - config: &Arc>, + config: &CryptoConfig, ) -> ::ServerConfig { - assert!(config.psk.is_none(), "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; @@ -181,10 +85,9 @@ impl Crypto for TlsCrypto { } fn new_client_config( - config: &Arc>, + config: &CryptoConfig, remote_public: Self::PublicKey, ) -> ::ClientConfig { - assert!(config.psk.is_none(), "invalid config"); let mut client = crate::tls::make_client_config(&config.keypair, remote_public.to_peer_id()) .expect("invalid config"); diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index cb5cfc78f97..d9c331337dd 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -189,7 +189,6 @@ impl EndpointConfig { let crypto_config = Arc::new(CryptoConfig { keypair: config.keypair, - psk: config.psk, keylogger: config.keylogger, transport: transport.clone(), }); diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 74b4abafc76..9c8f1b3857d 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -26,15 +26,11 @@ 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::crypto::ToLibp2p; pub use crate::muxer::{QuicMuxer, QuicMuxerError}; pub use crate::transport::{QuicDial, QuicTransport}; -#[cfg(feature = "noise")] -pub use quinn_noise::{KeyLog, KeyLogFile}; pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; use libp2p_core::transport::TransportError; @@ -45,7 +41,6 @@ use thiserror::Error; /// Quic configuration. pub struct QuicConfig { pub keypair: C::Keypair, - pub psk: Option<[u8; 32]>, pub transport: TransportConfig, pub keylogger: Option, } @@ -54,7 +49,6 @@ 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.to_public()) - .field("psk", &self.psk) .field("transport", &self.transport) .finish() } @@ -70,7 +64,6 @@ where pub fn new(keypair: C::Keypair) -> Self { Self { keypair, - psk: None, transport: TransportConfig::default(), keylogger: None, } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index de038593be8..9aabf5cb7ee 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -364,11 +364,6 @@ mod tests { ); } - #[cfg(feature = "noise")] - #[test] - fn multiaddr_to_udp_noise() { - multiaddr_to_udp_conversion::(); - } #[cfg(feature = "tls")] #[test] fn multiaddr_to_udp_tls() { @@ -396,10 +391,4 @@ mod tests { 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 36729b2327b..e68f6fd0567 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -15,21 +15,11 @@ 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<()> { - smoke::().await -} - #[cfg(feature = "tls")] #[async_std::test] async fn smoke_tls() -> Result<()> { @@ -40,13 +30,6 @@ 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 { @@ -291,53 +274,6 @@ impl RequestResponseCodec for PingCodec { } } -#[cfg(feature = "noise")] -#[async_std::test] -async fn dial_failure_noise() -> 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 = libp2p_quic::NoiseCrypto::generate_keypair(); - 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<()> { From f35fde1c88c74aff520379025650a27bc259b474 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 9 Nov 2021 18:18:53 +0300 Subject: [PATCH 022/218] Remove generic crypto code --- transports/quic/Cargo.toml | 4 -- transports/quic/src/crypto.rs | 83 +++++------------------ transports/quic/src/endpoint.rs | 99 +++++++++++++--------------- transports/quic/src/lib.rs | 31 ++++----- transports/quic/src/muxer.rs | 26 ++++---- transports/quic/src/transport.rs | 110 ++++++++++++------------------- transports/quic/tests/smoke.rs | 56 ++++------------ 7 files changed, 144 insertions(+), 265 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 9683d2be3ae..5e639c01a60 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -7,10 +7,6 @@ description = "TLS based QUIC transport implementation for libp2p" repository = "https://github.com/libp2p/rust-libp2p" license = "MIT" -[features] -tls = [] -default = ["tls"] - [dependencies] async-global-executor = "2.0.2" async-io = "1.6.0" diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index 3313d294d5e..9a0e5bfa613 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -18,65 +18,25 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use libp2p_core::PeerId; +use libp2p_core::identity::{PublicKey, Keypair}; use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; use std::sync::Arc; -pub struct CryptoConfig { - pub keypair: C::Keypair, - pub keylogger: Option, +pub struct CryptoConfig { + pub keypair: Keypair, + pub keylogger: Option>, pub transport: Arc, } -pub trait ToLibp2p { - fn to_public(&self) -> libp2p_core::identity::PublicKey; - fn to_peer_id(&self) -> PeerId { - self.to_public().to_peer_id() - } -} - -#[cfg(feature = "tls")] -impl ToLibp2p for libp2p_core::identity::Keypair { - 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: &CryptoConfig, - ) -> ::ServerConfig; - fn new_client_config( - config: &CryptoConfig, - 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; -} - -#[cfg(feature = "tls")] #[derive(Clone, Copy, Debug)] -pub struct TlsCrypto; +pub(crate) struct TlsCrypto; -#[cfg(feature = "tls")] -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; +impl TlsCrypto { - fn new_server_config( - config: &CryptoConfig, - ) -> ::ServerConfig { + pub fn new_server_config( + config: &CryptoConfig, + ) -> ::ServerConfig { 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; @@ -84,10 +44,10 @@ impl Crypto for TlsCrypto { Arc::new(server) } - fn new_client_config( - config: &CryptoConfig, - remote_public: Self::PublicKey, - ) -> ::ClientConfig { + pub fn new_client_config( + config: &CryptoConfig, + remote_public: PublicKey, + ) -> ::ClientConfig { let mut client = crate::tls::make_client_config(&config.keypair, remote_public.to_peer_id()) .expect("invalid config"); @@ -97,26 +57,15 @@ impl Crypto for TlsCrypto { Arc::new(client) } - fn supported_quic_versions() -> Vec { + pub fn supported_quic_versions() -> Vec { quinn_proto::DEFAULT_SUPPORTED_VERSIONS.to_vec() } - fn default_quic_version() -> u32 { + pub 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 extract_public_key(generic_key: libp2p_core::PublicKey) -> Option { - Some(generic_key) - } - - fn keylogger() -> Self::Keylogger { + pub fn keylogger() -> Arc { Arc::new(rustls::KeyLogFile::new()) } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index d9c331337dd..c2cb4f6a8f4 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -18,16 +18,17 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::crypto::{Crypto, CryptoConfig}; +use crate::crypto::{CryptoConfig, TlsCrypto}; use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; -use quinn_proto::crypto::Session; -use quinn_proto::generic::{ClientConfig, ServerConfig}; +use quinn_proto::{ServerConfig as QuinnServerConfig}; +use quinn_proto::generic::{ClientConfig as QuinnClientConfig}; use quinn_proto::{ ConnectionEvent, ConnectionHandle, DatagramEvent, EcnCodepoint, EndpointEvent, Transmit, }; +use libp2p_core::identity::PublicKey; use std::collections::{HashMap, VecDeque}; use std::io::IoSliceMut; use std::mem::MaybeUninit; @@ -40,15 +41,15 @@ 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. addr: SocketAddr, /// The remotes public key. - public_key: C::PublicKey, + public_key: PublicKey, /// Channel to return the result of the dialing to. - tx: oneshot::Sender, QuicError>>, + 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. @@ -61,19 +62,19 @@ enum ToEndpoint { } #[derive(Debug)] -pub struct TransportChannel { - tx: mpsc::UnboundedSender>, - rx: mpsc::Receiver, QuicError>>, +pub struct TransportChannel { + tx: mpsc::UnboundedSender, + rx: mpsc::Receiver>, port: u16, ty: SocketType, } -impl TransportChannel { +impl TransportChannel { pub fn dial( &mut self, addr: SocketAddr, - public_key: C::PublicKey, - ) -> oneshot::Receiver, QuicError>> { + public_key: PublicKey, + ) -> oneshot::Receiver> { let (tx, rx) = oneshot::channel(); let msg = ToEndpoint::Dial { addr, @@ -87,7 +88,7 @@ impl TransportChannel { pub fn poll_incoming( &mut self, cx: &mut Context, - ) -> Poll, QuicError>>> { + ) -> Poll>> { Pin::new(&mut self.rx).poll_next(cx) } @@ -101,15 +102,15 @@ impl TransportChannel { } #[derive(Debug)] -pub struct ConnectionChannel { +pub struct ConnectionChannel { id: ConnectionHandle, - tx: mpsc::UnboundedSender>, + 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), @@ -141,23 +142,23 @@ impl ConnectionChannel { } #[derive(Debug)] -struct EndpointChannel { - rx: mpsc::UnboundedReceiver>, - tx: mpsc::Sender, QuicError>>, +struct EndpointChannel { + rx: mpsc::UnboundedReceiver, + tx: mpsc::Sender>, port: u16, max_datagrams: usize, - connection_tx: mpsc::UnboundedSender>, + connection_tx: mpsc::UnboundedSender, } -impl EndpointChannel { - 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::Sender) { + ) -> (ConnectionChannel, mpsc::Sender) { let (tx, rx) = mpsc::channel(12); let channel = ConnectionChannel { id, @@ -170,19 +171,19 @@ impl EndpointChannel { } } -type QuinnEndpointConfig = quinn_proto::generic::EndpointConfig; -type QuinnEndpoint = quinn_proto::generic::Endpoint; +type QuinnEndpointConfig = quinn_proto::EndpointConfig; +type QuinnEndpoint = quinn_proto::Endpoint; -pub struct EndpointConfig { +pub struct EndpointConfig { socket: UdpSocket, - endpoint: QuinnEndpoint, + endpoint: QuinnEndpoint, port: u16, - crypto_config: Arc>, + 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); @@ -193,13 +194,13 @@ impl EndpointConfig { transport: transport.clone(), }); - let mut server_config = ServerConfig::::default(); + let mut server_config = QuinnServerConfig::default(); server_config.transport = transport; - server_config.crypto = C::new_server_config(&crypto_config); + server_config.crypto = TlsCrypto::new_server_config(&crypto_config); let mut endpoint_config = QuinnEndpointConfig::default(); endpoint_config - .supported_versions(C::supported_quic_versions(), C::default_quic_version())?; + .supported_versions(TlsCrypto::supported_quic_versions(), TlsCrypto::default_quic_version())?; let socket = UdpSocket::bind(addr)?; let port = socket.local_addr()?.port(); @@ -217,11 +218,7 @@ impl EndpointConfig { }) } - pub fn spawn(self) -> TransportChannel - where - ::ClientConfig: Send + Unpin, - ::HeaderKey: Unpin, - ::PacketKey: Unpin, + pub fn spawn(self) -> TransportChannel { let (tx1, rx1) = mpsc::unbounded(); let (tx2, rx2) = mpsc::channel(1); @@ -243,20 +240,20 @@ impl EndpointConfig { } } -struct Endpoint { - channel: EndpointChannel, - endpoint: QuinnEndpoint, +struct Endpoint { + channel: EndpointChannel, + endpoint: QuinnEndpoint, socket: UdpSocket, - crypto_config: Arc>, + crypto_config: Arc, connections: HashMap>, outgoing: VecDeque, recv_buf: Box<[u8]>, - incoming_slot: Option>, + 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() @@ -291,7 +288,7 @@ impl Endpoint { self.outgoing.push_back(transmit); } - fn send_incoming(&mut self, muxer: QuicMuxer, cx: &mut Context) -> bool { + 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 => { @@ -328,11 +325,7 @@ impl Endpoint { } } -impl Future for Endpoint -where - ::ClientConfig: Unpin, - ::HeaderKey: Unpin, - ::PacketKey: Unpin, +impl Future for Endpoint { type Output = (); @@ -363,8 +356,8 @@ where public_key, tx, }) => { - let crypto = C::new_client_config(&me.crypto_config, public_key); - let client_config = ClientConfig { + let crypto = TlsCrypto::new_client_config(&me.crypto_config, public_key); + let client_config = QuinnClientConfig { transport: me.crypto_config.transport.clone(), crypto, }; diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 9c8f1b3857d..28f997be429 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -21,47 +21,40 @@ mod crypto; mod endpoint; mod muxer; -#[cfg(feature = "tls")] mod tls; mod transport; -pub use crate::crypto::Crypto; -#[cfg(feature = "tls")] -pub use crate::crypto::TlsCrypto; -pub use crate::crypto::ToLibp2p; +use crate::crypto::TlsCrypto; pub use crate::muxer::{QuicMuxer, QuicMuxerError}; pub use crate::transport::{QuicDial, QuicTransport}; pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; use libp2p_core::transport::TransportError; use libp2p_core::Multiaddr; -use quinn_proto::crypto::Session; +use libp2p_core::identity::Keypair; +use std::sync::Arc; use thiserror::Error; /// Quic configuration. -pub struct QuicConfig { - pub keypair: C::Keypair, +pub struct QuicConfig { + pub keypair: Keypair, pub transport: TransportConfig, - pub keylogger: Option, + pub keylogger: Option>, } -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.to_public()) + .field("keypair", &self.keypair.public()) .field("transport", &self.transport) .finish() } } -impl QuicConfig -where - ::ClientConfig: Send + Unpin, - ::HeaderKey: Unpin, - ::PacketKey: Unpin, +impl QuicConfig { /// Creates a new config from a keypair. - pub fn new(keypair: C::Keypair) -> Self { + pub fn new(keypair: Keypair) -> Self { Self { keypair, transport: TransportConfig::default(), @@ -71,7 +64,7 @@ where /// Enable keylogging. pub fn enable_keylogger(&mut self) -> &mut Self { - self.keylogger = Some(C::keylogger()); + self.keylogger = Some(TlsCrypto::keylogger()); self } @@ -79,7 +72,7 @@ where pub async fn listen_on( self, addr: Multiaddr, - ) -> Result, TransportError> { + ) -> Result> { QuicTransport::new(self, addr).await } } diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 9b279d8e29a..13eb2fbbaa0 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -18,14 +18,13 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::crypto::Crypto; use crate::endpoint::ConnectionChannel; use async_io::Timer; use futures::prelude::*; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use libp2p_core::{Multiaddr, PeerId}; use parking_lot::Mutex; -use quinn_proto::generic::Connection; +use quinn_proto::Connection as QuinnConnection; use quinn_proto::{ ConnectionError, Dir, Event, FinishError, ReadError, ReadableError, StreamEvent, StreamId, VarInt, WriteError, @@ -39,24 +38,24 @@ use std::time::Instant; use thiserror::Error; /// State for a single opened QUIC connection. -pub struct QuicMuxer { - inner: Mutex>, +pub struct QuicMuxer { + inner: Mutex, } -impl std::fmt::Debug for QuicMuxer { +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`]. -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: QuinnConnection, /// Connection waker. waker: Option, /// Connection timer. @@ -78,8 +77,8 @@ struct SubstreamState { write_waker: Option, } -impl QuicMuxer { - pub fn new(endpoint: ConnectionChannel, connection: Connection) -> Self { +impl QuicMuxer { + pub fn new(endpoint: ConnectionChannel, connection: QuinnConnection) -> Self { Self { inner: Mutex::new(QuicMuxerInner { accept_incoming: false, @@ -101,7 +100,10 @@ impl QuicMuxer { pub fn peer_id(&self) -> Option { let inner = self.inner.lock(); let session = inner.connection.crypto_session(); - C::peer_id(session) + let certificate = session.get_peer_certificates()?.into_iter().next()?; + Some(crate::tls::extract_peerid_or_panic( + quinn_proto::Certificate::from(certificate).as_der(), + )) } pub fn local_addr(&self) -> Multiaddr { @@ -126,7 +128,7 @@ impl QuicMuxer { } } -impl StreamMuxer for QuicMuxer { +impl StreamMuxer for QuicMuxer { type Substream = StreamId; type OutboundSubstream = (); type Error = QuicMuxerError; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 9aabf5cb7ee..7bfb4a3e2a6 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -18,19 +18,18 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::crypto::Crypto; use crate::endpoint::{EndpointConfig, TransportChannel}; use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; use futures::channel::oneshot; use futures::prelude::*; use if_watch::{IfEvent, IfWatcher}; +use libp2p_core::identity::PublicKey; 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}; use std::pin::Pin; use std::sync::Arc; @@ -38,22 +37,18 @@ use std::task::{Context, Poll}; use udp_socket::SocketType; #[derive(Clone)] -pub struct QuicTransport { - inner: Arc>>, +pub struct QuicTransport { + inner: Arc>, } -impl QuicTransport -where - ::ClientConfig: Send + Unpin, - ::HeaderKey: Unpin, - ::PacketKey: Unpin, +impl QuicTransport { /// Creates a new quic transport. pub async fn new( - config: QuicConfig, + 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() { @@ -82,14 +77,14 @@ where } } -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, } @@ -98,26 +93,23 @@ enum Addresses { Ip(Option), } -impl Transport for QuicTransport -where - ::HeaderKey: Unpin, - ::PacketKey: Unpin, +impl Transport for QuicTransport { - type Output = (PeerId, QuicMuxer); + type Output = (PeerId, QuicMuxer); type Error = QuicError; type Listener = Self; - type ListenerUpgrade = QuicUpgrade; - type Dial = QuicDial; + type ListenerUpgrade = QuicUpgrade; + type Dial = QuicDial; fn listen_on(self, addr: Multiaddr) -> Result> { - multiaddr_to_socketaddr::(&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"); @@ -137,8 +129,8 @@ where } } -impl Stream for QuicTransport { - type Item = Result, QuicError>, QuicError>; +impl Stream for QuicTransport { + type Item = Result, QuicError>; fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { let mut inner = self.inner.lock(); @@ -190,17 +182,14 @@ impl Stream for QuicTransport { } #[allow(clippy::large_enum_variant)] -pub enum QuicDial { - Dialing(oneshot::Receiver, QuicError>>), - Upgrade(QuicUpgrade), +pub enum QuicDial { + Dialing(oneshot::Receiver>), + Upgrade(QuicUpgrade), } -impl Future for QuicDial -where - ::HeaderKey: Unpin, - ::PacketKey: Unpin, +impl Future for QuicDial { - type Output = Result<(PeerId, QuicMuxer), QuicError>; + type Output = Result<(PeerId, QuicMuxer), QuicError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { loop { @@ -219,22 +208,19 @@ where } } -pub struct QuicUpgrade { - muxer: Option>, +pub struct QuicUpgrade { + muxer: Option, } -impl QuicUpgrade { - fn new(muxer: QuicMuxer) -> Self { +impl QuicUpgrade { + fn new(muxer: QuicMuxer) -> Self { Self { muxer: Some(muxer) } } } -impl Future for QuicUpgrade -where - ::HeaderKey: Unpin, - ::PacketKey: Unpin, +impl Future for QuicUpgrade { - type Output = Result<(PeerId, QuicMuxer), QuicError>; + type Output = Result<(PeerId, QuicMuxer), QuicError>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let inner = Pin::into_inner(self); @@ -261,9 +247,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( +fn multiaddr_to_socketaddr( addr: &Multiaddr, -) -> Result<(SocketAddr, Option), ()> { +) -> Result<(SocketAddr, Option), ()> { let mut iter = addr.iter().peekable(); let proto1 = iter.next().ok_or(())?; let proto2 = iter.next().ok_or(())?; @@ -275,7 +261,6 @@ fn multiaddr_to_socketaddr( } let public_key = libp2p_core::PublicKey::from_protobuf_encoding(peer_id.digest()).map_err(|_| ())?; - let public_key = C::extract_public_key(public_key).ok_or(())?; iter.next(); Some(public_key) } else { @@ -309,16 +294,17 @@ pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { mod tests { use super::*; - fn multiaddr_to_udp_conversion() { + #[test] + fn multiaddr_to_udp_conversion() { use std::net::{Ipv4Addr, Ipv6Addr}; - assert!(multiaddr_to_socketaddr::( + 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() @@ -329,7 +315,7 @@ mod tests { )) ); assert_eq!( - multiaddr_to_socketaddr::( + multiaddr_to_socketaddr( &"/ip4/255.255.255.255/udp/8080/quic" .parse::() .unwrap() @@ -340,14 +326,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() @@ -364,31 +350,19 @@ mod tests { ); } - #[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; + fn multiaddr_to_pk_conversion() { use std::net::Ipv4Addr; - let peer_id = keypair.to_public().to_peer_id(); + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); + let peer_id = keypair.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()), + 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()) + Some(keypair.public()) )) ); } - - #[cfg(feature = "tls")] - #[test] - fn multiaddr_to_pk_tls() { - let keypair = libp2p_core::identity::Keypair::generate_ed25519(); - multiaddr_to_pk_conversion::(keypair); - } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index e68f6fd0567..6e030c4722a 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -10,44 +10,21 @@ use libp2p::request_response::{ RequestResponseEvent, RequestResponseMessage, }; use libp2p::swarm::{Swarm, SwarmEvent}; -use libp2p_quic::{Crypto, QuicConfig, ToLibp2p}; -use quinn_proto::crypto::Session; +use libp2p_quic::QuicConfig; use rand::RngCore; use std::{io, iter}; -#[cfg(feature = "tls")] fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } -#[cfg(feature = "tls")] -#[async_std::test] -async fn smoke_tls() -> Result<()> { - smoke::().await -} - -trait GenerateKeypair: Crypto { - fn generate_keypair() -> Self::Keypair; -} - -#[cfg(feature = "tls")] -impl GenerateKeypair for libp2p_quic::TlsCrypto { - fn generate_keypair() -> Self::Keypair { - generate_tls_keypair() - } -} - -async fn create_swarm( +async fn create_swarm( keylog: bool, ) -> Result>> -where - ::ClientConfig: Send + Unpin, - ::HeaderKey: Unpin, - ::PacketKey: Unpin, { - let keypair = C::generate_keypair(); - let peer_id = keypair.to_peer_id(); - let mut transport = QuicConfig::::new(keypair); + let keypair = generate_tls_keypair(); + let peer_id = keypair.public().to_peer_id(); + let mut transport = QuicConfig::new(keypair); if keylog { transport.enable_keylogger(); } @@ -63,12 +40,8 @@ where Ok(Swarm::new(transport, behaviour, peer_id)) } -async fn smoke() -> Result<()> -where - ::ClientConfig: Send + Unpin, - ::HeaderKey: Unpin, - ::PacketKey: Unpin, -{ +#[async_std::test] +async fn smoke() -> Result<()> { tracing_subscriber::fmt() .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) .try_init() @@ -76,8 +49,8 @@ where 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()?)?; @@ -274,22 +247,21 @@ impl RequestResponseCodec for PingCodec { } } -#[cfg(feature = "tls")] #[async_std::test] -async fn dial_failure_tls() -> Result<()> { +async fn dial_failure() -> 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?; + 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 = libp2p_quic::TlsCrypto::generate_keypair(); - let fake_peer_id = keypair.to_peer_id(); + let keypair = generate_tls_keypair(); + let fake_peer_id = keypair.public().to_peer_id(); let mut addr = match a.next().await { Some(SwarmEvent::NewListenAddr { address, .. }) => address, From 756f2e702c6bf000a4bed45acf422bb2efa210be Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 9 Nov 2021 18:37:49 +0300 Subject: [PATCH 023/218] Apply cargo fmt --- transports/quic/src/crypto.rs | 3 +-- transports/quic/src/endpoint.rs | 18 +++++++++--------- transports/quic/src/lib.rs | 5 ++--- transports/quic/src/transport.rs | 27 ++++++++++----------------- transports/quic/tests/smoke.rs | 5 +---- 5 files changed, 23 insertions(+), 35 deletions(-) diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index 9a0e5bfa613..ad9fdbe2c16 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use libp2p_core::identity::{PublicKey, Keypair}; +use libp2p_core::identity::{Keypair, PublicKey}; use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; use std::sync::Arc; @@ -33,7 +33,6 @@ pub struct CryptoConfig { pub(crate) struct TlsCrypto; impl TlsCrypto { - pub fn new_server_config( config: &CryptoConfig, ) -> ::ServerConfig { diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index c2cb4f6a8f4..2ec239e518a 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -23,12 +23,12 @@ use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; -use quinn_proto::{ServerConfig as QuinnServerConfig}; -use quinn_proto::generic::{ClientConfig as QuinnClientConfig}; +use libp2p_core::identity::PublicKey; +use quinn_proto::generic::ClientConfig as QuinnClientConfig; +use quinn_proto::ServerConfig as QuinnServerConfig; use quinn_proto::{ ConnectionEvent, ConnectionHandle, DatagramEvent, EcnCodepoint, EndpointEvent, Transmit, }; -use libp2p_core::identity::PublicKey; use std::collections::{HashMap, VecDeque}; use std::io::IoSliceMut; use std::mem::MaybeUninit; @@ -199,8 +199,10 @@ impl EndpointConfig { server_config.crypto = TlsCrypto::new_server_config(&crypto_config); let mut endpoint_config = QuinnEndpointConfig::default(); - endpoint_config - .supported_versions(TlsCrypto::supported_quic_versions(), TlsCrypto::default_quic_version())?; + endpoint_config.supported_versions( + TlsCrypto::supported_quic_versions(), + TlsCrypto::default_quic_version(), + )?; let socket = UdpSocket::bind(addr)?; let port = socket.local_addr()?.port(); @@ -218,8 +220,7 @@ impl EndpointConfig { }) } - pub fn spawn(self) -> TransportChannel - { + pub fn spawn(self) -> TransportChannel { let (tx1, rx1) = mpsc::unbounded(); let (tx2, rx2) = mpsc::channel(1); let transport = TransportChannel { @@ -325,8 +326,7 @@ impl Endpoint { } } -impl Future for Endpoint -{ +impl Future for Endpoint { type Output = (); fn poll(self: Pin<&mut Self>, cx: &mut Context) -> Poll { diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 28f997be429..d609f2eaaa6 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -29,9 +29,9 @@ pub use crate::muxer::{QuicMuxer, QuicMuxerError}; pub use crate::transport::{QuicDial, QuicTransport}; pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; +use libp2p_core::identity::Keypair; use libp2p_core::transport::TransportError; use libp2p_core::Multiaddr; -use libp2p_core::identity::Keypair; use std::sync::Arc; use thiserror::Error; @@ -51,8 +51,7 @@ impl std::fmt::Debug for QuicConfig { } } -impl QuicConfig -{ +impl QuicConfig { /// Creates a new config from a keypair. pub fn new(keypair: Keypair) -> Self { Self { diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 7bfb4a3e2a6..d1ec1fd0b28 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -41,8 +41,7 @@ pub struct QuicTransport { inner: Arc>, } -impl QuicTransport -{ +impl QuicTransport { /// Creates a new quic transport. pub async fn new( config: QuicConfig, @@ -93,8 +92,7 @@ enum Addresses { Ip(Option), } -impl Transport for QuicTransport -{ +impl Transport for QuicTransport { type Output = (PeerId, QuicMuxer); type Error = QuicError; type Listener = Self; @@ -102,8 +100,7 @@ impl Transport for QuicTransport 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) } @@ -187,8 +184,7 @@ pub enum QuicDial { Upgrade(QuicUpgrade), } -impl Future for QuicDial -{ +impl Future for QuicDial { type Output = Result<(PeerId, QuicMuxer), QuicError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { @@ -218,8 +214,7 @@ impl QuicUpgrade { } } -impl Future for QuicUpgrade -{ +impl Future for QuicUpgrade { type Output = Result<(PeerId, QuicMuxer), QuicError>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -247,9 +242,7 @@ impl Future for QuicUpgrade /// 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(())?; @@ -298,10 +291,10 @@ mod tests { 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( diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 6e030c4722a..7336b66b7d1 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -18,10 +18,7 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } -async fn create_swarm( - keylog: bool, -) -> Result>> -{ +async fn create_swarm(keylog: bool) -> Result>> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let mut transport = QuicConfig::new(keypair); From 3b1ff348999da1543b7c394f4db9f7364e061508 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 11 Nov 2021 21:26:24 +0300 Subject: [PATCH 024/218] Don't require remote PK to dial a peer --- transports/quic/src/crypto.rs | 7 +- transports/quic/src/endpoint.rs | 23 +----- transports/quic/src/tls/mod.rs | 6 +- transports/quic/src/tls/verifier.rs | 23 ++---- transports/quic/src/transport.rs | 114 +++++++++++----------------- transports/quic/tests/smoke.rs | 28 ++----- 6 files changed, 67 insertions(+), 134 deletions(-) diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index ad9fdbe2c16..145990c7973 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use libp2p_core::identity::{Keypair, PublicKey}; +use libp2p_core::identity::Keypair; use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; use std::sync::Arc; @@ -45,11 +45,8 @@ impl TlsCrypto { pub fn new_client_config( config: &CryptoConfig, - remote_public: PublicKey, ) -> ::ClientConfig { - let mut client = - crate::tls::make_client_config(&config.keypair, remote_public.to_peer_id()) - .expect("invalid config"); + let mut client = crate::tls::make_client_config(&config.keypair).expect("invalid config"); if let Some(key_log) = config.keylogger.clone() { client.key_log = key_log; } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 2ec239e518a..728bb11948e 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -23,7 +23,6 @@ use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; -use libp2p_core::identity::PublicKey; use quinn_proto::generic::ClientConfig as QuinnClientConfig; use quinn_proto::ServerConfig as QuinnServerConfig; use quinn_proto::{ @@ -46,8 +45,6 @@ enum ToEndpoint { 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>, }, @@ -70,17 +67,9 @@ pub struct TransportChannel { } impl TransportChannel { - pub fn dial( - &mut self, - addr: SocketAddr, - public_key: PublicKey, - ) -> oneshot::Receiver> { + pub fn dial(&mut self, addr: SocketAddr) -> oneshot::Receiver> { let (tx, rx) = oneshot::channel(); - let msg = ToEndpoint::Dial { - addr, - public_key, - tx, - }; + let msg = ToEndpoint::Dial { addr, tx }; self.tx.unbounded_send(msg).expect("endpoint has crashed"); rx } @@ -351,12 +340,8 @@ impl Future for Endpoint { 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 = TlsCrypto::new_client_config(&me.crypto_config, public_key); + Some(ToEndpoint::Dial { addr, tx }) => { + let crypto = TlsCrypto::new_client_config(&me.crypto_config); let client_config = QuinnClientConfig { transport: me.crypto_config.transport.clone(), crypto, diff --git a/transports/quic/src/tls/mod.rs b/transports/quic/src/tls/mod.rs index 1cf8f3571b6..9241c527b9f 100644 --- a/transports/quic/src/tls/mod.rs +++ b/transports/quic/src/tls/mod.rs @@ -23,7 +23,6 @@ mod certificate; mod verifier; -use libp2p_core::PeerId; use std::sync::Arc; use thiserror::Error; @@ -50,13 +49,12 @@ pub enum ConfigError { pub fn make_client_config( keypair: &libp2p_core::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 verifier = verifier::Libp2pCertificateVerifier; let mut crypto = rustls::ClientConfig::new(); crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; @@ -76,7 +74,7 @@ pub fn make_server_config( let private_key = cert.serialize_private_key_der(); let cert = rustls::Certificate(cert.serialize_der()?); let key = rustls::PrivateKey(private_key); - let verifier = verifier::Libp2pClientCertificateVerifier; + let verifier = verifier::Libp2pCertificateVerifier; let mut crypto = rustls::ServerConfig::new(Arc::new(verifier)); crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 3427aa4275a..793d8a91455 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -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 Libp2pServerCertificateVerifier(pub(crate) PeerId); +pub(crate) struct Libp2pCertificateVerifier; /// libp2p requires the following of X.509 server certificate chains: /// @@ -39,7 +39,7 @@ pub(crate) struct Libp2pServerCertificateVerifier(pub(crate) PeerId); /// - The certificate must be self-signed. /// - The certificate must have a valid libp2p extension that includes a /// signature of its public key. -impl rustls::ServerCertVerifier for Libp2pServerCertificateVerifier { +impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { fn verify_server_cert( &self, _roots: &rustls::RootCertStore, @@ -47,13 +47,7 @@ impl rustls::ServerCertVerifier for Libp2pServerCertificateVerifier { _dns_name: webpki::DNSNameRef<'_>, _ocsp_response: &[u8], ) -> Result { - 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()) + verify_presented_certs(presented_certs).map(|_| ServerCertVerified::assertion()) } fn verify_tls12_signature( @@ -77,11 +71,6 @@ impl rustls::ServerCertVerifier for Libp2pServerCertificateVerifier { } } -/// 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 @@ -89,7 +78,7 @@ pub(crate) struct Libp2pClientCertificateVerifier; /// - The certificate must be self-signed. /// - The certificate must have a valid libp2p extension that includes a /// signature of its public key. -impl rustls::ClientCertVerifier for Libp2pClientCertificateVerifier { +impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { fn offer_client_auth(&self) -> bool { true } @@ -186,7 +175,7 @@ fn parse_certificate( Ok((parsed, libp2p_extension)) } -fn verify_presented_certs(presented_certs: &[Certificate]) -> Result { +fn verify_presented_certs(presented_certs: &[Certificate]) -> Result<(), TLSError> { if presented_certs.len() != 1 { return Err(TLSError::NoCertificatesPresented); } @@ -198,7 +187,7 @@ fn verify_presented_certs(presented_certs: &[Certificate]) -> Result { diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index d1ec1fd0b28..c3ffd428c64 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -24,7 +24,6 @@ use crate::{QuicConfig, QuicError}; use futures::channel::oneshot; use futures::prelude::*; use if_watch::{IfEvent, IfWatcher}; -use libp2p_core::identity::PublicKey; use libp2p_core::multiaddr::{Multiaddr, Protocol}; use libp2p_core::muxing::{StreamMuxer, StreamMuxerBox}; use libp2p_core::transport::{Boxed, ListenerEvent, Transport, TransportError}; @@ -48,8 +47,7 @@ impl QuicTransport { addr: Multiaddr, ) -> Result> { let socket_addr = multiaddr_to_socketaddr(&addr) - .map_err(|_| TransportError::MultiaddrNotSupported(addr.clone()))? - .0; + .ok_or_else(|| TransportError::MultiaddrNotSupported(addr.clone()))?; let addresses = if socket_addr.ip().is_unspecified() { let watcher = IfWatcher::new() .await @@ -100,24 +98,22 @@ impl Transport for QuicTransport { type Dial = QuicDial; fn listen_on(self, addr: Multiaddr) -> Result> { - multiaddr_to_socketaddr(&addr).map_err(|_| TransportError::MultiaddrNotSupported(addr))?; + multiaddr_to_socketaddr(&addr) + .ok_or_else(|| 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())); - }; + let socket_addr = multiaddr_to_socketaddr(&addr).ok_or_else(|| { + tracing::debug!("invalid multiaddr"); + 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); + let rx = self.inner.lock().channel.dial(socket_addr); Ok(QuicDial::Dialing(rx)) } @@ -240,38 +236,29 @@ impl Future for QuicUpgrade { } } -/// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns an error if the format +/// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns None 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(())?; +fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { + let mut iter = addr.iter(); + let proto1 = iter.next()?; + let proto2 = iter.next()?; + let proto3 = iter.next()?; - let peer_id = if let Some(Protocol::P2p(peer_id)) = iter.peek() { - if peer_id.code() != multihash::Code::Identity.into() { - return Err(()); + while let Some(proto) = iter.next() { + match proto { + Protocol::P2p(_) => {} // Ignore a `/p2p/...` prefix of possibly outer protocols, if present. + _ => return None, } - let public_key = - libp2p_core::PublicKey::from_protobuf_encoding(peer_id.digest()).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)) + Some(SocketAddr::new(ip.into(), port)) } (Protocol::Ip6(ip), Protocol::Udp(port), Protocol::Quic) => { - Ok((SocketAddr::new(ip.into(), port), peer_id)) + Some(SocketAddr::new(ip.into(), port)) } - _ => Err(()), + _ => None, } } @@ -288,12 +275,12 @@ mod tests { use super::*; #[test] - fn multiaddr_to_udp_conversion() { + fn multiaddr_to_socketaddr_conversion() { use std::net::{Ipv4Addr, Ipv6Addr}; assert!( multiaddr_to_socketaddr(&"/ip4/127.0.0.1/udp/1234".parse::().unwrap()) - .is_err() + .is_none() ); assert_eq!( @@ -302,27 +289,35 @@ mod tests { .parse::() .unwrap() ), - Ok(( - SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 12345,), - None + Some(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), + 12345, )) ); + + assert!(multiaddr_to_socketaddr( + &"/ip4/127.0.0.1/udp/12345/quic/tcp/12345" + .parse::() + .unwrap() + ) + .is_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 + Some(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(255, 255, 255, 255)), + 8080, )) ); 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 + Some(SocketAddr::new( + IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), + 12345, )) ); assert_eq!( @@ -331,30 +326,11 @@ mod tests { .parse::() .unwrap() ), - Ok(( - SocketAddr::new( - IpAddr::V6(Ipv6Addr::new( - 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, - )), - 8080, - ), - None - )) - ); - } - - #[test] - fn multiaddr_to_pk_conversion() { - use std::net::Ipv4Addr; - - let keypair = libp2p_core::identity::Keypair::generate_ed25519(); - let peer_id = keypair.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,), - Some(keypair.public()) + Some(SocketAddr::new( + IpAddr::V6(Ipv6Addr::new( + 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, + )), + 8080, )) ); } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 7336b66b7d1..4135a08bbf6 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -4,7 +4,6 @@ use futures::future::FutureExt; use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; use futures::stream::StreamExt; use libp2p::core::upgrade; -use libp2p::multiaddr::Protocol; use libp2p::request_response::{ ProtocolName, ProtocolSupport, RequestResponse, RequestResponseCodec, RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, @@ -51,11 +50,10 @@ async fn smoke() -> Result<()> { Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; - let mut addr = match a.next().await { + let 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); @@ -252,34 +250,27 @@ async fn dial_failure() -> Result<()> { .ok(); log_panics::init(); - let mut a = create_swarm(true).await?; - let mut b = create_swarm(false).await?; + let mut a = create_swarm(false).await?; + let mut b = create_swarm(true).await?; Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; - let keypair = generate_tls_keypair(); - let fake_peer_id = keypair.public().to_peer_id(); - - let mut addr = match a.next().await { + let addr = match a.next().await { Some(SwarmEvent::NewListenAddr { address, .. }) => address, e => panic!("{:?}", e), }; - addr.push(Protocol::P2p(fake_peer_id.into())); + let a_peer_id = &Swarm::local_peer_id(&a).clone(); + drop(a); // stop a swarm so b can never reach it - b.behaviour_mut().add_address(&fake_peer_id, addr); + b.behaviour_mut().add_address(a_peer_id, addr); b.behaviour_mut() - .send_request(&fake_peer_id, Ping(b"hello world".to_vec())); + .send_request(a_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::OutgoingConnectionError { .. }) => {} e => panic!("{:?}", e), @@ -290,8 +281,5 @@ async fn dial_failure() -> Result<()> { e => panic!("{:?}", e), }; - assert!(a.next().await.is_some()); // ConnectionClosed - assert!(a.next().now_or_never().is_none()); - Ok(()) } From 63e4696e83d267fd202583aa88e297bdd20b7c34 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 14 Dec 2021 23:05:45 +0300 Subject: [PATCH 025/218] - Get rid of barebones-x509 - Add more comments to x509 code --- transports/quic/Cargo.toml | 5 +- transports/quic/src/muxer.rs | 5 +- transports/quic/src/tls/certificate.rs | 398 +++++++++++++++++++++++-- transports/quic/src/tls/mod.rs | 75 +++-- transports/quic/src/tls/verifier.rs | 189 +++++------- 5 files changed, 487 insertions(+), 185 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 589c673875e..7792ce32046 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -10,12 +10,11 @@ license = "MIT" [dependencies] async-global-executor = "2.0.2" async-io = "1.6.0" -barebones-x509 = { version = "0.5.0", features = ["webpki", "rustls", "std"] } bytes = "1.0.1" +der-parser = { version = "6.0.0", default-features = false } futures = "0.3.15" if-watch = "0.2.2" libp2p-core = { version = "0.31.0", path = "../../core" } -multihash = { version = "0.14.0", default-features = false } parking_lot = "0.11.1" quinn-proto = { version = "0.7.3", default-features = false, features = ["tls-rustls"] } rcgen = { version = "0.8.11" } @@ -24,8 +23,8 @@ rustls = { version = "0.19.1", features = ["dangerous_configuration"] } thiserror = "1.0.26" tracing = "0.1.26" udp-socket = "0.1.5" -untrusted = { version = "0.7.1" } webpki = { version = "0.21.4", features = ["std"] } +x509-parser = "0.12.0" yasna = { version = "0.4.0" } [dev-dependencies] diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 13eb2fbbaa0..491aa7f038e 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -101,9 +101,8 @@ impl QuicMuxer { let inner = self.inner.lock(); let session = inner.connection.crypto_session(); let certificate = session.get_peer_certificates()?.into_iter().next()?; - Some(crate::tls::extract_peerid_or_panic( - quinn_proto::Certificate::from(certificate).as_der(), - )) + let certificate = crate::tls::certificate::parse_certificate(&certificate.0).ok()?; + Some(PeerId::from_public_key(&certificate.extension.public_key)) } pub fn local_addr(&self) -> Multiaddr { diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index 59b6c2a4e77..3d35cee4796 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -1,4 +1,4 @@ -// Copyright 2020 Parity Technologies (UK) Ltd. +// Copyright 2021 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"), @@ -18,47 +18,71 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -//! Certificate handling for libp2p +//! X.509 certificate handling for libp2p //! //! This module handles generation, signing, and verification of certificates. -use super::LIBP2P_SIGNING_PREFIX_LENGTH; -use libp2p_core::identity::Keypair; +use libp2p_core::identity; +use x509_parser::prelude::*; -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; -static LIBP2P_SIGNATURE_ALGORITHM: &rcgen::SignatureAlgorithm = &rcgen::PKCS_ECDSA_P256_SHA256; +/// The libp2p Public Key Extension is a X.509 extension +/// with the Object Identier 1.3.6.1.4.1.53594.1.1, +/// allocated by IANA to the libp2p project at Protocol Labs. +const P2P_EXT_OID: [u64; 9] = [1, 3, 6, 1, 4, 1, 53594, 1, 1]; + +/// The peer signs the concatenation of the string `libp2p-tls-handshake:` +/// and the public key that it used to generate the certificate carrying +/// the libp2p Public Key Extension, using its private host key. +/// This signature provides cryptographic proof that the peer was +/// in possession of the private host key at the time the certificate was signed. +const P2P_SIGNING_PREFIX: [u8; 21] = *b"libp2p-tls-handshake:"; + +// Certificates MUST use the NamedCurve encoding for elliptic curve parameters. +// Similarly, hash functions with an output length less than 256 bits MUST NOT be used. +static P2P_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 { +pub fn make_certificate( + keypair: &identity::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_der(); - 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)? - }; + // SHOULD NOT be related to the host's key. + // Endpoints MAY generate a new key and certificate + // for every connection attempt, or they MAY reuse the same key + // and certificate for multiple connections. + let certif_keypair = rcgen::KeyPair::generate(P2P_SIGNATURE_ALGORITHM)?; // Generate the libp2p-specific extension. + // The certificate MUST contain the libp2p Public Key Extension. let libp2p_extension: rcgen::CustomExtension = { + // The peer signs the concatenation of the string `libp2p-tls-handshake:` + // and the public key that it used to generate the certificate carrying + // the libp2p Public Key Extension, using its private host key. + let signature = { + let mut msg = vec![]; + msg.extend(P2P_SIGNING_PREFIX); + msg.extend(certif_keypair.public_key_der()); + + keypair + .sign(&msg) + .map_err(|_| rcgen::RcgenError::RingUnspecified)? + }; + + // The public host key and the signature are ANS.1-encoded + // into the SignedKey data structure, which is carried + // in the libp2p Public Key Extension. + // SignedKey ::= SEQUENCE { + // publicKey OCTET STRING, + // signature OCTET STRING + // } let extension_content = { let serialized_pubkey = keypair.public().to_protobuf_encoding(); - yasna::encode_der(&(serialized_pubkey, libp2p_ext_signature)) + yasna::encode_der(&(serialized_pubkey, signature)) }; - let mut ext = rcgen::CustomExtension::from_oid_content(LIBP2P_OID, extension_content); + // This extension MAY be marked critical. + let mut ext = rcgen::CustomExtension::from_oid_content(&P2P_EXT_OID, extension_content); ext.set_criticality(true); ext }; @@ -67,10 +91,328 @@ pub(crate) fn make_cert(keypair: &Keypair) -> Result, +} + +/// An X.509 certificate with a libp2p-specific extension +/// is used to secure libp2p connections. +pub struct P2pCertificate<'a> { + certificate: X509Certificate<'a>, + /// This is a specific libp2p Public Key Extension with two values: + /// * the public host key + /// * a signature performed using the private host key + pub(crate) extension: P2pExtension, +} + +/// Parse TLS certificate from DER input that includes a libp2p-specific +/// certificate extension containing a public key of a peer. +pub fn parse_certificate(der_input: &[u8]) -> Result { + use webpki::Error; + let x509 = X509Certificate::from_der(der_input) + .map(|(_rest_input, x509)| x509) + .map_err(|_| Error::BadDER)?; + + let p2p_ext_oid = der_parser::oid::Oid::from(&P2P_EXT_OID) + .expect("This is a valid OID of p2p extension; qed"); + + let mut libp2p_extension = None; + + for ext in x509.extensions() { + let oid = &ext.oid; + if oid == &p2p_ext_oid && libp2p_extension.is_some() { + // The extension was already parsed + return Err(Error::BadDER); + } + + if oid == &p2p_ext_oid { + // The public host key and the signature are ANS.1-encoded + // into the SignedKey data structure, which is carried + // in the libp2p Public Key Extension. + // SignedKey ::= SEQUENCE { + // publicKey OCTET STRING, + // signature OCTET STRING + // } + let (public_key, signature): (Vec, Vec) = + yasna::decode_der(ext.value).map_err(|_| Error::ExtensionValueInvalid)?; + // The publicKey field of SignedKey contains the public host key + // of the endpoint, encoded using the following protobuf: + // enum KeyType { + // RSA = 0; + // Ed25519 = 1; + // Secp256k1 = 2; + // ECDSA = 3; + // } + // message PublicKey { + // required KeyType Type = 1; + // required bytes Data = 2; + // } + let public_key = identity::PublicKey::from_protobuf_encoding(&public_key) + .map_err(|_| Error::UnknownIssuer)?; + let ext = P2pExtension { + public_key, + signature, + }; + libp2p_extension = Some(ext); + continue; + } + + if ext.critical { + // Endpoints MUST abort the connection attempt if the certificate + // contains critical extensions that the endpoint does not understand. + return Err(Error::UnsupportedCriticalExtension); + } + + // Implementations MUST ignore non-critical extensions with unknown OIDs. + } + + if let Some(extension) = libp2p_extension { + Ok(P2pCertificate { + certificate: x509, + extension, + }) + } else { + // The certificate MUST contain the libp2p Public Key Extension. + // If this extension is missing, endpoints MUST abort the connection attempt. + Err(Error::BadDER) + } +} + +impl P2pCertificate<'_> { + /// This method validates the certificate according to libp2p TLS 1.3 specs. + /// The certificate MUST: + /// 1. be valid at the time it is received by the peer; + /// 2. use the NamedCurve encoding; + /// 3. use hash functions with an output length not less than 256 bits; + /// 4. be self signed; + /// 5. contain a valid signature in the specific libp2p extension. + pub fn verify(&self) -> Result<(), webpki::Error> { + use webpki::Error; + // The certificate MUST have NotBefore and NotAfter fields set + // such that the certificate is valid at the time it is received by the peer. + if !self.certificate.validity().is_valid() { + return Err(Error::InvalidCertValidity); + } + + // Certificates MUST use the NamedCurve encoding for elliptic curve parameters. + // Similarly, hash functions with an output length less than 256 bits + // MUST NOT be used, due to the possibility of collision attacks. + // In particular, MD5 and SHA1 MUST NOT be used. + // Endpoints MUST abort the connection attempt if it is not used. + let signature_scheme = self.signature_scheme()?; + // Endpoints MUST abort the connection attempt if the certificate’s + // self-signature is not valid. + let raw_certificate = self.certificate.tbs_certificate.as_ref(); + let signature = self.certificate.signature_value.data; + // check if self signed + self.verify_signature(signature_scheme, raw_certificate, signature) + .map_err(|_| Error::SignatureAlgorithmMismatch)?; + + let subject_pki = self.certificate.public_key().raw; + + // The peer signs the concatenation of the string `libp2p-tls-handshake:` + // and the public key that it used to generate the certificate carrying + // the libp2p Public Key Extension, using its private host key. + let mut msg = vec![]; + msg.extend(P2P_SIGNING_PREFIX); + msg.extend(subject_pki); + + // This signature provides cryptographic proof that the peer was in possession + // of the private host key at the time the certificate was signed. + // Peers MUST verify the signature, and abort the connection attempt + // if signature verification fails. + let user_owns_sk = self + .extension + .public_key + .verify(&msg, &self.extension.signature); + if !user_owns_sk { + return Err(Error::UnknownIssuer); + } + Ok(()) + } + + /// Return the signature scheme corresponding to [`AlgorithmIdentifier`]s + /// of `subject_pki` and `signature_algorithm` + /// according to ``. + pub fn signature_scheme(&self) -> Result { + // Certificates MUST use the NamedCurve encoding for elliptic curve parameters. + // Endpoints MUST abort the connection attempt if it is not used. + use oid_registry::*; + use rustls::SignatureScheme::*; + use webpki::Error; + let signature_algorithm = &self.certificate.signature_algorithm; + let pki_algorithm = &self.certificate.tbs_certificate.subject_pki.algorithm; + + if pki_algorithm.algorithm == OID_PKCS1_RSAENCRYPTION { + if signature_algorithm.algorithm == OID_PKCS1_SHA256WITHRSA { + return Ok(RSA_PKCS1_SHA256); + } + if signature_algorithm.algorithm == OID_PKCS1_SHA384WITHRSA { + return Ok(RSA_PKCS1_SHA384); + } + if signature_algorithm.algorithm == OID_PKCS1_SHA512WITHRSA { + return Ok(RSA_PKCS1_SHA512); + } + if signature_algorithm.algorithm == OID_PKCS1_RSASSAPSS { + // According to https://datatracker.ietf.org/doc/html/rfc4055#section-3.1: + // Inside of params there shuld be a sequence of: + // - Hash Algorithm + // - Mask Algorithm + // - Salt Length + // - Trailer Field + + // We are interested in Hash Algorithm only, however the der parser parses + // params into a mess, so here is a workaround to fix it: + fn get_hash_oid<'a>( + signature_algorithm: &'a AlgorithmIdentifier, + ) -> Option> { + let params = signature_algorithm.parameters.as_ref()?; + let params = params.as_sequence().ok()?; + let first_param = params.get(0)?; + let hash_oid_der = first_param.as_slice().ok()?; + let (_, obj) = der_parser::parse_der(hash_oid_der).ok()?; + let hash_oid = obj.as_sequence().ok()?.get(0)?.as_oid_val().ok()?; + Some(hash_oid) + } + + let hash_oid = get_hash_oid(signature_algorithm).ok_or(Error::BadDER)?; + + if hash_oid == OID_NIST_HASH_SHA256 { + return Ok(RSA_PSS_SHA256); + } + if hash_oid == OID_NIST_HASH_SHA384 { + return Ok(RSA_PSS_SHA384); + } + if hash_oid == OID_NIST_HASH_SHA512 { + return Ok(RSA_PSS_SHA512); + } + + // Default hash algo is SHA-1, however: + // In particular, MD5 and SHA1 MUST NOT be used. + return Err(Error::UnsupportedSignatureAlgorithm); + } + } + + if pki_algorithm.algorithm == OID_KEY_TYPE_EC_PUBLIC_KEY { + let signature_param = pki_algorithm + .parameters + .as_ref() + .ok_or(Error::BadDER)? + .as_oid_val() + .map_err(|_| Error::BadDER)?; + if signature_param == OID_EC_P256 + && signature_algorithm.algorithm == OID_SIG_ECDSA_WITH_SHA256 + { + return Ok(ECDSA_NISTP256_SHA256); + } + if signature_param == OID_NIST_EC_P384 + && signature_algorithm.algorithm == OID_SIG_ECDSA_WITH_SHA384 + { + return Ok(ECDSA_NISTP384_SHA384); + } + if signature_param == OID_NIST_EC_P521 + && signature_algorithm.algorithm == OID_SIG_ECDSA_WITH_SHA512 + { + return Ok(ECDSA_NISTP521_SHA512); + } + return Err(Error::UnsupportedSignatureAlgorithm); + } + + if signature_algorithm.algorithm == OID_SIG_ED25519 { + return Ok(ED25519); + } + if signature_algorithm.algorithm == OID_SIG_ED448 { + return Ok(ED448); + } + + Err(Error::UnsupportedSignatureAlgorithm) + } + + /// Get a [`ring::signature::UnparsedPublicKey`] for this `signature_scheme`. + /// Return `Error` if the `signature_scheme` does not match the public key signature + /// and hashing algorithm or if the `signature_scheme` is not supported. + pub fn public_key( + &self, + signature_scheme: rustls::SignatureScheme, + ) -> Result, webpki::Error> { + use ring::signature; + use rustls::SignatureScheme::*; + use webpki::Error; + + let current_signature_scheme = self.signature_scheme()?; + if signature_scheme != current_signature_scheme { + // This certificate was signed with a different signature scheme + return Err(Error::UnsupportedSignatureAlgorithmForPublicKey); + } + + let verification_algorithm: &dyn signature::VerificationAlgorithm = match signature_scheme { + RSA_PKCS1_SHA256 => &signature::RSA_PKCS1_2048_8192_SHA256, + RSA_PKCS1_SHA384 => &signature::RSA_PKCS1_2048_8192_SHA384, + RSA_PKCS1_SHA512 => &signature::RSA_PKCS1_2048_8192_SHA512, + ECDSA_NISTP256_SHA256 => &signature::ECDSA_P256_SHA256_ASN1, + ECDSA_NISTP384_SHA384 => &signature::ECDSA_P384_SHA384_ASN1, + ECDSA_NISTP521_SHA512 => { + // See https://github.com/briansmith/ring/issues/824 + return Err(Error::UnsupportedSignatureAlgorithm); + } + RSA_PSS_SHA256 => &signature::RSA_PSS_2048_8192_SHA256, + RSA_PSS_SHA384 => &signature::RSA_PSS_2048_8192_SHA384, + RSA_PSS_SHA512 => &signature::RSA_PSS_2048_8192_SHA512, + ED25519 => &signature::ED25519, + ED448 => { + // See https://github.com/briansmith/ring/issues/463 + return Err(Error::UnsupportedSignatureAlgorithm); + } + // Similarly, hash functions with an output length less than 256 bits + // MUST NOT be used, due to the possibility of collision attacks. + // In particular, MD5 and SHA1 MUST NOT be used. + RSA_PKCS1_SHA1 => return Err(Error::UnsupportedSignatureAlgorithm), + ECDSA_SHA1_Legacy => return Err(Error::UnsupportedSignatureAlgorithm), + Unknown(_) => return Err(Error::UnsupportedSignatureAlgorithm), + }; + let spki = &self.certificate.tbs_certificate.subject_pki; + let key = + signature::UnparsedPublicKey::new(verification_algorithm, spki.subject_public_key.data); + Ok(key) + } + /// Verify the `signature` of the `message` signed by the private key corresponding to the public key stored + /// in the certificate. + pub fn verify_signature( + &self, + signature_scheme: rustls::SignatureScheme, + message: &[u8], + signature: &[u8], + ) -> Result<(), webpki::Error> { + let pk = self.public_key(signature_scheme)?; + pk.verify(message, signature) + .map_err(|_| webpki::Error::InvalidSignatureForPublicKey)?; + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use crate::tls::certificate; + #[test] + fn sanity_check() { + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); + let cert = certificate::make_certificate(&keypair).unwrap(); + let cert_der = cert.serialize_der().unwrap(); + let parsed_cert = certificate::parse_certificate(&cert_der).unwrap(); + assert!(parsed_cert.verify().is_ok()); + assert_eq!(keypair.public(), parsed_cert.extension.public_key); + } +} diff --git a/transports/quic/src/tls/mod.rs b/transports/quic/src/tls/mod.rs index 9241c527b9f..f4a1fc84a32 100644 --- a/transports/quic/src/tls/mod.rs +++ b/transports/quic/src/tls/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2020 Parity Technologies (UK) Ltd. +// Copyright 2021 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"), @@ -18,67 +18,78 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -//! TLS configuration for `libp2p-quic`. +//! TLS configuration for QUIC based on libp2p TLS specs. -mod certificate; +pub(crate) mod certificate; mod verifier; use std::sync::Arc; use thiserror::Error; -pub use verifier::extract_peerid_or_panic; +use rustls::{ + ciphersuite::{ + TLS13_AES_128_GCM_SHA256, TLS13_AES_256_GCM_SHA384, TLS13_CHACHA20_POLY1305_SHA256, + }, + SupportedCipherSuite, +}; -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 +/// A list of the TLS 1.3 cipher suites supported by rustls. +// By default rustls creates client/server configs with both +// TLS 1.3 __and__ 1.2 cipher suites. But we don't need 1.2. +static TLS13_CIPHERSUITES: [&SupportedCipherSuite; 3] = [ + // TLS1.3 suites + &TLS13_CHACHA20_POLY1305_SHA256, + &TLS13_AES_256_GCM_SHA384, + &TLS13_AES_128_GCM_SHA256, +]; + +const P2P_ALPN: [u8; 6] = *b"libp2p"; /// 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), } +/// Create a TLS client configuration for libp2p. pub fn make_client_config( keypair: &libp2p_core::identity::Keypair, ) -> 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::Libp2pCertificateVerifier; - - let mut crypto = rustls::ClientConfig::new(); + let (certificate, key) = make_cert_key(keypair)?; + let verifier = Arc::new(verifier::Libp2pCertificateVerifier); + let mut crypto = rustls::ClientConfig::with_ciphersuites(&TLS13_CIPHERSUITES); 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![cert], key)?; - crypto - .dangerous() - .set_certificate_verifier(Arc::new(verifier)); + crypto.set_protocols(&[P2P_ALPN.to_vec()]); + crypto.set_single_client_cert(vec![certificate], key)?; + crypto.dangerous().set_certificate_verifier(verifier); Ok(crypto) } +/// Create a TLS server configuration for libp2p. pub fn make_server_config( keypair: &libp2p_core::identity::Keypair, ) -> Result { - let cert = certificate::make_cert(keypair)?; + let (certificate, key) = make_cert_key(keypair)?; + let verifier = Arc::new(verifier::Libp2pCertificateVerifier); + let mut crypto = rustls::ServerConfig::with_ciphersuites(verifier, &TLS13_CIPHERSUITES); + crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; + crypto.set_protocols(&[P2P_ALPN.to_vec()]); + crypto.set_single_cert(vec![certificate], key)?; + Ok(crypto) +} + +/// Create a random private key and certificate signed with this key for rustls. +fn make_cert_key( + keypair: &libp2p_core::identity::Keypair, +) -> Result<(rustls::Certificate, rustls::PrivateKey), ConfigError> { + let cert = certificate::make_certificate(&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::Libp2pCertificateVerifier; - - 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) + Ok((cert, key)) } diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 793d8a91455..0133b797b53 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -1,4 +1,4 @@ -// Copyright 2020 Parity Technologies (UK) Ltd. +// Copyright 2021 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"), @@ -18,15 +18,16 @@ // 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; +//! TLS 1.3 certificates and handshakes handling for libp2p +//! +//! This module handles a verification of a client/server certificate chain +//! and signatures allegedly by the given certificates. + use rustls::{ internal::msgs::handshake::DigitallySignedStruct, Certificate, ClientCertVerified, - HandshakeSignatureValid, ServerCertVerified, TLSError, + DistinguishedNames, HandshakeSignatureValid, RootCertStore, ServerCertVerified, + SignatureScheme, TLSError, }; -use untrusted::{Input, Reader}; -use webpki::Error; /// Implementation of the `rustls` certificate verification traits for libp2p. /// @@ -39,14 +40,38 @@ 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. +impl Libp2pCertificateVerifier { + /// Return the list of SignatureSchemes that this verifier will handle, + /// in `verify_tls12_signature` and `verify_tls13_signature` calls. + /// + /// This should be in priority order, with the most preferred first. + pub fn verification_schemes() -> Vec { + vec![ + // TODO SignatureScheme::ECDSA_NISTP521_SHA512 is not supported by `ring` yet + SignatureScheme::ECDSA_NISTP384_SHA384, + SignatureScheme::ECDSA_NISTP256_SHA256, + // TODO SignatureScheme::ED448 is not supported by `ring` yet + SignatureScheme::ED25519, + // In particular, RSA SHOULD NOT be used unless + // no elliptic curve algorithms are supported. + SignatureScheme::RSA_PSS_SHA512, + SignatureScheme::RSA_PSS_SHA384, + SignatureScheme::RSA_PSS_SHA256, + SignatureScheme::RSA_PKCS1_SHA512, + SignatureScheme::RSA_PKCS1_SHA384, + SignatureScheme::RSA_PKCS1_SHA256, + ] + } +} + impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { fn verify_server_cert( &self, - _roots: &rustls::RootCertStore, - presented_certs: &[rustls::Certificate], + _roots: &RootCertStore, + presented_certs: &[Certificate], _dns_name: webpki::DNSNameRef<'_>, _ocsp_response: &[u8], - ) -> Result { + ) -> Result { verify_presented_certs(presented_certs).map(|_| ServerCertVerified::assertion()) } @@ -56,8 +81,10 @@ impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { _cert: &Certificate, _dss: &DigitallySignedStruct, ) -> Result { + // The libp2p handshake uses TLS 1.3 (and higher). + // Endpoints MUST NOT negotiate lower TLS versions. Err(TLSError::PeerIncompatibleError( - "Only TLS 1.3 is supported".to_string(), + "Only TLS 1.3 certificates are supported".to_string(), )) } @@ -67,7 +94,11 @@ impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { cert: &Certificate, dss: &DigitallySignedStruct, ) -> Result { - verify_tls13_signature(message, cert, dss) + verify_tls13_signature(cert, dss.scheme, message, dss.sig.0.as_ref()) + } + + fn supported_verify_schemes(&self) -> Vec { + Self::verification_schemes() } } @@ -86,7 +117,7 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { fn client_auth_root_subjects( &self, _dns_name: Option<&webpki::DNSName>, - ) -> Option { + ) -> Option { Some(vec![]) } @@ -94,7 +125,7 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { &self, presented_certs: &[Certificate], _dns_name: Option<&webpki::DNSName>, - ) -> Result { + ) -> Result { verify_presented_certs(presented_certs).map(|_| ClientCertVerified::assertion()) } @@ -104,8 +135,10 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { _cert: &Certificate, _dss: &DigitallySignedStruct, ) -> Result { + // The libp2p handshake uses TLS 1.3 (and higher). + // Endpoints MUST NOT negotiate lower TLS versions. Err(TLSError::PeerIncompatibleError( - "Only TLS 1.3 is supported".to_string(), + "Only TLS 1.3 certificates are supported".to_string(), )) } @@ -115,119 +148,37 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { 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()) + verify_tls13_signature(cert, dss.scheme, message, dss.sig.0.as_ref()) } -} - -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 supported_verify_schemes(&self) -> Vec { + Self::verification_schemes() } } -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)) -} - +/// When receiving the certificate chain, an endpoint +/// MUST check these conditions and abort the connection attempt if +/// (a) the presented certificate is not yet valid, OR +/// (b) if it is expired. +/// Endpoints MUST abort the connection attempt if more than one certificate is received, +/// or if the certificate’s self-signature is not valid. 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().spki()) - .map_err(TLSError::WebPKIError)?; - Ok(()) -} - -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> { - // 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; - 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, - }) - }) - }) + crate::tls::certificate::parse_certificate(presented_certs[0].as_ref()) + .and_then(|cert| cert.verify()) + .map_err(TLSError::WebPKIError) } -/// 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) +fn verify_tls13_signature( + cert: &Certificate, + signature_scheme: SignatureScheme, + message: &[u8], + signature: &[u8], +) -> Result { + crate::tls::certificate::parse_certificate(cert.as_ref()) + .and_then(|cert| cert.verify_signature(signature_scheme, message, signature)) + .map(|()| HandshakeSignatureValid::assertion()) + .map_err(TLSError::WebPKIError) } From 2ea0ae75c172fb2e11f720fd0076e7c64112492a Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 15 Dec 2021 04:13:05 +0300 Subject: [PATCH 026/218] Add tests for x509 parser --- transports/quic/src/tls/certificate.rs | 46 +++++++++++++ .../quic/src/tls/test_assets/ed25519.der | Bin 0 -> 324 bytes transports/quic/src/tls/test_assets/ed448.der | Bin 0 -> 400 bytes transports/quic/src/tls/test_assets/gen.sh | 63 ++++++++++++++++++ .../src/tls/test_assets/nistp256_sha256.der | Bin 0 -> 388 bytes .../src/tls/test_assets/nistp384_sha256.der | Bin 0 -> 450 bytes .../src/tls/test_assets/nistp384_sha384.der | Bin 0 -> 450 bytes .../src/tls/test_assets/nistp521_sha512.der | Bin 0 -> 525 bytes .../quic/src/tls/test_assets/openssl.cfg | 6 ++ .../quic/src/tls/test_assets/pkcs1_sha256.der | Bin 0 -> 324 bytes .../src/tls/test_assets/rsa_pkcs1_sha256.der | Bin 0 -> 785 bytes .../src/tls/test_assets/rsa_pkcs1_sha384.der | Bin 0 -> 785 bytes .../src/tls/test_assets/rsa_pkcs1_sha512.der | Bin 0 -> 785 bytes .../src/tls/test_assets/rsa_pss_sha384.der | Bin 0 -> 878 bytes 14 files changed, 115 insertions(+) create mode 100644 transports/quic/src/tls/test_assets/ed25519.der create mode 100644 transports/quic/src/tls/test_assets/ed448.der create mode 100755 transports/quic/src/tls/test_assets/gen.sh create mode 100644 transports/quic/src/tls/test_assets/nistp256_sha256.der create mode 100644 transports/quic/src/tls/test_assets/nistp384_sha256.der create mode 100644 transports/quic/src/tls/test_assets/nistp384_sha384.der create mode 100644 transports/quic/src/tls/test_assets/nistp521_sha512.der create mode 100644 transports/quic/src/tls/test_assets/openssl.cfg create mode 100644 transports/quic/src/tls/test_assets/pkcs1_sha256.der create mode 100644 transports/quic/src/tls/test_assets/rsa_pkcs1_sha256.der create mode 100644 transports/quic/src/tls/test_assets/rsa_pkcs1_sha384.der create mode 100644 transports/quic/src/tls/test_assets/rsa_pkcs1_sha512.der create mode 100644 transports/quic/src/tls/test_assets/rsa_pss_sha384.der diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index 3d35cee4796..6cac34131b9 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -415,4 +415,50 @@ mod tests { assert!(parsed_cert.verify().is_ok()); assert_eq!(keypair.public(), parsed_cert.extension.public_key); } + + macro_rules! check_cert { + ($name:ident, $path:literal, $scheme:path) => { + #[test] + fn $name() { + let cert: &[u8] = include_bytes!($path); + + let cert = certificate::parse_certificate(cert).unwrap(); + // We don't verify certificate because p2p extension was not signed + // with the private key of the certificate. + assert_eq!( + cert.signature_scheme(), + Ok($scheme) + ); + } + }; + } + + check_cert!{ed448, "./test_assets/ed448.der", rustls::SignatureScheme::ED448} + check_cert!{ed25519, "./test_assets/ed25519.der", rustls::SignatureScheme::ED25519} + check_cert!{rsa_pkcs1_sha256, "./test_assets/rsa_pkcs1_sha256.der", rustls::SignatureScheme::RSA_PKCS1_SHA256} + check_cert!{rsa_pkcs1_sha384, "./test_assets/rsa_pkcs1_sha384.der", rustls::SignatureScheme::RSA_PKCS1_SHA384} + check_cert!{rsa_pkcs1_sha512, "./test_assets/rsa_pkcs1_sha512.der", rustls::SignatureScheme::RSA_PKCS1_SHA512} + check_cert!{nistp256_sha256, "./test_assets/nistp256_sha256.der", rustls::SignatureScheme::ECDSA_NISTP256_SHA256} + check_cert!{nistp384_sha384, "./test_assets/nistp384_sha384.der", rustls::SignatureScheme::ECDSA_NISTP384_SHA384} + check_cert!{nistp521_sha512, "./test_assets/nistp521_sha512.der", rustls::SignatureScheme::ECDSA_NISTP521_SHA512} + + #[test] + fn rsa_pss_sha384() { + let cert: &[u8] = include_bytes!("./test_assets/rsa_pss_sha384.der"); + + let cert = certificate::parse_certificate(cert).unwrap(); + cert.verify().unwrap(); // that was a fairly generated certificate. + assert_eq!( + cert.signature_scheme(), + Ok(rustls::SignatureScheme::RSA_PSS_SHA384) + ); + } + + #[test] + fn nistp384_sha256() { + let cert: &[u8] = include_bytes!("./test_assets/nistp384_sha256.der"); + + let cert = certificate::parse_certificate(cert).unwrap(); + assert!(cert.signature_scheme().is_err()); + } } diff --git a/transports/quic/src/tls/test_assets/ed25519.der b/transports/quic/src/tls/test_assets/ed25519.der new file mode 100644 index 0000000000000000000000000000000000000000..494a199561a67047c63aa847ebd5a734d664a974 GIT binary patch literal 324 zcmXqLVstQQ{JemfiIIs(gsZ!KuJPrk8h>J!?Gd%_Vy}DSeZYW~jafUjz<|L(PMp`s z(9p=x)WFctz{E5P$Tb2oO`u$$3N5H&W<`by-3L?mpZR?`CQ59!Z)V-Wt`7G%!P(Va z`JM^-Zl^sCEv`4HHK=Ce(q?01VQgL$#RvrdS+Wc=SX4L|g%s|mOgtj`mczUK#Rs0- zUXDkYWBx7udC6Aw|C|-mpZ~qX&*Cs;#k`<1Dt|S%Y?=E^^l+_ObJC0J#}7>VU2{SB z&J?C_tuGxZ4gZd^A3k8ap-S(B*rsiTF6wtQK36_yos(}Zx|Omf`T3iC8RwaGO^j7t(-$c*dCA@9-p7CM LTr=(RtS{UEyA_6f literal 0 HcmV?d00001 diff --git a/transports/quic/src/tls/test_assets/ed448.der b/transports/quic/src/tls/test_assets/ed448.der new file mode 100644 index 0000000000000000000000000000000000000000..c74123868473acbc8b680c478d80aabc7371d6b7 GIT binary patch literal 400 zcmXqLV(c+!V&qxC%*4pVB*MQwaM@J6(&HYkoYmTlks*D;u+RYM}vxft)z6 zk)ffHp{aqPp@E5M6p(8KWST&^Ko!nV#mrU=*VyfM-|{N)?>iS8a&+&3Y3u()K5aR+ zIm*WxUn14%uUb0pFKWD}C=YQ|;vp7sy zF)!$h%3sYbTjo9!JzT5Sob=-Q@dML-*IW?3GleN!>q|#U!@r~KhY#3psM0$jwrN|T zi~1dn&y^2a=j2<9?q-Ggp_rlWg<_CUmqNG0l_FrDK;;PW5BCaUp$h2rkL mO`5a*>{jW{X%6E$_KeR(Eb8+q&&WykHGDhu;xlitFaQ9G!K6U| literal 0 HcmV?d00001 diff --git a/transports/quic/src/tls/test_assets/gen.sh b/transports/quic/src/tls/test_assets/gen.sh new file mode 100755 index 00000000000..4b7718874dd --- /dev/null +++ b/transports/quic/src/tls/test_assets/gen.sh @@ -0,0 +1,63 @@ +#ED25519 (works): +openssl genpkey -algorithm ed25519 -out privateKey.key +openssl req -new -subj="/" -key privateKey.key -out req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out ed25519.der + +#ED448 (works): +openssl genpkey -algorithm ed448 -out privateKey.key +openssl req -new -subj="/" -key privateKey.key -out req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out ed448.der + +#RSA_PKCS1_SHA256 (works): +openssl genpkey -algorithm rsa -out privateKey.key +openssl req -new -subj="/" -key privateKey.key -out req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -sha256 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out rsa_pkcs1_sha256.der + +#RSA_PKCS1_SHA384 (works): +# reuse privateKey.key and req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -sha384 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out rsa_pkcs1_sha384.der + +#RSA_PKCS1_SHA512 (works): +# reuse privateKey.key and req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -sha512 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out rsa_pkcs1_sha512.der + +#RSA-PSS TODO +# openssl genpkey -algorithm rsa-pss -pkeyopt rsa_keygen_bits:2048 -pkeyopt rsa_keygen_pubexp:3 -out privateKey.key +# # -sigopt rsa_pss_saltlen:20 +# # -sigopt rsa_padding_mode:pss +# # -sigopt rsa_mgf1_md:sha256 +# openssl req -x509 -nodes -days 365 -subj="/" -key privateKey.key -sha256 -sigopt rsa_pss_saltlen:20 -sigopt rsa_padding_mode:pss -sigopt rsa_mgf1_md:sha256 -out certificate.crt + +#ECDSA_NISTP256_SHA256 (works): +openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-256 -out privateKey.key +openssl req -new -subj="/" -key privateKey.key -out req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -sha256 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out nistp256_sha256.der + +#ECDSA_NISTP384_SHA384 (works): +openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-384 -out privateKey.key +openssl req -new -subj="/" -key privateKey.key -out req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -sha384 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out nistp384_sha384.der + +#ECDSA_NISTP521_SHA512 (works): +openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-521 -out privateKey.key +openssl req -new -subj="/" -key privateKey.key -out req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -sha512 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out nistp521_sha512.der + +#ECDSA_NISTP384_SHA256 (must fail): +openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-384 -out privateKey.key +openssl req -new -subj="/" -key privateKey.key -out req.pem +openssl x509 -req -in req.pem -signkey privateKey.key -sha256 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg +openssl x509 -outform der -in certificate.crt -out nistp384_sha256.der + + +# Remove tmp files + +rm req.pem certificate.crt privateKey.key diff --git a/transports/quic/src/tls/test_assets/nistp256_sha256.der b/transports/quic/src/tls/test_assets/nistp256_sha256.der new file mode 100644 index 0000000000000000000000000000000000000000..8023645e9b07e58ab410f71564699cc8433aebe8 GIT binary patch literal 388 zcmXqLVr(#IVpLzi%*4pVBoeuh$A9gw`qI<8*#%$TwHM>@k+^8U#m1r4=5fxJg_+5K z!9Y%&*T~S&$k5cl(9podGz!Qy0y0gYT%d|b17S9Huns0hs8(i1c4j9A7AM7ZKl|>U z^|{d|0l!Z7IyXK-&0$Y zm%ozhEISpuY;nCotwA*#mo^(C3uE)5C`KUo&yr=3!J@*!D5P*dW#SRhw;bN>FFx?x z_HsPJ9P@AK&r7zd|L3fj{`~J9einx*E9M2AQTeO6Wy{=WqK9kMnv-5!KYn1^@0tt3 zccw6fYklcRY4~@P{qOzsUR(cP@bVZrRqVBo@}pf&g3 zq6+cO<~!|rv8ifqPo<1>?AvUWbS{K0Vf^U+O^8XswRaWE=@pM=?bs@wcDHSHh2YAC STQ@y5C|BTrHNV5|x;FscJeIfs literal 0 HcmV?d00001 diff --git a/transports/quic/src/tls/test_assets/nistp384_sha256.der b/transports/quic/src/tls/test_assets/nistp384_sha256.der new file mode 100644 index 0000000000000000000000000000000000000000..5d76fa8f4a90ca3bba0a22150e4805d2ca9380a7 GIT binary patch literal 450 zcmXqLV%%rY#OShsnTe5!Nkl#(dWKqA?wTq_--bescs|x?tDhTiv2kd%d7QIlVP-O5 zFpv}HH8M0bGBhV>ja9pmg+YlqiGjuNWmseH z9Le0PpTAFUy=}kTa^Zx4Gk;#~J0!ko+G1y4v%mg-uB*G(GR7A_TK-f+tgawWX#0Bc zy0kNYFF)Xu$jO|1jxBMnk=@6YGcR`iHDl^K=hTvt*{`|GCcHx6NoM+k#q|cY2Gwj_ z+H8z0jLnOp7=hqFOO`cq%ekGextUSCuds7>a(4+aRfv`!C~?lwexjwC0r@ jpXU1e{N8I>`jY#B^pUBD$~Q(%uW#7c(2*{x@@WzPu{5%@ literal 0 HcmV?d00001 diff --git a/transports/quic/src/tls/test_assets/nistp384_sha384.der b/transports/quic/src/tls/test_assets/nistp384_sha384.der new file mode 100644 index 0000000000000000000000000000000000000000..a81a5ce1ab748be7714c385ae4f3525bd9024fd2 GIT binary patch literal 450 zcmXqLV%%rY#OShsnTe5!Nu=gc^!`1Y^>UiEDTD_NenC=ldlAS z-lX}{j&IKH@XKv(|9?$h-Fdi~adXzn9MP9gnKspYlUZqhJm<*YwpSbzECPgz*Js`? z7y56bbH40i%;WC(w-ZZ!x9{fPQ!RKkN9lLsuMDs24O1RX+Q-5+r|-@alg0H0wFcE} zT-t1mER4;Iq8NeTKTDQD28#*@qmaV=l!-?~-*R}jzxcp&+sp9?bIiY`KQGy;{-3jA z`t!ec_*opLte6*cM&+;OmMwFii5{+1YfgG`{rG`tziTcC-}PyRYgN zC(DVX8|v-Lrrsz%C=+1Fx=}fyNxddUMXYWeTi^!~CIio?eBN}+>C6G^l>dC5Y#Z=n lNz%U>Th1%0D~@{HF0c6_Joi(}?Ch7f&pvow&e6Qe1OS~`yc+-j literal 0 HcmV?d00001 diff --git a/transports/quic/src/tls/test_assets/nistp521_sha512.der b/transports/quic/src/tls/test_assets/nistp521_sha512.der new file mode 100644 index 0000000000000000000000000000000000000000..2846361f278e37f4338e35848304af02af4721e5 GIT binary patch literal 525 zcmXqLV&XJtV$52=%*4pVBqDR5TjaPvpWN2z&XlP(!8U^GU0huOHE>K0|Yy$x{cCZ#EMmARMMivHT=EgP#7Dka- zL6Wn6m|iqy516L5b>s6TtB%xP{c$aQ?$V%ZPgi$sZ+4p}wtwyY;FcE`)-f*XWvR1avZ&&|oGI`uZV*mRCW#>&gT-yYb~l)AT5b#c8xtwA*#mo^(C3uE)5C`KUo z&yr=3!J@*!D5P*dW#SRhw;bN>FFx?x_HsPJ9P@AK&r7zd|L3fj{`~J9einx*E9M2A zQTeO6Wy{=WqK9kMnv-5!KYn1^@0tt3ccw6fYklcRY4~@P{qOzsUR(cP@b!O7g%0}QSXCMSj|%2T3ly_8jatJb(T)^6+0-`4B(ZMk|iKikOg zE7wV^x6;2mb-V1e$mx?#HTE(@+^))3nl5JWdGW>YsO9R{W@R)mIWa!J;`~DKE5koE z<)Zly_Dr-$&9`PRS9U2HPnZ=8L8z9UOtY2y{g=1#{X%U7@a7CkZ0 LW8!JtW5NgkxUAVe literal 0 HcmV?d00001 diff --git a/transports/quic/src/tls/test_assets/openssl.cfg b/transports/quic/src/tls/test_assets/openssl.cfg new file mode 100644 index 00000000000..62f02baee8b --- /dev/null +++ b/transports/quic/src/tls/test_assets/openssl.cfg @@ -0,0 +1,6 @@ +[ p2p_ext ] +1.3.6.1.4.1.53594.1.1 = critical,ASN1:SEQUENCE:ExtBody + +[ ExtBody ] +pubkey = FORMAT:HEX,OCTETSTRING:08011220DF6491C415ED084B87E8F00CDB4A41C4035CFEA5F9D23D25FF9CA897E7FDDC0F +signature = FORMAT:HEX,OCTETSTRING:94A89E52CC24FD29B4B49DE615C37D268362E8D7C7C096FB7CD013DC9402572AF4886480FEC507C3C03DB07A2EC816B2B6714427DC28F379E0859C6F3B15BB05 diff --git a/transports/quic/src/tls/test_assets/pkcs1_sha256.der b/transports/quic/src/tls/test_assets/pkcs1_sha256.der new file mode 100644 index 0000000000000000000000000000000000000000..0449728ee28cbf651c604319dde98adccf09a972 GIT binary patch literal 324 zcmXqLVstQQ{JemfiIIs(M8EgZMO6_-S-n@gy3W5hD&ec=e{I0Z#;l!MV8CD?C(dhR zXlP_;Vq|D!YG4=z2DJv&Y+TxGj4X`Ji=r5T;6F>2K?aKo2cwX}{gjDEMBj3Fx4-zn zbKA@D2y@K8r9Us(s{Ws|V*2yHclcQxrmUD3bVlW`=9VpUpNSr>RclUqasBv#X}@bO z2;Z5)6t4B9BchN82d#7RtwndU!kp^JaLP?jrthO} zWJs;p?nk8;%NP1Nmzb63-iWqi3$a_aT_`a)YN6m+m89z*Z?;?i{eM(y=|t71f4*23 M%(xjA&LNQU!^hIZ7oGD#4|0^Herbh^>%92}FkhHa7utKm zX+DXLrckXxX&9^4=l>jM(E6~^c2D%J0 z@2gk9aBCcbJH4MnIN0VkvFc0+_}*^0VU|*QD&PtUG8S1y_DSehdfJ@7z~@G674Iu1 zLUEn2Bwz&)^fF31+LwsAx!+$i&T;DwxTI~BVHFbxI? zDuzgg_YDC73k3iJf&l>ls{O>va6j?)%|O$*2S_6zx_OsT=e})EL9`9%VdYzD?;oNlFtjR)LJf9x2dSf3xgE<8j*6V5KhZ z1kw>ELCebYN!v&&LNQU!^hIZ7oGD#4|0^Herbh^>%92}FkhHa7utKm zX+DXLrckXxX&9^4=l>jM(E6~^c2D%J0 z@2gk9aBCcbJH4MnIN0VkvFc0+_}*^0VU|*QD&PtUG8S1y_DSehdfJ@7z~@G674Iu1 zLUEn2Bwz&)^fF31+LwsAx!+$i&T;DwxTI~BVHFbxI? zDuzgg_YDC73lAYYhX=D?z&@=>dr?(eP@!eG}fqOFo%}ZW#*l76S6i> zZ1KzN`906wsd>zF`I^0zaN}~+b}pHlqZglMunBk($nm+p9it)W3g5lR!##vLawn|v z60*daTN04wlT8`+hs_S$UNc8*1%WZ6snSs?x0f=yQ&?r!4u{s?A=+ML3A1pxkgy^_ zm}b+S!ZuVb7#oKWf@~z4H2W^Luvc;>FKAMJ1*`5u=xBKDF+pTLQUYDmcW8DIPJ@W; PfHNE258&LNQU!^hIZ7oGD#4|0^Herbh^>%92}FkhHa7utKm zX+DXLrckXxX&9^4=l>jM(E6~^c2D%J0 z@2gk9aBCcbJH4MnIN0VkvFc0+_}*^0VU|*QD&PtUG8S1y_DSehdfJ@7z~@G674Iu1 zLUEn2Bwz&)^fF31+LwsAx!+$i&T;DwxTI~BVHFbxI? zDuzgg_YDC74Fv!Lf&l>lBDp_*l%n5u2o|HCz*VVC^{6CJm2jtSz-!c}4ao5QWO?hx z)1V(q2CcODGd4Jo^_){yo%|YxwKz4L#8LM{&w(PNI%VZ=vg(V|u1hz|-?aC|(Su}p z4BWTyQ$#^356{ToI-QLQfcf~WrZMAt^HL|tHAA#Iei|H-TlJDia@lNbKdt6QXt^mm z=@DU`6NPW5I%k5fQnftOKT<9-z+E=NI|#!P8V&W%HogIa>E)jUk61O_DT@suIFc18 zR0rsQrH7CX$>dO9665jGrO!rm%p2%zn_ZjXoLHSakH}9!39870w0HHFz{|#&(B{FI z%FM#V#LBQx#y|?8f&)!<5zsIL0|o;n34Q|u14A$bG7Jo&B>0UCjSLOUjDVD>fuT_p zP*oG75=cMHI!0Co<|amdkT@4p6C)$TwQ$SX3P%e9pUOXD@ikd{+^y;P@x5yvdjIj8 z--{%t>mA@2B}B&UzG|A?kMT%+b986Wu0Bay^@I z^H1}>O$k;EOP5c0bh)fa$Ys&vq<$ys`@1YS?kZ+Zzs+1{m*e6x=k)@HAFpnPPTj5k zxq8aqPb_=P!$iz}Iu}=eepQukJmF8fzwKwfO$mI@`2tJ5&ITB0c|K#iuahsnb4|an zkH3n4%6>MDEY>X_7 z&5NQK8CkLnGFVhN7=;w}XFX-n+bWZ?{g!*pF2|ruM!%Eq{ie){6`j>#CBN4*hsEKX z(e?!u4wLP6G}+0!|K53&eWU-mP332giU+;EsS;LmxqjioGx=Nxt~Z~tZg=~8Bv^3I zt&J58-^#z#zWc;JnSqrG$pILrj0_AJ{kQu*S3hp{nrfoiTfHWJVdLk#=;iZ%hsDJ9 zZU}t%C__JM$H9_26YD*Av^q~Ribe^ybJg$7P7%9!?kH#f=Z(?V54E_?+iW#kNn-9} znPlJVnph{l~n?D^Fb!O|zdKy;*vejm#bg zYh8Oy_kw*#gsg6#-uXeK>1+75vu}O=rcc_-?dX4W#;3{lb2t1Ky{qmTvHg8d!nUr0 zO@D$fp8L^ZAkF_=RAApphvPFqd%_=&@}5{08G6=uNA#6FW$(YK lNgdWa{IqpPkGqoH+}o-4D^nglx@N$}K5+wIlp(ixFaY)iXKMfe literal 0 HcmV?d00001 From 0e8e5b5e5aeb1edcde1c89b192cfc508e8706d90 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 15 Dec 2021 16:09:37 +0300 Subject: [PATCH 027/218] Upgrade quinn-proto, rustls, webpki --- transports/quic/Cargo.toml | 6 +- transports/quic/src/crypto.rs | 17 +----- transports/quic/src/endpoint.rs | 32 ++++------- transports/quic/src/muxer.rs | 13 +++-- transports/quic/src/tls/certificate.rs | 12 ++-- transports/quic/src/tls/mod.rs | 39 ++++++++----- transports/quic/src/tls/verifier.rs | 80 ++++++++++++++++---------- 7 files changed, 101 insertions(+), 98 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 7792ce32046..e5956278663 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -16,14 +16,14 @@ futures = "0.3.15" if-watch = "0.2.2" libp2p-core = { version = "0.31.0", path = "../../core" } parking_lot = "0.11.1" -quinn-proto = { version = "0.7.3", default-features = false, features = ["tls-rustls"] } +quinn-proto = { version = "0.8.0", default-features = false, features = ["tls-rustls"] } rcgen = { version = "0.8.11" } ring = { version = "0.16.20" } -rustls = { version = "0.19.1", features = ["dangerous_configuration"] } +rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } thiserror = "1.0.26" tracing = "0.1.26" udp-socket = "0.1.5" -webpki = { version = "0.21.4", features = ["std"] } +webpki = "0.22.0" x509-parser = "0.12.0" yasna = { version = "0.4.0" } diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs index 145990c7973..de4e482373a 100644 --- a/transports/quic/src/crypto.rs +++ b/transports/quic/src/crypto.rs @@ -19,7 +19,6 @@ // DEALINGS IN THE SOFTWARE. use libp2p_core::identity::Keypair; -use quinn_proto::crypto::Session; use quinn_proto::TransportConfig; use std::sync::Arc; @@ -33,9 +32,7 @@ pub struct CryptoConfig { pub(crate) struct TlsCrypto; impl TlsCrypto { - pub fn new_server_config( - config: &CryptoConfig, - ) -> ::ServerConfig { + pub fn new_server_config(config: &CryptoConfig) -> Arc { 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; @@ -43,9 +40,7 @@ impl TlsCrypto { Arc::new(server) } - pub fn new_client_config( - config: &CryptoConfig, - ) -> ::ClientConfig { + pub fn new_client_config(config: &CryptoConfig) -> Arc { let mut client = crate::tls::make_client_config(&config.keypair).expect("invalid config"); if let Some(key_log) = config.keylogger.clone() { client.key_log = key_log; @@ -53,14 +48,6 @@ impl TlsCrypto { Arc::new(client) } - pub fn supported_quic_versions() -> Vec { - quinn_proto::DEFAULT_SUPPORTED_VERSIONS.to_vec() - } - - pub fn default_quic_version() -> u32 { - quinn_proto::DEFAULT_SUPPORTED_VERSIONS[0] - } - pub fn keylogger() -> Arc { Arc::new(rustls::KeyLogFile::new()) } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 728bb11948e..50e01c5ae38 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -23,10 +23,10 @@ use crate::muxer::QuicMuxer; use crate::{QuicConfig, QuicError}; use futures::channel::{mpsc, oneshot}; use futures::prelude::*; -use quinn_proto::generic::ClientConfig as QuinnClientConfig; -use quinn_proto::ServerConfig as QuinnServerConfig; use quinn_proto::{ - ConnectionEvent, ConnectionHandle, DatagramEvent, EcnCodepoint, EndpointEvent, Transmit, + ClientConfig as QuinnClientConfig, ConnectionEvent, ConnectionHandle, DatagramEvent, + EcnCodepoint, Endpoint as QuinnEndpoint, EndpointEvent, ServerConfig as QuinnServerConfig, + Transmit, }; use std::collections::{HashMap, VecDeque}; use std::io::IoSliceMut; @@ -160,9 +160,6 @@ impl EndpointChannel { } } -type QuinnEndpointConfig = quinn_proto::EndpointConfig; -type QuinnEndpoint = quinn_proto::Endpoint; - pub struct EndpointConfig { socket: UdpSocket, endpoint: QuinnEndpoint, @@ -173,7 +170,7 @@ pub struct EndpointConfig { impl EndpointConfig { pub fn new(mut config: QuicConfig, addr: SocketAddr) -> Result { - config.transport.max_concurrent_uni_streams(0)?; + config.transport.max_concurrent_uni_streams(0u32.into()); config.transport.datagram_receive_buffer_size(None); let transport = Arc::new(config.transport); @@ -183,22 +180,14 @@ impl EndpointConfig { transport: transport.clone(), }); - let mut server_config = QuinnServerConfig::default(); + let crypto = TlsCrypto::new_server_config(&crypto_config); + let mut server_config = QuinnServerConfig::with_crypto(crypto); server_config.transport = transport; - server_config.crypto = TlsCrypto::new_server_config(&crypto_config); - let mut endpoint_config = QuinnEndpointConfig::default(); - endpoint_config.supported_versions( - TlsCrypto::supported_quic_versions(), - TlsCrypto::default_quic_version(), - )?; + let endpoint = QuinnEndpoint::new(Default::default(), Some(Arc::new(server_config))); 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, @@ -342,10 +331,9 @@ impl Future for Endpoint { match event { Some(ToEndpoint::Dial { addr, tx }) => { let crypto = TlsCrypto::new_client_config(&me.crypto_config); - let client_config = QuinnClientConfig { - transport: me.crypto_config.transport.clone(), - crypto, - }; + let mut client_config = QuinnClientConfig::new(crypto); + client_config.transport = me.crypto_config.transport.clone(); + let (id, connection) = match me.endpoint.connect(client_config, addr, "server_name") { Ok(c) => c, diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 491aa7f038e..45cd5f06c8c 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -100,9 +100,11 @@ impl QuicMuxer { pub fn peer_id(&self) -> Option { let inner = self.inner.lock(); let session = inner.connection.crypto_session(); - let certificate = session.get_peer_certificates()?.into_iter().next()?; - let certificate = crate::tls::certificate::parse_certificate(&certificate.0).ok()?; - Some(PeerId::from_public_key(&certificate.extension.public_key)) + let identity = session.peer_identity()?; + let certs: Box> = identity.downcast().ok()?; + let cert = certs.get(0)?; + let p2p_cert = crate::tls::certificate::parse_certificate(&cert.0).ok()?; + Some(PeerId::from_public_key(&p2p_cert.extension.public_key)) } pub fn local_addr(&self) -> Multiaddr { @@ -143,9 +145,8 @@ impl StreamMuxer for QuicMuxer { inner.connection.handle_event(event); } - // TODO: `quinn` got plans to add max_datagrams as an arg to `poll_transmit` - // let _max_datagrams = inner.endpoint.max_datagrams(); - while let Some(transmit) = inner.connection.poll_transmit(now) { + let max_datagrams = inner.endpoint.max_datagrams(); + while let Some(transmit) = inner.connection.poll_transmit(now, max_datagrams) { inner.endpoint.send_transmit(transmit); } diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index 6cac34131b9..dd1e55a1b84 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -124,7 +124,7 @@ pub fn parse_certificate(der_input: &[u8]) -> Result Result Result { Some(hash_oid) } - let hash_oid = get_hash_oid(signature_algorithm).ok_or(Error::BadDER)?; + let hash_oid = get_hash_oid(signature_algorithm).ok_or(Error::BadDer)?; if hash_oid == OID_NIST_HASH_SHA256 { return Ok(RSA_PSS_SHA256); @@ -310,9 +310,9 @@ impl P2pCertificate<'_> { let signature_param = pki_algorithm .parameters .as_ref() - .ok_or(Error::BadDER)? + .ok_or(Error::BadDer)? .as_oid_val() - .map_err(|_| Error::BadDER)?; + .map_err(|_| Error::BadDer)?; if signature_param == OID_EC_P256 && signature_algorithm.algorithm == OID_SIG_ECDSA_WITH_SHA256 { diff --git a/transports/quic/src/tls/mod.rs b/transports/quic/src/tls/mod.rs index f4a1fc84a32..5183ef57955 100644 --- a/transports/quic/src/tls/mod.rs +++ b/transports/quic/src/tls/mod.rs @@ -27,7 +27,7 @@ use std::sync::Arc; use thiserror::Error; use rustls::{ - ciphersuite::{ + cipher_suite::{ TLS13_AES_128_GCM_SHA256, TLS13_AES_256_GCM_SHA384, TLS13_CHACHA20_POLY1305_SHA256, }, SupportedCipherSuite, @@ -36,11 +36,11 @@ use rustls::{ /// A list of the TLS 1.3 cipher suites supported by rustls. // By default rustls creates client/server configs with both // TLS 1.3 __and__ 1.2 cipher suites. But we don't need 1.2. -static TLS13_CIPHERSUITES: [&SupportedCipherSuite; 3] = [ +static TLS13_CIPHERSUITES: &[SupportedCipherSuite] = &[ // TLS1.3 suites - &TLS13_CHACHA20_POLY1305_SHA256, - &TLS13_AES_256_GCM_SHA384, - &TLS13_AES_128_GCM_SHA256, + TLS13_CHACHA20_POLY1305_SHA256, + TLS13_AES_256_GCM_SHA384, + TLS13_AES_128_GCM_SHA256, ]; const P2P_ALPN: [u8; 6] = *b"libp2p"; @@ -50,7 +50,7 @@ const P2P_ALPN: [u8; 6] = *b"libp2p"; pub enum ConfigError { /// TLS private key or certificate rejected #[error("TLS private or certificate key rejected: {0}")] - TLSError(#[from] rustls::TLSError), + TLSError(#[from] rustls::Error), /// Certificate generation error #[error("Certificate generation error: {0}")] RcgenError(#[from] rcgen::RcgenError), @@ -62,11 +62,15 @@ pub fn make_client_config( ) -> Result { let (certificate, key) = make_cert_key(keypair)?; let verifier = Arc::new(verifier::Libp2pCertificateVerifier); - let mut crypto = rustls::ClientConfig::with_ciphersuites(&TLS13_CIPHERSUITES); - crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; - crypto.set_protocols(&[P2P_ALPN.to_vec()]); - crypto.set_single_client_cert(vec![certificate], key)?; - crypto.dangerous().set_certificate_verifier(verifier); + let mut crypto = rustls::ClientConfig::builder() + .with_cipher_suites(TLS13_CIPHERSUITES) + .with_safe_default_kx_groups() + .with_protocol_versions(&[&rustls::version::TLS13]) + .expect("Cipher suites and kx groups are configured; qed") + .with_custom_certificate_verifier(verifier) + .with_single_cert(vec![certificate], key) + .expect("Client cert key DER is valid; qed"); + crypto.alpn_protocols = vec![P2P_ALPN.to_vec()]; Ok(crypto) } @@ -76,10 +80,15 @@ pub fn make_server_config( ) -> Result { let (certificate, key) = make_cert_key(keypair)?; let verifier = Arc::new(verifier::Libp2pCertificateVerifier); - let mut crypto = rustls::ServerConfig::with_ciphersuites(verifier, &TLS13_CIPHERSUITES); - crypto.versions = vec![rustls::ProtocolVersion::TLSv1_3]; - crypto.set_protocols(&[P2P_ALPN.to_vec()]); - crypto.set_single_cert(vec![certificate], key)?; + let mut crypto = rustls::ServerConfig::builder() + .with_cipher_suites(TLS13_CIPHERSUITES) + .with_safe_default_kx_groups() + .with_protocol_versions(&[&rustls::version::TLS13]) + .expect("Cipher suites and kx groups are configured; qed") + .with_client_cert_verifier(verifier) + .with_single_cert(vec![certificate], key) + .expect("Server cert key DER is valid; qed"); + crypto.alpn_protocols = vec![P2P_ALPN.to_vec()]; Ok(crypto) } diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 0133b797b53..2ca6fa45b6b 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -24,9 +24,10 @@ //! and signatures allegedly by the given certificates. use rustls::{ - internal::msgs::handshake::DigitallySignedStruct, Certificate, ClientCertVerified, - DistinguishedNames, HandshakeSignatureValid, RootCertStore, ServerCertVerified, - SignatureScheme, TLSError, + client::{HandshakeSignatureValid, ServerCertVerified, ServerCertVerifier}, + internal::msgs::handshake::DigitallySignedStruct, + server::{ClientCertVerified, ClientCertVerifier}, + Certificate, DistinguishedNames, Error as TlsError, SignatureScheme, }; /// Implementation of the `rustls` certificate verification traits for libp2p. @@ -64,15 +65,17 @@ impl Libp2pCertificateVerifier { } } -impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { +impl ServerCertVerifier for Libp2pCertificateVerifier { fn verify_server_cert( &self, - _roots: &RootCertStore, - presented_certs: &[Certificate], - _dns_name: webpki::DNSNameRef<'_>, + end_entity: &Certificate, + intermediates: &[Certificate], + _server_name: &rustls::ServerName, + _scts: &mut dyn Iterator, _ocsp_response: &[u8], - ) -> Result { - verify_presented_certs(presented_certs).map(|_| ServerCertVerified::assertion()) + _now: std::time::SystemTime, + ) -> Result { + verify_presented_certs(end_entity, intermediates).map(|_| ServerCertVerified::assertion()) } fn verify_tls12_signature( @@ -80,10 +83,10 @@ impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { _message: &[u8], _cert: &Certificate, _dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { // The libp2p handshake uses TLS 1.3 (and higher). // Endpoints MUST NOT negotiate lower TLS versions. - Err(TLSError::PeerIncompatibleError( + Err(TlsError::PeerIncompatibleError( "Only TLS 1.3 certificates are supported".to_string(), )) } @@ -93,7 +96,7 @@ impl rustls::ServerCertVerifier for Libp2pCertificateVerifier { message: &[u8], cert: &Certificate, dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { verify_tls13_signature(cert, dss.scheme, message, dss.sig.0.as_ref()) } @@ -109,24 +112,22 @@ 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. -impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { +impl ClientCertVerifier for Libp2pCertificateVerifier { fn offer_client_auth(&self) -> bool { true } - fn client_auth_root_subjects( - &self, - _dns_name: Option<&webpki::DNSName>, - ) -> Option { + fn client_auth_root_subjects(&self) -> 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()) + end_entity: &Certificate, + intermediates: &[Certificate], + _now: std::time::SystemTime, + ) -> Result { + verify_presented_certs(end_entity, intermediates).map(|_| ClientCertVerified::assertion()) } fn verify_tls12_signature( @@ -134,10 +135,10 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { _message: &[u8], _cert: &Certificate, _dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { // The libp2p handshake uses TLS 1.3 (and higher). // Endpoints MUST NOT negotiate lower TLS versions. - Err(TLSError::PeerIncompatibleError( + Err(TlsError::PeerIncompatibleError( "Only TLS 1.3 certificates are supported".to_string(), )) } @@ -147,7 +148,7 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { message: &[u8], cert: &Certificate, dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { verify_tls13_signature(cert, dss.scheme, message, dss.sig.0.as_ref()) } @@ -162,13 +163,18 @@ impl rustls::ClientCertVerifier for Libp2pCertificateVerifier { /// (b) if it is expired. /// Endpoints MUST abort the connection attempt if more than one certificate is received, /// or if the certificate’s self-signature is not valid. -fn verify_presented_certs(presented_certs: &[Certificate]) -> Result<(), TLSError> { - if presented_certs.len() != 1 { - return Err(TLSError::NoCertificatesPresented); +fn verify_presented_certs( + end_entity: &Certificate, + intermediates: &[Certificate], +) -> Result<(), TlsError> { + if !intermediates.is_empty() { + return Err(TlsError::General( + "libp2p-tls requires exactly one certificate".into(), + )); } - crate::tls::certificate::parse_certificate(presented_certs[0].as_ref()) + crate::tls::certificate::parse_certificate(end_entity.as_ref()) .and_then(|cert| cert.verify()) - .map_err(TLSError::WebPKIError) + .map_err(pki_error) } fn verify_tls13_signature( @@ -176,9 +182,21 @@ fn verify_tls13_signature( signature_scheme: SignatureScheme, message: &[u8], signature: &[u8], -) -> Result { +) -> Result { crate::tls::certificate::parse_certificate(cert.as_ref()) .and_then(|cert| cert.verify_signature(signature_scheme, message, signature)) .map(|()| HandshakeSignatureValid::assertion()) - .map_err(TLSError::WebPKIError) + .map_err(pki_error) +} + +fn pki_error(error: webpki::Error) -> TlsError { + use webpki::Error::*; + match error { + BadDer | BadDerTime => TlsError::InvalidCertificateEncoding, + InvalidSignatureForPublicKey => TlsError::InvalidCertificateSignature, + UnsupportedSignatureAlgorithm | UnsupportedSignatureAlgorithmForPublicKey => { + TlsError::InvalidCertificateSignatureType + } + e => TlsError::InvalidCertificateData(format!("invalid peer certificate: {}", e)), + } } From f5236a8feae07b30c9c34cfd8159dc3538635e1d Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 15 Dec 2021 16:10:26 +0300 Subject: [PATCH 028/218] Fix fmt --- transports/quic/src/tls/certificate.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index dd1e55a1b84..7bcb3993458 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -433,14 +433,14 @@ mod tests { }; } - check_cert!{ed448, "./test_assets/ed448.der", rustls::SignatureScheme::ED448} - check_cert!{ed25519, "./test_assets/ed25519.der", rustls::SignatureScheme::ED25519} - check_cert!{rsa_pkcs1_sha256, "./test_assets/rsa_pkcs1_sha256.der", rustls::SignatureScheme::RSA_PKCS1_SHA256} - check_cert!{rsa_pkcs1_sha384, "./test_assets/rsa_pkcs1_sha384.der", rustls::SignatureScheme::RSA_PKCS1_SHA384} - check_cert!{rsa_pkcs1_sha512, "./test_assets/rsa_pkcs1_sha512.der", rustls::SignatureScheme::RSA_PKCS1_SHA512} - check_cert!{nistp256_sha256, "./test_assets/nistp256_sha256.der", rustls::SignatureScheme::ECDSA_NISTP256_SHA256} - check_cert!{nistp384_sha384, "./test_assets/nistp384_sha384.der", rustls::SignatureScheme::ECDSA_NISTP384_SHA384} - check_cert!{nistp521_sha512, "./test_assets/nistp521_sha512.der", rustls::SignatureScheme::ECDSA_NISTP521_SHA512} + check_cert! {ed448, "./test_assets/ed448.der", rustls::SignatureScheme::ED448} + check_cert! {ed25519, "./test_assets/ed25519.der", rustls::SignatureScheme::ED25519} + check_cert! {rsa_pkcs1_sha256, "./test_assets/rsa_pkcs1_sha256.der", rustls::SignatureScheme::RSA_PKCS1_SHA256} + check_cert! {rsa_pkcs1_sha384, "./test_assets/rsa_pkcs1_sha384.der", rustls::SignatureScheme::RSA_PKCS1_SHA384} + check_cert! {rsa_pkcs1_sha512, "./test_assets/rsa_pkcs1_sha512.der", rustls::SignatureScheme::RSA_PKCS1_SHA512} + check_cert! {nistp256_sha256, "./test_assets/nistp256_sha256.der", rustls::SignatureScheme::ECDSA_NISTP256_SHA256} + check_cert! {nistp384_sha384, "./test_assets/nistp384_sha384.der", rustls::SignatureScheme::ECDSA_NISTP384_SHA384} + check_cert! {nistp521_sha512, "./test_assets/nistp521_sha512.der", rustls::SignatureScheme::ECDSA_NISTP521_SHA512} #[test] fn rsa_pss_sha384() { From 24b47b3e7f47dfeae6756a16247f85faff3188ae Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 15 Dec 2021 16:11:08 +0300 Subject: [PATCH 029/218] Check that generated certs are parsed but not valid --- transports/quic/src/tls/certificate.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index 7bcb3993458..b856ddcc855 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -423,12 +423,10 @@ mod tests { let cert: &[u8] = include_bytes!($path); let cert = certificate::parse_certificate(cert).unwrap(); - // We don't verify certificate because p2p extension was not signed - // with the private key of the certificate. - assert_eq!( - cert.signature_scheme(), - Ok($scheme) - ); + assert!(cert.verify().is_err()); // Because p2p extension + // was not signed with the private key + // of the certificate. + assert_eq!(cert.signature_scheme(), Ok($scheme)); } }; } From b92a6608f00ddc77a11e1025e7d27445c5ef1e02 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 15 Dec 2021 16:11:35 +0300 Subject: [PATCH 030/218] Add idle timeout for smoke tests --- transports/quic/tests/smoke.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 4135a08bbf6..bf6f7f4008d 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -21,6 +21,9 @@ async fn create_swarm(keylog: bool) -> Result>> let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let mut transport = QuicConfig::new(keypair); + transport + .transport + .max_idle_timeout(Some(quinn_proto::VarInt::from_u32(1_000u32).into())); if keylog { transport.enable_keylogger(); } From 4a317df41adccb0a1dd65ad3df049683fa2ca4ea Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 2 Feb 2022 17:12:16 +0300 Subject: [PATCH 031/218] Replace everything with old code --- transports/quic/Cargo.toml | 4 + transports/quic/src/connection.rs | 435 ++++++++++++++ transports/quic/src/crypto.rs | 54 -- transports/quic/src/endpoint.rs | 906 +++++++++++++++++------------- transports/quic/src/error.rs | 90 +++ transports/quic/src/lib.rs | 112 ++-- transports/quic/src/muxer.rs | 516 +++++++---------- transports/quic/src/transport.rs | 408 +++++--------- transports/quic/src/upgrade.rs | 87 +++ 9 files changed, 1536 insertions(+), 1076 deletions(-) create mode 100644 transports/quic/src/connection.rs delete mode 100644 transports/quic/src/crypto.rs create mode 100644 transports/quic/src/error.rs create mode 100644 transports/quic/src/upgrade.rs diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index e5956278663..0520f08c04f 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -27,6 +27,10 @@ webpki = "0.22.0" x509-parser = "0.12.0" yasna = { version = "0.4.0" } +async-std = "*" +log = "*" +futures-timer = "*" + [dev-dependencies] anyhow = "1.0.41" async-std = { version = "1.9.0", features = ["attributes"] } diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs new file mode 100644 index 00000000000..f65ca718af0 --- /dev/null +++ b/transports/quic/src/connection.rs @@ -0,0 +1,435 @@ +// 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. + +//! A single QUIC connection. +//! +//! The [`Connection`] struct of this module contains, amongst other things, a +//! [`quinn_proto::Connection`] state machine and an `Arc`. This struct is responsible +//! for communication between quinn_proto's connection and its associated endpoint. +//! All interactions with a QUIC connection should be done through this struct. +// TODO: docs + +use crate::endpoint::Endpoint; + +use futures::{channel::mpsc, prelude::*}; +use std::{ + fmt, + net::SocketAddr, + pin::Pin, + sync::Arc, + task::{Context, Poll}, + time::Instant, +}; + +/// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Upgrade`]. +/// +/// Contains everything needed to process a connection with a remote. +/// Tied to a specific [`crate::Endpoint`]. +pub(crate) struct Connection { + /// Endpoint this connection belongs to. + endpoint: Arc, + /// Future whose job is to send a message to the endpoint. Only one at a time. + pending_to_endpoint: Option + Send + Sync>>>, + /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. + /// Passed at initialization. + from_endpoint: mpsc::Receiver, + + /// The QUIC state machine for this specific connection. + connection: quinn_proto::Connection, + /// Identifier for this connection according to the endpoint. Used when sending messages to + /// the endpoint. + connection_id: quinn_proto::ConnectionHandle, + /// `Future` that triggers at the `Instant` that `self.connection.poll_timeout()` indicates. + next_timeout: Option, + + /// In other to avoid race conditions where a "connected" event happens if we were not + /// handshaking, we cache whether the connection is handshaking and only set this to true + /// after a "connected" event has been received. + /// + /// In other words, this flag indicates whether a "connected" hasn't been received yet. + is_handshaking: bool, + /// Contains a `Some` if the connection is closed, with the reason of the closure. + /// Contains `None` if it is still open. + /// Contains `Some` if and only if a `ConnectionLost` event has been emitted. + closed: Option, +} + +/// Error on the connection as a whole. +#[derive(Debug, Clone, thiserror::Error)] +pub enum Error { + /// Endpoint has force-killed this connection because it was too busy. + #[error("Endpoint has force-killed our connection")] + ClosedChannel, + /// Error in the inner state machine. + #[error("{0}")] + Quinn(#[from] quinn_proto::ConnectionError), +} + +impl Connection { + /// Crate-internal function that builds a [`Connection`] from raw components. + /// + /// This function assumes that there exists a background task that will process the messages + /// sent to `to_endpoint` and send us messages on `from_endpoint`. + /// + /// The `from_endpoint` can be purposefully closed by the endpoint if the connection is too + /// slow to process. + // TODO: is this necessary ^? figure out if quinn_proto doesn't forbid that situation in the first place + /// + /// `connection_id` is used to identify the local connection in the messages sent to + /// `to_endpoint`. + /// + /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of + /// its methods has ever been called. Failure to comply might lead to logic errors and panics. + // TODO: maybe abstract `to_endpoint` more and make it generic? dunno + pub(crate) fn from_quinn_connection( + endpoint: Arc, + connection: quinn_proto::Connection, + connection_id: quinn_proto::ConnectionHandle, + from_endpoint: mpsc::Receiver, + ) -> Self { + assert!(!connection.is_closed()); + let is_handshaking = connection.is_handshaking(); + + Connection { + endpoint, + pending_to_endpoint: None, + connection, + next_timeout: None, + from_endpoint, + connection_id, + is_handshaking, + closed: None, + } + } + + /// Returns the certificates sent by the remote through the underlying TLS session. + /// Returns `None` if the connection is still handshaking. + // TODO: it seems to happen that is_handshaking is false but this returns None + pub(crate) fn peer_certificates( + &self, + ) -> Option> { + self.connection + .crypto_session() + .get_peer_certificates() + .map(|l| l.into_iter().map(|l| l.into())) + } + + /// Returns the address of the node we're connected to. + // TODO: can change /!\ + pub(crate) fn remote_addr(&self) -> SocketAddr { + self.connection.remote_address() + } + + /// Returns `true` if this connection is still pending. Returns `false` if we are connected to + /// the remote or if the connection is closed. + pub(crate) fn is_handshaking(&self) -> bool { + self.is_handshaking + } + + /// If the connection is closed, returns why. If the connection is open, returns `None`. + /// + /// > **Note**: This method is also the main way to determine whether a connection is closed. + pub(crate) fn close_reason(&self) -> Option<&Error> { + debug_assert!(!self.is_handshaking); + self.closed.as_ref() + } + + /// Start closing the connection. A [`ConnectionEvent::ConnectionLost`] event will be + /// produced in the future. + pub(crate) fn close(&mut self) { + // TODO: what if the user calls this multiple times? + // We send a dummy `0` error code with no message, as the API of StreamMuxer doesn't + // support this. + self.connection + .close(Instant::now(), From::from(0u32), Default::default()); + } + + /// Pops a new substream opened by the remote. + /// + /// If `None` is returned, then a [`ConnectionEvent::StreamAvailable`] event will later be + /// produced when a substream is available. + pub(crate) fn pop_incoming_substream(&mut self) -> Option { + self.connection.accept(quinn_proto::Dir::Bi) + } + + /// Pops a new substream opened locally. + /// + /// The API can be thought as if outgoing substreams were automatically opened by the local + /// QUIC connection and were added to a queue for availability. + /// + /// If `None` is returned, then a [`ConnectionEvent::StreamOpened`] event will later be + /// produced when a substream is available. + pub(crate) fn pop_outgoing_substream(&mut self) -> Option { + self.connection.open(quinn_proto::Dir::Bi) + } + + /// Reads data from the given substream. Similar to the API of `std::io::Read`. + /// + /// If `Err(ReadError::Blocked)` is returned, then a [`ConnectionEvent::StreamReadable`] event + /// will later be produced when the substream has readable data. A + /// [`ConnectionEvent::StreamStopped`] event can also be emitted. + pub(crate) fn read_substream( + &mut self, + id: quinn_proto::StreamId, + buf: &mut [u8], + ) -> Result { + self.connection.read(id, buf).map(|n| { + // `n` is `None` in case of EOF. + // See https://github.com/quinn-rs/quinn/blob/9aa3bde3aa1319b2c743f792312508de9270b8c6/quinn/src/streams.rs#L367-L370 + debug_assert_ne!(n, Some(0)); // Sanity check + n.unwrap_or(0) + }) + } + + /// Writes data to the given substream. Similar to the API of `std::io::Write`. + /// + /// If `Err(WriteError::Blocked)` is returned, then a [`ConnectionEvent::StreamWritable`] event + /// will later be produced when the substream can be written to. A + /// [`ConnectionEvent::StreamStopped`] event can also be emitted. + pub(crate) fn write_substream( + &mut self, + id: quinn_proto::StreamId, + buf: &[u8], + ) -> Result { + self.connection.write(id, buf) + } + + /// Closes the given substream. + /// + /// [`Connection::write_substream`] must no longer be called. The substream is however still + /// readable. + /// + /// On success, a [`ConnectionEvent::StreamFinished`] event will later be produced when the + /// substream has been effectively closed. A [`ConnectionEvent::StreamStopped`] event can also + /// be emitted. + pub(crate) fn shutdown_substream( + &mut self, + id: quinn_proto::StreamId, + ) -> Result<(), quinn_proto::FinishError> { + self.connection.finish(id) + } + + /// Polls the connection for an event that happend on it. + pub(crate) fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { + // Nothing more can be done if the connection is closed. + // Return `Pending` without registering the waker, essentially freezing the task forever. + if self.closed.is_some() { + return Poll::Pending; + } + + // Process events that the endpoint has sent to us. + loop { + match Pin::new(&mut self.from_endpoint).poll_next(cx) { + Poll::Ready(Some(event)) => self.connection.handle_event(event), + Poll::Ready(None) => { + debug_assert!(self.closed.is_none()); + let err = Error::ClosedChannel; + self.closed = Some(err.clone()); + return Poll::Ready(ConnectionEvent::ConnectionLost(err)); + } + Poll::Pending => break, + } + } + + 'send_pending: loop { + // Sending the pending event to the endpoint. If the endpoint is too busy, we just + // stop the processing here. + // There is a bit of a question in play here: should we continue to accept events + // through `from_endpoint` if `to_endpoint` is busy? + // We need to be careful to avoid a potential deadlock if both `from_endpoint` and + // `to_endpoint` are full. As such, we continue to transfer data from `from_endpoint` + // to the `quinn_proto::Connection` (see above). + // However we don't deliver substream-related events to the user as long as + // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` + // being full to the user. + if let Some(pending_to_endpoint) = &mut self.pending_to_endpoint { + match Future::poll(Pin::new(pending_to_endpoint), cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(()) => self.pending_to_endpoint = None, + } + } + + let now = Instant::now(); + + // Poll the connection for packets to send on the UDP socket and try to send them on + // `to_endpoint`. + while let Some(transmit) = self.connection.poll_transmit(now) { + let endpoint = self.endpoint.clone(); + debug_assert!(self.pending_to_endpoint.is_none()); + self.pending_to_endpoint = Some(Box::pin(async move { + // TODO: ECN bits not handled + endpoint + .send_udp_packet(transmit.destination, transmit.contents) + .await; + })); + continue 'send_pending; + } + + // The connection also needs to be able to send control messages to the endpoint. This is + // handled here, and we try to send them on `to_endpoint` as well. + while let Some(endpoint_event) = self.connection.poll_endpoint_events() { + let endpoint = self.endpoint.clone(); + let connection_id = self.connection_id; + debug_assert!(self.pending_to_endpoint.is_none()); + self.pending_to_endpoint = Some(Box::pin(async move { + endpoint + .report_quinn_event(connection_id, endpoint_event) + .await; + })); + continue 'send_pending; + } + + // Timeout system. + // We break out of the following loop until if `poll_timeout()` returns `None` or if + // polling `self.next_timeout` returns `Poll::Pending`. + loop { + if let Some(next_timeout) = &mut self.next_timeout { + match Future::poll(Pin::new(next_timeout), cx) { + Poll::Ready(()) => { + self.connection.handle_timeout(now); + self.next_timeout = None; + } + Poll::Pending => break, + } + } else if let Some(when) = self.connection.poll_timeout() { + if when <= now { + self.connection.handle_timeout(now); + } else { + let delay = when - now; + self.next_timeout = Some(futures_timer::Delay::new(delay)); + } + } else { + break; + } + } + + // The final step consists in handling the events related to the various substreams. + while let Some(event) = self.connection.poll() { + match event { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Uni, + }) + | quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Uni, + }) + | quinn_proto::Event::DatagramReceived => { + // We don't use datagrams or unidirectional streams. If these events + // happen, it is by some code not compatible with libp2p-quic. + self.connection + .close(Instant::now(), From::from(0u32), Default::default()); + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { + return Poll::Ready(ConnectionEvent::StreamReadable(id)); + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Writable { id }) => { + return Poll::Ready(ConnectionEvent::StreamWritable(id)); + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { + id, .. + }) => { + // The `Stop` QUIC event is more or less similar to a `Reset`, except that + // it applies only on the writing side of the pipe. + return Poll::Ready(ConnectionEvent::StreamStopped(id)); + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Bi, + }) => { + return Poll::Ready(ConnectionEvent::StreamAvailable); + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Bi, + }) => { + return Poll::Ready(ConnectionEvent::StreamOpened); + } + quinn_proto::Event::ConnectionLost { reason } => { + debug_assert!(self.closed.is_none()); + self.is_handshaking = false; + let err = Error::Quinn(reason); + self.closed = Some(err.clone()); + return Poll::Ready(ConnectionEvent::ConnectionLost(err)); + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { + id, + }) => { + return Poll::Ready(ConnectionEvent::StreamFinished(id)); + } + quinn_proto::Event::Connected => { + debug_assert!(self.is_handshaking); + debug_assert!(!self.connection.is_handshaking()); + self.is_handshaking = false; + return Poll::Ready(ConnectionEvent::Connected); + } + quinn_proto::Event::HandshakeDataReady => { + debug_assert!(self.is_handshaking); + debug_assert!(self.connection.is_handshaking()); + } + } + } + + break; + } + + Poll::Pending + } +} + +impl fmt::Debug for Connection { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_tuple("Connection").finish() + } +} + +impl Drop for Connection { + fn drop(&mut self) { + // TODO: don't do that if already drained + // We send a message to the endpoint. + self.endpoint.report_quinn_event_non_block(self.connection_id, quinn_proto::EndpointEvent::drained()); + } +} + +/// Event generated by the [`Connection`]. +#[derive(Debug)] +pub(crate) enum ConnectionEvent { + /// Now connected to the remote and certificates are available. + Connected, + + /// Connection has been closed and can no longer be used. + ConnectionLost(Error), + + /// Generated after [`Connection::pop_incoming_substream`] has been called and has returned + /// `None`. After this event has been generated, this method is guaranteed to return `Some`. + StreamAvailable, + /// Generated after [`Connection::pop_outgoing_substream`] has been called and has returned + /// `None`. After this event has been generated, this method is guaranteed to return `Some`. + StreamOpened, + + /// Generated after [`Connection::read_substream`] has been called and has returned a + /// `Blocked` error. + StreamReadable(quinn_proto::StreamId), + /// Generated after [`Connection::write_substream`] has been called and has returned a + /// `Blocked` error. + StreamWritable(quinn_proto::StreamId), + + /// Generated after [`Connection::shutdown_substream`] has been called. + StreamFinished(quinn_proto::StreamId), + /// A substream has been stopped. This concept is similar to the concept of a substream being + /// "reset", as in a TCP socket being reset for example. + StreamStopped(quinn_proto::StreamId), +} diff --git a/transports/quic/src/crypto.rs b/transports/quic/src/crypto.rs deleted file mode 100644 index de4e482373a..00000000000 --- a/transports/quic/src/crypto.rs +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2021 David Craven. -// -// 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::Keypair; -use quinn_proto::TransportConfig; -use std::sync::Arc; - -pub struct CryptoConfig { - pub keypair: Keypair, - pub keylogger: Option>, - pub transport: Arc, -} - -#[derive(Clone, Copy, Debug)] -pub(crate) struct TlsCrypto; - -impl TlsCrypto { - pub fn new_server_config(config: &CryptoConfig) -> Arc { - 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; - } - Arc::new(server) - } - - pub fn new_client_config(config: &CryptoConfig) -> Arc { - let mut client = crate::tls::make_client_config(&config.keypair).expect("invalid config"); - if let Some(key_log) = config.keylogger.clone() { - client.key_log = key_log; - } - Arc::new(client) - } - - pub fn keylogger() -> Arc { - Arc::new(rustls::KeyLogFile::new()) - } -} diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 50e01c5ae38..1ac98f1efee 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -1,4 +1,4 @@ -// Copyright 2021 David Craven. +// Copyright 2017-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"), @@ -18,439 +18,595 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::crypto::{CryptoConfig, TlsCrypto}; -use crate::muxer::QuicMuxer; -use crate::{QuicConfig, QuicError}; -use futures::channel::{mpsc, oneshot}; -use futures::prelude::*; -use quinn_proto::{ - ClientConfig as QuinnClientConfig, ConnectionEvent, ConnectionHandle, DatagramEvent, - EcnCodepoint, Endpoint as QuinnEndpoint, EndpointEvent, ServerConfig as QuinnServerConfig, - Transmit, +//! Background task dedicated to manage the QUIC state machine. +//! +//! Considering that all QUIC communications happen over a single UDP socket, one needs to +//! maintain a unique synchronization point that holds the state of all the active connections. +//! +//! The [`Endpoint`] object represents this synchronization point. It maintains a background task +//! whose role is to interface with the UDP socket. Communication between the background task and +//! the rest of the code only happens through channels. See the documentation of the +//! [`background_task`] for a thorough description. + +use crate::{connection::Connection, x509}; + +use async_std::net::SocketAddr; +use futures::{ + channel::{mpsc, oneshot}, + lock::Mutex, + prelude::*, +}; +use libp2p_core::multiaddr::Multiaddr; +use std::{ + collections::{HashMap, VecDeque}, + fmt, io, + sync::{Arc, Weak}, + task::Poll, + time::{Duration, Instant}, }; -use std::collections::{HashMap, 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, - /// 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::Receiver>, - port: u16, - ty: SocketType, -} - -impl TransportChannel { - pub fn dial(&mut self, addr: SocketAddr) -> oneshot::Receiver> { - let (tx, rx) = oneshot::channel(); - let msg = ToEndpoint::Dial { addr, 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::Receiver, - port: u16, - max_datagrams: usize, +/// Represents the configuration for the [`Endpoint`]. +#[derive(Debug, Clone)] +pub struct Config { + /// The client configuration to pass to `quinn_proto`. + client_config: quinn_proto::ClientConfig, + /// The server configuration to pass to `quinn_proto`. + server_config: Arc, + /// The endpoint configuration to pass to `quinn_proto`. + endpoint_config: Arc, + /// The [`Multiaddr`] to use to spawn the UDP socket. + multiaddr: Multiaddr, } -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 +impl Config { + /// Creates a new configuration object with default values. + pub fn new( + keypair: &libp2p_core::identity::Keypair, + multiaddr: Multiaddr, + ) -> Result { + let mut transport = quinn_proto::TransportConfig::default(); + transport.stream_window_uni(0).unwrap(); // Can only panic if value is out of range. + transport.datagram_receive_buffer_size(None); + transport.keep_alive_interval(Some(Duration::from_millis(10))); + let transport = Arc::new(transport); + let (client_tls_config, server_tls_config) = x509::make_tls_config(keypair)?; + let mut server_config = quinn_proto::ServerConfig::default(); + server_config.transport = transport.clone(); + server_config.crypto = Arc::new(server_tls_config); + let mut client_config = quinn_proto::ClientConfig::default(); + client_config.transport = transport; + client_config.crypto = Arc::new(client_tls_config); + Ok(Self { + client_config, + server_config: Arc::new(server_config), + endpoint_config: Default::default(), + multiaddr: multiaddr, + }) } } -#[derive(Debug)] -struct EndpointChannel { - rx: mpsc::UnboundedReceiver, - tx: mpsc::Sender>, - port: u16, - max_datagrams: usize, - connection_tx: mpsc::UnboundedSender, +/// Object containing all the QUIC resources shared between all connections. +// TODO: expand docs +// TODO: Debug trait +// TODO: remove useless fields +pub struct Endpoint { + /// Channel to the background of the endpoint. + /// See [`Endpoint::new_connections`] (just below) for a commentary about the mutex. + to_endpoint: Mutex>, + + /// Channel where new connections are being sent. + /// This is protected by a futures-friendly `Mutex`, meaning that receiving a connection is + /// done in two steps: locking this mutex, and grabbing the next element on the `Receiver`. + /// The only consequence of this `Mutex` is that multiple simultaneous calls to + /// [`Endpoint::next_incoming`] are serialized. + new_connections: Mutex>, + + /// Copy of [`Endpoint::to_endpoint`], except not behind a `Mutex`. Used if we want to be + /// guaranteed a slot in the messages buffer. + to_endpoint2: mpsc::Sender, + + /// Configuration passed at initialization. + // TODO: remove? + config: Config, + /// Multiaddr of the local UDP socket passed in the configuration at initialization after it + /// has potentially been modified to handle port number `0`. + local_multiaddr: Multiaddr, } -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::Sender) { - let (tx, rx) = mpsc::channel(12); - let channel = ConnectionChannel { - id, - tx: self.connection_tx.clone(), - rx, - port: self.port, - max_datagrams: self.max_datagrams, +impl Endpoint { + /// Builds a new `Endpoint`. + pub fn new(config: Config) -> Result, io::Error> { + let local_socket_addr = match crate::transport::multiaddr_to_socketaddr(&config.multiaddr) { + Ok(a) => a, + Err(()) => panic!(), // TODO: Err(TransportError::MultiaddrNotSupported(multiaddr)), }; - (channel, tx) - } -} - -pub struct EndpointConfig { - socket: UdpSocket, - endpoint: QuinnEndpoint, - port: u16, - crypto_config: Arc, - capabilities: UdpCapabilities, -} - -impl EndpointConfig { - pub fn new(mut config: QuicConfig, addr: SocketAddr) -> Result { - config.transport.max_concurrent_uni_streams(0u32.into()); - config.transport.datagram_receive_buffer_size(None); - let transport = Arc::new(config.transport); - let crypto_config = Arc::new(CryptoConfig { - keypair: config.keypair, - keylogger: config.keylogger, - transport: transport.clone(), + // NOT blocking, as per man:bind(2), as we pass an IP address. + let socket = std::net::UdpSocket::bind(&local_socket_addr)?; + // TODO: + /*let port_is_zero = local_socket_addr.port() == 0; + let local_socket_addr = socket.local_addr()?; + if port_is_zero { + assert_ne!(local_socket_addr.port(), 0); + assert_eq!(multiaddr.pop(), Some(Protocol::Quic)); + assert_eq!(multiaddr.pop(), Some(Protocol::Udp(0))); + multiaddr.push(Protocol::Udp(local_socket_addr.port())); + multiaddr.push(Protocol::Quic); + }*/ + + let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(32); + let to_endpoint2 = to_endpoint_tx.clone(); + let (new_connections_tx, new_connections_rx) = mpsc::channel(1); + + let endpoint = Arc::new(Endpoint { + to_endpoint: Mutex::new(to_endpoint_tx), + to_endpoint2, + new_connections: Mutex::new(new_connections_rx), + config: config.clone(), + local_multiaddr: config.multiaddr.clone(), // TODO: no }); - let crypto = TlsCrypto::new_server_config(&crypto_config); - let mut server_config = QuinnServerConfig::with_crypto(crypto); - server_config.transport = transport; + // TODO: just for testing, do proper task spawning + async_std::task::spawn(background_task( + config.clone(), + Arc::downgrade(&endpoint), + async_std::net::UdpSocket::from(socket), + new_connections_tx, + to_endpoint_rx.fuse(), + )); + + Ok(endpoint) + + // TODO: IP address stuff + /*if socket_addr.ip().is_unspecified() { + info!("returning all local IPs for unspecified address"); + let suffixes = [Protocol::Udp(socket_addr.port()), Protocol::Quic]; + let local_addresses = + host_addresses(&suffixes).map_err(|e| TransportError::Other(Error::IO(e)))?; + for (_, _, address) in local_addresses { + info!("sending address {:?}", address); + new_connections + .unbounded_send(ListenerEvent::NewAddress(address)) + .expect("we have a reference to the peer, so this will not fail; qed") + } + } else { + info!("sending address {:?}", multiaddr); + new_connections + .unbounded_send(ListenerEvent::NewAddress(multiaddr.clone())) + .expect("we have a reference to the peer, so this will not fail; qed"); + } - let endpoint = QuinnEndpoint::new(Default::default(), Some(Arc::new(server_config))); + if socket_addr.ip().is_unspecified() { + debug!("returning all local IPs for unspecified address"); + let local_addresses = + host_addresses(&[Protocol::Udp(socket_addr.port()), Protocol::Quic]) + .map_err(|e| TransportError::Other(Error::IO(e)))?; + for i in local_addresses { + info!("sending address {:?}", i.2); + reference + .new_connections + .unbounded_send(ListenerEvent::NewAddress(i.2)) + .expect("we have a reference to the peer, so this will not fail; qed") + } + } else { + info!("sending address {:?}", multiaddr); + reference + .new_connections + .unbounded_send(ListenerEvent::NewAddress(multiaddr)) + .expect("we have a reference to the peer, so this will not fail; qed"); + } - let socket = UdpSocket::bind(addr)?; - let port = socket.local_addr()?.port(); - let capabilities = UdpSocket::capabilities()?; - Ok(Self { - socket, - endpoint, - port, - crypto_config, - capabilities, - }) + let endpoint = EndpointRef { reference, channel }; + let join_handle = spawn(endpoint.clone()); + Ok((Self(endpoint), join_handle))*/ } - pub fn spawn(self) -> TransportChannel { - let (tx1, rx1) = mpsc::unbounded(); - let (tx2, rx2) = mpsc::channel(1); - 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 + /// Asks the endpoint to start dialing the given address. + /// + /// Note that this method only *starts* the dialing. `Ok` is returned as soon as possible, even + /// when the remote might end up being unreachable. + pub(crate) async fn dial( + &self, + addr: SocketAddr, + ) -> Result { + // The two `expect`s below can panic if the background task has stopped. The background + // task can stop only if the `Endpoint` is destroyed or if the task itself panics. In other + // words, we panic here iff a panic has already happened somewhere else, which is a + // reasonable thing to do. + let (tx, rx) = oneshot::channel(); + self.to_endpoint + .lock().await + .send(ToEndpoint::Dial { addr, result: tx }) + .await + .expect("background task has crashed"); + rx.await.expect("background task has crashed") } -} - -struct Endpoint { - channel: EndpointChannel, - endpoint: QuinnEndpoint, - socket: UdpSocket, - crypto_config: Arc, - connections: HashMap>, - outgoing: VecDeque, - recv_buf: Box<[u8]>, - incoming_slot: Option, - event_slot: Option<(ConnectionHandle, ConnectionEvent)>, -} -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, - crypto_config: config.crypto_config, - connections: Default::default(), - outgoing: Default::default(), - recv_buf, - incoming_slot: None, - event_slot: None, - } + /// Tries to pop a new incoming connection from the queue. + pub(crate) async fn next_incoming(&self) -> Connection { + // The `expect` below can panic if the background task has stopped. The background task + // can stop only if the `Endpoint` is destroyed or if the task itself panics. In other + // words, we panic here iff a panic has already happened somewhere else, which is a + // reasonable thing to do. + let mut new_connections = self.new_connections.lock().await; + new_connections + .next() + .await + .expect("background task has crashed") } - 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); + /// Asks the endpoint to send a UDP packet. + /// + /// Note that this method only queues the packet and returns as soon as the packet is in queue. + /// There is no guarantee that the packet will actually be sent, but considering that this is + /// a UDP packet, you cannot rely on the packet being delivered anyway. + pub(crate) async fn send_udp_packet( + &self, + destination: SocketAddr, + data: impl Into>, + ) { + let _ = self + .to_endpoint + .lock().await + .send(ToEndpoint::SendUdpPacket { + destination, + data: data.into(), + }) + .await; } - 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, - } + /// Report to the endpoint an event on a [`quinn_proto::Connection`]. + /// + /// This is typically called by a [`Connection`]. + /// + /// If `event.is_drained()` is true, the event indicates that the connection no longer exists. + /// This must therefore be the last event sent using this [`quinn_proto::ConnectionHandle`]. + pub(crate) async fn report_quinn_event( + &self, + connection_id: quinn_proto::ConnectionHandle, + event: quinn_proto::EndpointEvent, + ) { + self.to_endpoint + .lock().await + .send(ToEndpoint::ProcessConnectionEvent { + connection_id, + event, + }) + .await + .expect("background task has crashed"); } - 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, - } + /// Similar to [`Endpoint::report_quinn_event`], except that the message sending is guaranteed + /// to be instantaneous and to succeed. + /// + /// This method bypasses back-pressure mechanisms and is meant to be called only from + /// destructors, where waiting is not advisable. + pub(crate) fn report_quinn_event_non_block( + &self, + connection_id: quinn_proto::ConnectionHandle, + event: quinn_proto::EndpointEvent, + ) { + // We implement this by cloning the `mpsc::Sender`. Since each sender is guaranteed a slot + // in the buffer, cloning the sender reserves the slot and sending thus always succeeds. + let result = self.to_endpoint2 + .clone() + .try_send(ToEndpoint::ProcessConnectionEvent { + connection_id, + event, + }); + assert!(result.is_ok()); } } -impl Future for Endpoint { - 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"); - } - } +/// Message sent to the endpoint background task. +#[derive(Debug)] +enum ToEndpoint { + /// Instruct the endpoint to start connecting to the given address. + Dial { + /// UDP address to connect to. + addr: SocketAddr, + /// Channel to return the result of the dialing to. + result: 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. Only `quinn_proto` knows what is in + /// there. + ProcessConnectionEvent { + connection_id: quinn_proto::ConnectionHandle, + event: quinn_proto::EndpointEvent, + }, + /// Instruct the endpoint to send a packet of data on its UDP socket. + SendUdpPacket { + /// Destination of the UDP packet. + destination: SocketAddr, + /// Packet of data to send. + data: Vec, + }, +} - if let Some((id, event)) = me.event_slot.take() { - if !me.send_event(id, event, cx) { - tracing::info!("cleared event slot"); +/// Task that runs in the background for as long as the endpont is alive. Responsible for +/// processing messages and the UDP socket. +/// +/// The `receiver` parameter must be the receiving side of the `Endpoint::to_endpoint` sender. +/// +/// # Behaviour +/// +/// This background task is responsible for the following: +/// +/// - Sending packets on the UDP socket. +/// - Receiving packets from the UDP socket and feed them to the [`quinn_proto::Endpoint`] state +/// machine. +/// - Transmitting events generated by the [`quinn_proto::Endpoint`] to the corresponding +/// [`Connection`]. +/// - Receiving messages from the `receiver` and processing the requested actions. This includes +/// UDP packets to send and events emitted by the [`Connection`] objects. +/// - Sending new connections on `new_connections`. +/// +/// When it comes to channels, there exists three main multi-producer-single-consumer channels +/// in play: +/// +/// - One channel, represented by `Endpoint::to_endpoint` and `receiver`, that communicates +/// messages from [`Endpoint`] to the background task and from the [`Connection`] to the +/// background task. +/// - One channel per each existing connection that communicates messages from the background +/// task to that [`Connection`]. +/// - One channel for the background task to send newly-opened connections to. The receiving +/// side is normally processed by a "listener" as defined by the [`libp2p_core::Transport`] +/// trait. +/// +/// In order to avoid an unbounded buffering of events, we prioritize sending data on the UDP +/// socket over everything else. If the network interface is too busy to process our packets, +/// everything comes to a freeze (including receiving UDP packets) until it is ready to accept +/// more. +/// +/// Apart from freezing when the network interface is too busy, the background task should sleep +/// as little as possible. It is in particular important for the `receiver` to be drained as +/// quickly as possible in order to avoid unnecessary back-pressure on the [`Connection`] objects. +/// +/// ## Back-pressure on `new_connections` +/// +/// The [`quinn_proto::Endpoint`] object contains an accept buffer, in other words a buffer of the +/// incoming connections waiting to be accepted. When a new connection is signalled, we send this +/// new connection on the `new_connections` channel in an asynchronous way, and we only free a slot +/// in the accept buffer once the element has actually been enqueued on `new_connections`. There +/// are therefore in total three buffers in play: the `new_connections` channel itself, the queue +/// of elements being sent on `new_connections`, and the accept buffer of the +/// [`quinn_proto::Endpoint`]. +/// +/// Unfortunately, this design has the consequence that, on the network layer, we will accept a +/// certain number of incoming connections even if [`Endpoint::next_incoming`] is never even +/// called. The `quinn-proto` library doesn't provide any way to not accept incoming connections +/// apart from filling the accept buffer. +/// +/// ## Back-pressure on connections +/// +/// Because connections are processed by the user at a rate of their choice, we cannot properly +/// handle the situation where the channel from the background task to individual connections is +/// full. Sleeping the task while waiting for the connection to be processed by the user could +/// even lead to a deadlock if this processing is also sleeping waiting for some other action that +/// itself depends on the background task (e.g. if processing the connection is waiting for a +/// message arriving on a different connection). +/// +/// In an ideal world, we would handle a background-task-to-connection channel being full by +/// dropping UDP packets destined to this connection, as a way to back-pressure the remote. +/// Unfortunately, the `quinn-proto` library doesn't provide any way for us to know which +/// connection a UDP packet is destined for before it has been turned into a [`ConnectionEvent`], +/// and because these [`ConnectionEvent`]s are sometimes used to synchronize the states of the +/// endpoint and connection, it would be a logic error to silently drop them. +/// +/// We handle this tricky situation by simply killing connections as soon as their associated +/// channel is full. +/// +// TODO: actually implement the killing of connections if channel is full, at the moment we just +// wait +/// # Shutdown +/// +/// The background task shuts down if `endpoint_weak`, `receiver` or `new_connections` become +/// disconnected/invalid. This corresponds to the lifetime of the associated [`Endpoint`]. +/// +/// Keep in mind that we pass an `Arc` whenever we create a new connection, which +/// guarantees that the [`Endpoint`], and therefore the background task, is properly kept alive +/// for as long as any QUIC connection is open. +/// +async fn background_task( + config: Config, + endpoint_weak: Weak, + udp_socket: async_std::net::UdpSocket, + mut new_connections: mpsc::Sender, + mut receiver: stream::Fuse>, +) { + // The actual QUIC state machine. + let mut endpoint = quinn_proto::Endpoint::new( + config.endpoint_config.clone(), + Some(config.server_config.clone()), + ); + + // List of all active connections, with a sender to notify them of events. + let mut alive_connections = HashMap::>::new(); + + // Buffer where we write packets received from the UDP socket. + let mut socket_recv_buffer = vec![0; 65536]; + + // The quinn_proto endpoint can give us new connections for as long as its accept buffer + // isn't full. This buffer is used to push these new connections while we are waiting to + // send them on the `new_connections` channel. We only call `endpoint.accept()` when we remove + // an element from this list, which guarantees that it doesn't grow unbounded. + // TODO: with_capacity? + let mut queued_new_connections = VecDeque::new(); + + // Next packet waiting to be transmitted on the UDP socket, if any. + // Note that this variable isn't strictly necessary, but it reduces code duplication in the + // code below. + let mut next_packet_out: Option<(SocketAddr, Vec)> = None; + + // Main loop of the task. + loop { + // Start by flushing `next_packet_out`. + if let Some((destination, data)) = next_packet_out.take() { + // We block the current task until the packet is sent. This way, if the + // network interface is too busy, we back-pressure all of our internal + // channels. + // TODO: set ECN bits; there is no support for them in the ecosystem right now + match udp_socket.send_to(&data, destination).await { + Ok(n) if n == data.len() => {} + Ok(_) => log::error!( + "QUIC UDP socket violated expectation that packets are always fully \ + transferred" + ), + + // Errors on the socket are expected to never happen, and we handle them by simply + // printing a log message. The packet gets discarded in case of error, but we are + // robust to packet losses and it is consequently not a logic error to process with + // normal operations. + Err(err) => log::error!("Error while sending on QUIC UDP socket: {:?}", err), } } - while let Some(transmit) = me.endpoint.poll_transmit() { - me.transmit(transmit); + // The endpoint might request packets to be sent out. This is handled in priority to avoid + // buffering up packets. + if let Some(packet) = endpoint.poll_transmit() { + debug_assert!(next_packet_out.is_none()); + next_packet_out = Some((packet.destination, packet.contents)); + continue; } - if me.event_slot.is_none() { - while let Poll::Ready(event) = me.channel.poll_next_event(cx) { - match event { - Some(ToEndpoint::Dial { addr, tx }) => { - let crypto = TlsCrypto::new_client_config(&me.crypto_config); - let mut client_config = QuinnClientConfig::new(crypto); - client_config.transport = me.crypto_config.transport.clone(); - - let (id, connection) = - match me.endpoint.connect(client_config, addr, "server_name") { + futures::select! { + message = receiver.next() => { + // Received a message from a different part of the code requesting us to + // do something. + match message { + // Shut down if the endpoint has shut down. + None => return, + + Some(ToEndpoint::Dial { addr, result }) => { + // This `"l"` seems necessary because an empty string is an invalid domain + // name. While we don't use domain names, the underlying rustls library + // is based upon the assumption that we do. + let (connection_id, connection) = + match endpoint.connect(config.client_config.clone(), addr, "l") { Ok(c) => c, Err(err) => { - tracing::error!("dial failure: {}", err); - let _ = tx.send(Err(err.into())); + let _ = result.send(Err(err)); 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(); + + let endpoint_arc = match endpoint_weak.upgrade() { + Some(ep) => ep, + None => return, // Shut down the task if the endpoint is dead. + }; + + debug_assert_eq!(connection.side(), quinn_proto::Side::Client); + let (tx, rx) = mpsc::channel(16); + let connection = Connection::from_quinn_connection(endpoint_arc, connection, connection_id, rx); + alive_connections.insert(connection_id, tx); + let _ = result.send(Ok(connection)); } - Some(ToEndpoint::ConnectionEvent { - connection_id, - event, - }) => { + + // A connection wants to notify the endpoint of something. + Some(ToEndpoint::ProcessConnectionEvent { connection_id, event }) => { + debug_assert!(alive_connections.contains_key(&connection_id)); + // We "drained" event indicates that the connection no longer exists and + // its ID can be reclaimed. let is_drained_event = event.is_drained(); if is_drained_event { - me.connections.remove(&connection_id); + alive_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; - } + if let Some(event_back) = endpoint.handle_event(connection_id, event) { + debug_assert!(!is_drained_event); + // TODO: don't await here /!\ + alive_connections.get_mut(&connection_id).unwrap().send(event_back).await; } } - Some(ToEndpoint::Transmit(transmit)) => { - me.transmit(transmit); - } - None => { - me.endpoint.reject_new_connections(); - return Poll::Ready(()); + + // Data needs to be sent on the UDP socket. + Some(ToEndpoint::SendUdpPacket { destination, data }) => { + debug_assert!(next_packet_out.is_none()); + next_packet_out = Some((destination, data)); + continue; } } } - } - while !me.outgoing.is_empty() { - let transmits: &[_] = me.outgoing.make_contiguous(); - match me.socket.poll_send(cx, transmits) { - Poll::Ready(Ok(n)) => { - me.outgoing.drain(..n); + // The future we create here wakes up if two conditions are fulfilled: + // + // - The `new_connections` channel is ready to accept a new element. + // - `queued_new_connections` is not empty. + // + // When this happens, we pop an element from `queued_new_connections`, put it on the + // channel, and call `endpoint.accept()`, thereby allowing the QUIC state machine to + // feed a new incoming connection to us. + readiness = { + let active = !queued_new_connections.is_empty(); + let new_connections = &mut new_connections; + future::poll_fn(move |cx| { + if active { new_connections.poll_ready(cx) } else { Poll::Pending } + }).fuse() + } => { + if readiness.is_err() { + // new_connections channel has been dropped, meaning that the endpoint has + // been destroyed. + return; } - Poll::Ready(Err(err)) => tracing::error!("send_to: {}", err), - Poll::Pending => break, - } - } - 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(); - fn init_iovs<'a>( - iovs: &'a mut MaybeUninit<[IoSliceMut<'a>; BATCH_SIZE]>, - 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); - // 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)); - }); - - unsafe { - // SAFETY: all elements are initialized - iovs.assume_init_mut() - } + let elem = queued_new_connections.pop_front() + .expect("if queue is empty, the future above is always Pending; qed"); + new_connections.start_send(elem) + .expect("future is waken up only if poll_ready returned Ready; qed"); + endpoint.accept(); } - 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, + + result = udp_socket.recv_from(&mut socket_recv_buffer).fuse() => { + let (packet_len, packet_src) = match result { + Ok(v) => v, + // Errors on the socket are expected to never happen, and we handle them by + // simply printing a log message. Err(err) => { - tracing::error!("recv_from: {}", err); + log::error!("Error while receive on QUIC UDP socket: {:?}", 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))) => { - 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; - } + + // Received a UDP packet from the socket. + debug_assert!(packet_len <= socket_recv_buffer.len()); + let packet = From::from(&socket_recv_buffer[..packet_len]); + // TODO: ECN bits aren't handled + match endpoint.handle(Instant::now(), packet_src, None, packet) { + None => {}, + Some((connec_id, quinn_proto::DatagramEvent::ConnectionEvent(event))) => { + // Event to send to an existing connection. + if let Some(sender) = alive_connections.get_mut(&connec_id) { + let _ = sender.send(event).await; // TODO: don't await here /!\ + } else { + log::error!("State mismatch: event for closed connection"); } - } + }, + Some((connec_id, quinn_proto::DatagramEvent::NewConnection(connec))) => { + // A new connection has been received. `connec_id` is a newly-allocated + // identifier. + debug_assert_eq!(connec.side(), quinn_proto::Side::Server); + let (tx, rx) = mpsc::channel(16); + alive_connections.insert(connec_id, tx); + let endpoint_arc = match endpoint_weak.upgrade() { + Some(ep) => ep, + None => return, // Shut down the task if the endpoint is dead. + }; + let connection = Connection::from_quinn_connection(endpoint_arc, connec, connec_id, rx); + + // As explained in the documentation, we put this new connection in an + // intermediary buffer. At the next loop iteration we will try to move it + // to the `new_connections` channel. We call `endpoint.accept()` only once + // the element has successfully been sent on `new_connections`. + queued_new_connections.push_back(connection); + }, } } - me.recv_buf = recv_buf; } + } +} - Poll::Pending +impl fmt::Debug for Endpoint { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_tuple("Endpoint").finish() } } diff --git a/transports/quic/src/error.rs b/transports/quic/src/error.rs new file mode 100644 index 00000000000..9671950d4e3 --- /dev/null +++ b/transports/quic/src/error.rs @@ -0,0 +1,90 @@ +// 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 futures::channel::mpsc::SendError; +use io::ErrorKind; +use std::io; +use thiserror::Error; + +/// An error that can be returned by libp2p-quic. +#[derive(Error, Debug)] +pub enum Error { + /// Fatal I/O error + #[error("Fatal I/O error {0}")] + IO(#[from] std::io::Error), + /// QUIC protocol error + #[error("QUIC protocol error: {0}")] + ConnectionError(#[from] quinn_proto::ConnectionError), + /// Peer stopped receiving data + #[error("Peer stopped receiving data: code {0}")] + Stopped(quinn_proto::VarInt), + /// Connection was prematurely closed + #[error("Connection was prematurely closed")] + ConnectionLost, + /// Error making the connection. + #[error("Connection failure: {0}")] + ConnectError(#[from] quinn_proto::ConnectError), + /// Cannot listen on the same endpoint more than once + #[error("Cannot listen on the same endpoint more than once")] + AlreadyListening, + /// The stream was reset by the peer. + #[error("Peer reset stream: code {0}")] + Reset(quinn_proto::VarInt), + /// Either an attempt was made to write to a stream that was already shut down, + /// or a previous operation on this stream failed. + #[error( + "Use of a stream that has is no longer valid. This is a \ + bug in the application." + )] + ExpiredStream, + /// Reading from a stream that has not been written to. + #[error("Reading from a stream that has not been written to.")] + CannotReadFromUnwrittenStream, + /// Fatal internal error or network failure + #[error("Fatal internal error or network failure")] + NetworkFailure, + /// Connection already being closed + #[error("Connection already being closed")] + ConnectionClosing, +} + +impl From for Error { + fn from(_: SendError) -> Error { + Error::NetworkFailure + } +} + +impl From for io::Error { + fn from(e: Error) -> Self { + match e { + Error::IO(e) => io::Error::new(e.kind(), Error::IO(e)), + Error::ConnectionError(e) => e.into(), + e @ Error::NetworkFailure + | e @ Error::ConnectionClosing + | e @ Error::ConnectError(_) => io::Error::new(ErrorKind::Other, e), + e @ Error::Stopped(_) | e @ Error::Reset(_) | e @ Error::ConnectionLost => { + io::Error::new(ErrorKind::ConnectionAborted, e) + } + e @ Error::ExpiredStream => io::Error::new(ErrorKind::BrokenPipe, e), + e @ Error::AlreadyListening => io::Error::new(ErrorKind::AddrInUse, e), + e @ Error::CannotReadFromUnwrittenStream => io::Error::new(ErrorKind::NotConnected, e), + } + } +} diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index d609f2eaaa6..97dc27ef996 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -1,4 +1,4 @@ -// Copyright 2021 David Craven. +// 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"), @@ -18,74 +18,52 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -mod crypto; -mod endpoint; -mod muxer; -mod tls; -mod transport; - -use crate::crypto::TlsCrypto; -pub use crate::muxer::{QuicMuxer, QuicMuxerError}; -pub use crate::transport::{QuicDial, QuicTransport}; -pub use quinn_proto::{ConfigError, ConnectError, ConnectionError, TransportConfig}; - -use libp2p_core::identity::Keypair; -use libp2p_core::transport::TransportError; -use libp2p_core::Multiaddr; -use std::sync::Arc; -use thiserror::Error; +#![recursion_limit = "1024"] -/// Quic configuration. -pub struct QuicConfig { - pub keypair: Keypair, - pub transport: TransportConfig, - pub keylogger: Option>, -} +//! Implementation of the libp2p `Transport` and `StreamMuxer` traits for QUIC. +//! +//! # Usage +//! +//! Example: +//! +//! ``` +//! use libp2p_quic::{Config, Endpoint}; +//! use libp2p_core::Multiaddr; +//! +//! let keypair = libp2p_core::identity::Keypair::generate_ed25519(); +//! let addr = "/ip4/127.0.0.1/udp/12345/quic".parse().expect("bad address?"); +//! let quic_config = Config::new(&keypair, addr).expect("could not make config"); +//! let quic_endpoint = Endpoint::new(quic_config).expect("I/O error"); +//! ``` +//! +//! The `Endpoint` struct implements the `Transport` trait of the `core` library. See the +//! documentation of `core` and of libp2p in general to learn how to use the `Transport` trait. +//! +//! Note that QUIC provides transport, security, and multiplexing in a single protocol. Therefore, +//! QUIC connections do not need to be upgraded. You will get a compile-time error if you try. +//! Instead, you must pass all needed configuration into the constructor. +//! +//! # Design Notes +//! +//! The entry point is the `Endpoint` struct. It represents a single QUIC endpoint. You +//! should generally have one of these per process. +//! +//! `Endpoint` manages a background task that processes all incoming packets. Each +//! `QuicConnection` also manages a background task, which handles socket output and timer polling. -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("transport", &self.transport) - .finish() - } -} +#![deny(unsafe_code)] -impl QuicConfig { - /// Creates a new config from a keypair. - pub fn new(keypair: Keypair) -> Self { - Self { - keypair, - transport: TransportConfig::default(), - keylogger: None, - } - } - - /// Enable keylogging. - pub fn enable_keylogger(&mut self) -> &mut Self { - self.keylogger = Some(TlsCrypto::keylogger()); - self - } +mod connection; +mod endpoint; +mod error; +mod muxer; +mod upgrade; +mod x509; - /// Spawns a new endpoint. - pub async fn listen_on( - self, - addr: Multiaddr, - ) -> Result> { - QuicTransport::new(self, addr).await - } -} +pub mod transport; -#[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 use endpoint::{Config, Endpoint}; +pub use error::Error; +pub use muxer::QuicMuxer; +pub use transport::QuicTransport; +pub use upgrade::Upgrade; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 45cd5f06c8c..f3477b9562b 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -1,4 +1,4 @@ -// Copyright 2021 David Craven. +// 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"), @@ -18,422 +18,326 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::endpoint::ConnectionChannel; -use async_io::Timer; -use futures::prelude::*; +use crate::connection::{Connection, ConnectionEvent}; +use crate::error::Error; + use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; -use libp2p_core::{Multiaddr, PeerId}; use parking_lot::Mutex; -use quinn_proto::Connection as QuinnConnection; -use quinn_proto::{ - ConnectionError, Dir, Event, FinishError, ReadError, ReadableError, StreamEvent, StreamId, - VarInt, WriteError, +use std::{ + collections::HashMap, + fmt, + task::{Context, Poll, Waker}, }; -use std::collections::{HashMap, 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. pub struct QuicMuxer { + // Note: This could theoretically be an asynchronous future, in order to yield the current + // task if a task running in parallel is already holding the lock. However, using asynchronous + // mutexes without async/await is extremely tedious and maybe not worth the effort. 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`]. +/// Mutex-protected fields of [`QuicMuxer`]. struct QuicMuxerInner { - /// Accept incoming streams. - accept_incoming: bool, - /// Endpoint channel. - endpoint: ConnectionChannel, /// Inner connection object that yields events. - connection: QuinnConnection, - /// Connection waker. - waker: Option, - /// Connection timer. - timer: Option, - /// State of all open substreams. - substreams: HashMap, - /// Pending substreams. - pending_substreams: VecDeque, - /// Close waker. - close_waker: Option, + connection: Connection, + /// State of all the substreams that the muxer reports as open. + substreams: HashMap, + /// Waker to wake if a new outgoing substream is opened. + poll_substream_opened_waker: Option, + /// Waker to wake if the connection is closed. + poll_close_waker: Option, } /// State of a single substream. -#[derive(Debug, Default)] +#[derive(Default)] struct SubstreamState { - /// Waker to wake if the substream becomes readable. + /// Waker to wake if the substream becomes readable or stopped. read_waker: Option, - /// Waker to wake if the substream becomes writable. + /// Waker to wake if the substream becomes writable or stopped. write_waker: Option, + /// True if the substream has been closed. + finished: bool, + /// Waker to wake if the substream becomes closed or stopped. + finished_waker: Option, } impl QuicMuxer { - pub fn new(endpoint: ConnectionChannel, connection: QuinnConnection) -> Self { - Self { + /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. + /// + /// # Panic + /// + /// Panics if `connection.is_handshaking()` returns `true`. + pub(crate) fn from_connection(connection: Connection) -> Self { + assert!(!connection.is_handshaking()); + + QuicMuxer { inner: Mutex::new(QuicMuxerInner { - accept_incoming: false, - endpoint, connection, - waker: None, - timer: None, substreams: Default::default(), - pending_substreams: Default::default(), - close_waker: None, + poll_substream_opened_waker: None, + poll_close_waker: None, }), } } - - pub fn is_handshaking(&self) -> bool { - self.inner.lock().connection.is_handshaking() - } - - pub fn peer_id(&self) -> Option { - let inner = self.inner.lock(); - let session = inner.connection.crypto_session(); - let identity = session.peer_identity()?; - let certs: Box> = identity.downcast().ok()?; - let cert = certs.get(0)?; - let p2p_cert = crate::tls::certificate::parse_certificate(&cert.0).ok()?; - Some(PeerId::from_public_key(&p2p_cert.extension.public_key)) - } - - 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) - } - - pub(crate) fn set_accept_incoming(&self, accept: bool) { - let mut inner = self.inner.lock(); - inner.accept_incoming = accept; - } } impl StreamMuxer for QuicMuxer { - type Substream = StreamId; type OutboundSubstream = (); - type Error = QuicMuxerError; + type Substream = quinn_proto::StreamId; + type Error = Error; - fn poll_event( - &self, - cx: &mut Context, - ) -> Poll, Self::Error>> { + // TODO: what if called multiple times? register all wakers? + fn poll_event(&self, cx: &mut Context<'_>) -> Poll, Self::Error>> { + // We use `poll_inbound` to perform the background processing of the entire connection. 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, max_datagrams) { - 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() { + while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { - Event::HandshakeDataReady => {} - Event::Connected => { - // Break here so that the noise upgrade can finish. - return Poll::Pending; + ConnectionEvent::Connected => { + log::error!("Unexpected Connected event on established QUIC connection"); } - Event::ConnectionLost { reason } => { - tracing::debug!("connection lost because of {}", reason); - inner.substreams.clear(); - if let Some(waker) = inner.close_waker.take() { + ConnectionEvent::ConnectionLost(_) => { + if let Some(waker) = inner.poll_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); - let substream = inner.substreams.get_mut(&id).expect("known substream; qed"); - if let Some(waker) = substream.read_waker.take() { - waker.wake(); - } - } - Event::Stream(StreamEvent::Writable { id }) => { - tracing::trace!("stream writable {}", id); - let substream = inner.substreams.get_mut(&id).expect("known substream; qed"); - if let Some(waker) = substream.write_waker.take() { + + ConnectionEvent::StreamOpened => { + if let Some(waker) = inner.poll_substream_opened_waker.take() { waker.wake(); } } - Event::Stream(StreamEvent::Finished { id }) => { - tracing::trace!("stream finished {}", id); - if let Some(substream) = inner.substreams.get_mut(&id) { + ConnectionEvent::StreamReadable(substream) => { + if let Some(substream) = inner.substreams.get_mut(&substream) { if let Some(waker) = substream.read_waker.take() { waker.wake(); } + } + } + ConnectionEvent::StreamWritable(substream) => { + if let Some(substream) = inner.substreams.get_mut(&substream) { 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!("substream available"); - if let Some(waker) = inner.pending_substreams.pop_front() { - waker.wake(); + ConnectionEvent::StreamFinished(substream) | + ConnectionEvent::StreamStopped(substream) => { + if let Some(substream) = inner.substreams.get_mut(&substream) { + if let ConnectionEvent::StreamFinished(_) = event { + substream.finished = true; + } + if let Some(waker) = substream.read_waker.take() { + waker.wake(); + } + if let Some(waker) = substream.write_waker.take() { + waker.wake(); + } + if let Some(waker) = substream.finished_waker.take() { + 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)); + ConnectionEvent::StreamAvailable => { + // Handled below. } } } - // TODO quinn doesn't support `StreamMuxerEvent::AddressChange`. - - 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() { - if let Some(waker) = inner.close_waker.take() { - waker.wake(); - } + if let Some(substream) = inner.connection.pop_incoming_substream() { + inner.substreams.insert(substream, Default::default()); + Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) + } else { + Poll::Pending } - - inner.waker = Some(cx.waker().clone()); - Poll::Pending } - fn open_outbound(&self) -> Self::OutboundSubstream {} + fn open_outbound(&self) -> Self::OutboundSubstream { + () + } + // TODO: what if called multiple times? register all wakers? fn poll_outbound( &self, - cx: &mut Context, + cx: &mut Context<'_>, _: &mut Self::OutboundSubstream, ) -> Poll> { + // Note: this implementation makes it possible to poll the same `Self::OutboundSubstream` + // over and over again and get new substreams. Using the API this way is invalid and would + // normally result in a panic, but we decide to just ignore this problem. + 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 + if let Some(substream) = inner.connection.pop_outgoing_substream() { + inner.substreams.insert(substream, Default::default()); + return Poll::Ready(Ok(substream)); + } + + // Register `cx.waker()` as having to be woken up once a substream is available. + if !inner + .poll_substream_opened_waker + .as_ref() + .map_or(false, |w| w.will_wake(cx.waker())) + { + inner.poll_substream_opened_waker = Some(cx.waker().clone()); } + + Poll::Pending } fn destroy_outbound(&self, _: Self::OutboundSubstream) {} - fn read_substream( + fn write_substream( &self, - cx: &mut Context, - id: &mut Self::Substream, - mut buf: &mut [u8], + cx: &mut Context<'_>, + substream: &mut Self::Substream, + buf: &[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; + + match inner.connection.write_substream(*substream, buf) { + Ok(bytes) => Poll::Ready(Ok(bytes)), + Err(quinn_proto::WriteError::Stopped(err_code)) => { + Poll::Ready(Err(Error::Reset(err_code))) + }, + Err(quinn_proto::WriteError::Blocked) => { + if let Some(substream) = inner.substreams.get_mut(substream) { + if !substream + .write_waker + .as_ref() + .map_or(false, |w| w.will_wake(cx.waker())) + { + substream.write_waker = Some(cx.waker().clone()); + } } + Poll::Pending } - } - if chunks.finalize().should_transmit() { - if let Some(waker) = inner.waker.take() { - waker.wake(); + Err(quinn_proto::WriteError::UnknownStream) => { + log::error!( + "The application used a connection that is already being \ + closed. This is a bug in the application or in libp2p." + ); + Poll::Pending } } - let substream = inner.substreams.get_mut(id).expect("known substream; qed"); - if pending && bytes == 0 { - substream.read_waker = Some(cx.waker().clone()); - Poll::Pending - } else { - Poll::Ready(Ok(bytes)) - } } - fn write_substream( + fn read_substream( &self, - cx: &mut Context, - id: &mut Self::Substream, - buf: &[u8], + cx: &mut Context<'_>, + substream: &mut Self::Substream, + buf: &mut [u8], ) -> Poll> { let mut inner = self.inner.lock(); - match inner.connection.send_stream(*id).write(buf) { + + match inner.connection.read_substream(*substream, buf) { Ok(bytes) => Poll::Ready(Ok(bytes)), - Err(WriteError::Blocked) => { - let mut substream = inner.substreams.get_mut(id).expect("known substream; qed"); - substream.write_waker = Some(cx.waker().clone()); + Err(quinn_proto::ReadError::Blocked) => { + if let Some(substream) = inner.substreams.get_mut(substream) { + if !substream + .read_waker + .as_ref() + .map_or(false, |w| w.will_wake(cx.waker())) + { + substream.read_waker = Some(cx.waker().clone()); + } + } Poll::Pending } - Err(WriteError::Stopped(_)) => Poll::Ready(Ok(0)), - Err(WriteError::UnknownStream) => { - Poll::Ready(Err(QuicMuxerError::UnknownStream { id: *id })) + + Err(quinn_proto::ReadError::Reset(err_code)) => { + Poll::Ready(Err(Error::Reset(err_code))) + }, + + // `IllegalOrderedRead` happens if an unordered read followed with an ordered read are + // performed. `libp2p-quic` never does any unordered read. + Err(quinn_proto::ReadError::IllegalOrderedRead) => unreachable!(), + Err(quinn_proto::ReadError::UnknownStream) => { + log::error!( + "The application used a connection that is already being \ + closed. This is a bug in the application or in libp2p." + ); + Poll::Pending } } } fn shutdown_substream( &self, - _: &mut Context, - id: &mut Self::Substream, + cx: &mut Context<'_>, + substream: &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 })) - } + let inner = &mut *inner; + + let mut substream_state = inner.substreams.get_mut(substream) + .expect("invalid StreamMuxer::shutdown_substream API usage"); + if substream_state.finished { + return Poll::Ready(Ok(())) + } + + match inner.connection.shutdown_substream(*substream) { + Ok(()) => { + match substream_state.finished_waker { + Some(ref w) if w.will_wake(cx.waker()) => {}, + _ => substream_state.finished_waker = Some(cx.waker().clone()), + } + Poll::Pending + }, + Err(quinn_proto::FinishError::Stopped(err)) => Poll::Ready(Err(Error::Reset(err))), + Err(quinn_proto::FinishError::UnknownStream) => { + // Illegal usage of the API. + debug_assert!(false); + Poll::Ready(Err(Error::ExpiredStream)) + }, } } - fn destroy_substream(&self, id: Self::Substream) { - tracing::trace!("destroying substream {}", id); + fn destroy_substream(&self, substream: Self::Substream) { 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(); - } - } + inner.substreams.remove(&substream); } fn flush_substream( &self, - _cx: &mut Context, - _id: &mut Self::Substream, + cx: &mut Context<'_>, + _: &mut Self::Substream, ) -> Poll> { - // quinn doesn't support flushing, calling close will flush all substreams. - Poll::Ready(Ok(())) + self.flush_all(cx) } - fn flush_all(&self, _cx: &mut Context) -> Poll> { - // quinn doesn't support flushing, calling close will flush all substreams. + // TODO: what if called multiple times? register all wakers? + fn flush_all(&self, _cx: &mut Context<'_>) -> Poll> { + // TODO: call poll_transmit() and stuff 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(())); + // TODO: what if called multiple times? register all wakers? + fn close(&self, cx: &mut Context<'_>) -> Poll> { + // StreamMuxer's `close` documentation mentions that it automatically implies `flush_all`. + if let Poll::Pending = self.flush_all(cx)? { + return Poll::Pending; } - inner.close_waker = Some(cx.waker().clone()); - let inner = &mut *inner; - for id in inner.substreams.keys() { - let _ = inner.connection.send_stream(*id).finish(); + + // TODO: poll if closed or something + + let mut inner = self.inner.lock(); + //self.connection.close(); + + // Register `cx.waker()` as being woken up if the connection closes. + if !inner + .poll_close_waker + .as_ref() + .map_or(false, |w| w.will_wake(cx.waker())) + { + inner.poll_close_waker = Some(cx.waker().clone()); } 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) +impl fmt::Debug for QuicMuxer { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_tuple("QuicMuxer").finish() } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index c3ffd428c64..a0934418f15 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -1,4 +1,4 @@ -// Copyright 2021 David Craven. +// Copyright 2017-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"), @@ -18,247 +18,120 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::endpoint::{EndpointConfig, TransportChannel}; -use crate::muxer::QuicMuxer; -use crate::{QuicConfig, QuicError}; -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 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) - .ok_or_else(|| TransportError::MultiaddrNotSupported(addr.clone()))?; - 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() - } -} +//! Implementation of the [`Transport`] trait for QUIC. +//! +//! Combines all the objects in the other modules to implement the trait. -struct QuicTransportInner { - channel: TransportChannel, - addresses: Addresses, -} +use crate::{endpoint::Endpoint, muxer::QuicMuxer, upgrade::Upgrade}; -enum Addresses { - Unspecified(IfWatcher), - Ip(Option), +use futures::prelude::*; +use libp2p_core::{ + multiaddr::{Multiaddr, Protocol}, + transport::{ListenerEvent, TransportError}, + PeerId, Transport, +}; +use std::{net::SocketAddr, pin::Pin, sync::Arc}; + +// We reexport the errors that are exposed in the API. +// All of these types use one another. +pub use crate::connection::Error as Libp2pQuicConnectionError; +pub use quinn_proto::{ + ApplicationClose, ConfigError, ConnectError, ConnectionClose, ConnectionError, + TransportError as QuinnTransportError, TransportErrorCode, +}; + +/// Wraps around an `Arc` and implements the [`Transport`] trait. +/// +/// > **Note**: This type is necessary because Rust unfortunately forbids implementing the +/// > `Transport` trait directly on `Arc`. +#[derive(Debug, Clone)] +pub struct QuicTransport(pub Arc); + +/// Error that can happen on the transport. +#[derive(Debug, thiserror::Error)] +pub enum Error { + /// Error while trying to reach a remote. + #[error("{0}")] + Reach(ConnectError), + /// Error after the remote has been reached. + #[error("{0}")] + Established(Libp2pQuicConnectionError), } impl Transport for QuicTransport { type Output = (PeerId, QuicMuxer); - type Error = QuicError; - type Listener = Self; - type ListenerUpgrade = QuicUpgrade; - type Dial = QuicDial; + type Error = Error; + type Listener = Pin< + Box, Self::Error>> + Send>, + >; + type ListenerUpgrade = Upgrade; + type Dial = Pin> + Send>>; fn listen_on(self, addr: Multiaddr) -> Result> { - multiaddr_to_socketaddr(&addr) - .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; - Ok(self) - } - - fn dial(self, addr: Multiaddr) -> Result> { - let socket_addr = multiaddr_to_socketaddr(&addr).ok_or_else(|| { - tracing::debug!("invalid multiaddr"); - 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); - Ok(QuicDial::Dialing(rx)) - } - - fn address_translation(&self, _listen: &Multiaddr, observed: &Multiaddr) -> Option { - Some(observed.clone()) - } -} - -impl Stream for QuicTransport { - type Item = Result, 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: QuicUpgrade::new(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 enum QuicDial { - Dialing(oneshot::Receiver>), - Upgrade(QuicUpgrade), -} - -impl Future for QuicDial { - type Output = Result<(PeerId, QuicMuxer), QuicError>; - - fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { - loop { - 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), + // TODO: check address correctness + + // TODO: report the locally opened addresses + + Ok(stream::unfold((), move |()| { + let endpoint = self.0.clone(); + let addr = addr.clone(); + async move { + let connec = endpoint.next_incoming().await; + let remote_addr = socketaddr_to_multiaddr(&connec.remote_addr()); + let event = Ok(ListenerEvent::Upgrade { + upgrade: Upgrade::from_connection(connec), + local_addr: addr.clone(), // TODO: hack + remote_addr, + }); + Some((event, ())) } - } + }) + .boxed()) } -} - -pub struct QuicUpgrade { - muxer: Option, -} - -impl QuicUpgrade { - fn new(muxer: QuicMuxer) -> Self { - Self { muxer: Some(muxer) } + + fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { + panic!("not implemented") } -} - -impl Future for QuicUpgrade { - 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(Err(err)) => Poll::Ready(Err(err.into())), - Poll::Ready(Ok(_)) => { - unreachable!("muxer.incoming is set to false so no events can be produced"); + fn dial(self, addr: Multiaddr) -> Result> { + let socket_addr = if let Ok(socket_addr) = multiaddr_to_socketaddr(&addr) { + if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { + return Err(TransportError::MultiaddrNotSupported(addr)); } + socket_addr + } else { + return Err(TransportError::MultiaddrNotSupported(addr)); + }; + + Ok(async move { + let connection = self.0.dial(socket_addr).await.map_err(Error::Reach)?; + let final_connec = Upgrade::from_connection(connection).await?; + Ok(final_connec) } + .boxed()) } } -/// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns None if the format +/// 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) -> Option { +pub(crate) fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result { let mut iter = addr.iter(); - let proto1 = iter.next()?; - let proto2 = iter.next()?; - let proto3 = iter.next()?; + let proto1 = iter.next().ok_or(())?; + let proto2 = iter.next().ok_or(())?; + let proto3 = iter.next().ok_or(())?; - while let Some(proto) = iter.next() { - match proto { - Protocol::P2p(_) => {} // Ignore a `/p2p/...` prefix of possibly outer protocols, if present. - _ => return None, - } + if iter.next().is_some() { + return Err(()); } match (proto1, proto2, proto3) { (Protocol::Ip4(ip), Protocol::Udp(port), Protocol::Quic) => { - Some(SocketAddr::new(ip.into(), port)) + Ok(SocketAddr::new(ip.into(), port)) } (Protocol::Ip6(ip), Protocol::Udp(port), Protocol::Quic) => { - Some(SocketAddr::new(ip.into(), port)) + Ok(SocketAddr::new(ip.into(), port)) } - _ => None, + _ => Err(()), } } @@ -271,67 +144,54 @@ pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { } #[cfg(test)] -mod tests { - use super::*; - - #[test] - fn multiaddr_to_socketaddr_conversion() { - use std::net::{Ipv4Addr, Ipv6Addr}; +#[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_none() - ); + 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() - ), - Some(SocketAddr::new( - IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), - 12345, - )) - ); - - assert!(multiaddr_to_socketaddr( - &"/ip4/127.0.0.1/udp/12345/quic/tcp/12345" + assert_eq!( + multiaddr_to_socketaddr( + &"/ip4/127.0.0.1/udp/12345/quic" .parse::() .unwrap() - ) - .is_none()); - - assert_eq!( - multiaddr_to_socketaddr( - &"/ip4/255.255.255.255/udp/8080/quic" - .parse::() - .unwrap() - ), - Some(SocketAddr::new( - IpAddr::V4(Ipv4Addr::new(255, 255, 255, 255)), - 8080, - )) - ); - assert_eq!( - multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), - Some(SocketAddr::new( - IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), - 12345, - )) - ); - assert_eq!( - multiaddr_to_socketaddr( - &"/ip6/ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/udp/8080/quic" - .parse::() - .unwrap() - ), - Some(SocketAddr::new( - IpAddr::V6(Ipv6Addr::new( - 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, - )), - 8080, - )) - ); - } + ), + Ok(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), + 12345, + )) + ); + 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, + )) + ); + 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, + )) + ); + 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, + )) + ); } diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs new file mode 100644 index 00000000000..71f2d00b285 --- /dev/null +++ b/transports/quic/src/upgrade.rs @@ -0,0 +1,87 @@ +// Copyright 2017-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. + +//! Future that drives a QUIC connection until is has performed its TLS handshake. + +use crate::{ + connection::{Connection, ConnectionEvent}, + muxer::QuicMuxer, + transport, x509, +}; + +use futures::prelude::*; +use libp2p_core::PeerId; +use std::{ + fmt, + pin::Pin, + task::{Context, Poll}, +}; + +/// A QUIC connection currently being negotiated. +pub struct Upgrade { + connection: Option, +} + +impl Upgrade { + /// Builds an [`Upgrade`] that wraps around a [`Connection`]. + pub(crate) fn from_connection(connection: Connection) -> Self { + Upgrade { + connection: Some(connection), + } + } +} + +impl Future for Upgrade { + type Output = Result<(PeerId, QuicMuxer), transport::Error>; + + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let connection = match self.connection.as_mut() { + Some(c) => c, + None => panic!("Future polled after it has ended"), + }; + + loop { + if let Some(mut certificates) = connection.peer_certificates() { + let peer_id = x509::extract_peerid_or_panic(certificates.next().unwrap().as_der()); // TODO: bad API + let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); + return Poll::Ready(Ok((peer_id, muxer))); + } + + match Connection::poll_event(connection, cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(ConnectionEvent::Connected) => { + // `is_handshaking()` will return `false` at the next loop iteration. + continue; + } + Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { + return Poll::Ready(Err(transport::Error::Established(err))); + } + // TODO: enumerate the items and explain how they can't happen + _ => unreachable!(), + } + } + } +} + +impl fmt::Debug for Upgrade { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fmt::Debug::fmt(&self.connection, f) + } +} From d8aa2f37cd0c6a2c5f3c2f09f7f5ede0a317a26c Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 2 Feb 2022 17:13:20 +0300 Subject: [PATCH 032/218] Fix deps --- transports/quic/Cargo.toml | 4 +- transports/quic/src/connection.rs | 88 ++++++++------- transports/quic/src/endpoint.rs | 60 ++++++++--- transports/quic/src/lib.rs | 4 +- transports/quic/src/muxer.rs | 172 +++++++++++++++++++++--------- transports/quic/src/transport.rs | 73 +++++++++---- transports/quic/src/upgrade.rs | 6 +- transports/quic/tests/smoke.rs | 29 +++-- 8 files changed, 291 insertions(+), 145 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 0520f08c04f..9c78efaa1e5 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -21,20 +21,20 @@ rcgen = { version = "0.8.11" } ring = { version = "0.16.20" } rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } thiserror = "1.0.26" -tracing = "0.1.26" +tracing = "0.1.29" udp-socket = "0.1.5" webpki = "0.22.0" x509-parser = "0.12.0" yasna = { version = "0.4.0" } async-std = "*" -log = "*" futures-timer = "*" [dev-dependencies] anyhow = "1.0.41" async-std = { version = "1.9.0", features = ["attributes"] } async-trait = "0.1.50" +async-macros = "*" libp2p = { version = "0.42.0", default-features = false, features = ["request-response"], path = "../.." } log-panics = "2.0.0" rand = "0.8.4" diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index f65ca718af0..7e05e05693e 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -52,7 +52,7 @@ pub(crate) struct Connection { from_endpoint: mpsc::Receiver, /// The QUIC state machine for this specific connection. - connection: quinn_proto::Connection, + pub(crate) connection: quinn_proto::Connection, /// Identifier for this connection according to the endpoint. Used when sending messages to /// the endpoint. connection_id: quinn_proto::ConnectionHandle, @@ -98,6 +98,7 @@ impl Connection { /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of /// its methods has ever been called. Failure to comply might lead to logic errors and panics. // TODO: maybe abstract `to_endpoint` more and make it generic? dunno + #[tracing::instrument(skip_all)] pub(crate) fn from_quinn_connection( endpoint: Arc, connection: quinn_proto::Connection, @@ -124,11 +125,11 @@ impl Connection { // TODO: it seems to happen that is_handshaking is false but this returns None pub(crate) fn peer_certificates( &self, - ) -> Option> { - self.connection - .crypto_session() - .get_peer_certificates() - .map(|l| l.into_iter().map(|l| l.into())) + ) -> Option> { + let session = self.connection.crypto_session(); + let identity = session.peer_identity()?; + let certs: Box> = identity.downcast().ok()?; + Some(certs.into_iter()) } /// Returns the address of the node we're connected to. @@ -166,7 +167,7 @@ impl Connection { /// If `None` is returned, then a [`ConnectionEvent::StreamAvailable`] event will later be /// produced when a substream is available. pub(crate) fn pop_incoming_substream(&mut self) -> Option { - self.connection.accept(quinn_proto::Dir::Bi) + self.connection.streams().accept(quinn_proto::Dir::Bi) } /// Pops a new substream opened locally. @@ -177,56 +178,61 @@ impl Connection { /// If `None` is returned, then a [`ConnectionEvent::StreamOpened`] event will later be /// produced when a substream is available. pub(crate) fn pop_outgoing_substream(&mut self) -> Option { - self.connection.open(quinn_proto::Dir::Bi) - } - - /// Reads data from the given substream. Similar to the API of `std::io::Read`. - /// - /// If `Err(ReadError::Blocked)` is returned, then a [`ConnectionEvent::StreamReadable`] event - /// will later be produced when the substream has readable data. A - /// [`ConnectionEvent::StreamStopped`] event can also be emitted. - pub(crate) fn read_substream( - &mut self, - id: quinn_proto::StreamId, - buf: &mut [u8], - ) -> Result { - self.connection.read(id, buf).map(|n| { - // `n` is `None` in case of EOF. - // See https://github.com/quinn-rs/quinn/blob/9aa3bde3aa1319b2c743f792312508de9270b8c6/quinn/src/streams.rs#L367-L370 - debug_assert_ne!(n, Some(0)); // Sanity check - n.unwrap_or(0) - }) + self.connection.streams().open(quinn_proto::Dir::Bi) } - /// Writes data to the given substream. Similar to the API of `std::io::Write`. - /// - /// If `Err(WriteError::Blocked)` is returned, then a [`ConnectionEvent::StreamWritable`] event - /// will later be produced when the substream can be written to. A - /// [`ConnectionEvent::StreamStopped`] event can also be emitted. - pub(crate) fn write_substream( - &mut self, - id: quinn_proto::StreamId, - buf: &[u8], - ) -> Result { - self.connection.write(id, buf) - } + // /// Reads data from the given substream. Similar to the API of `std::io::Read`. + // /// + // /// If `Err(ReadError::Blocked)` is returned, then a [`ConnectionEvent::StreamReadable`] event + // /// will later be produced when the substream has readable data. A + // /// [`ConnectionEvent::StreamStopped`] event can also be emitted. + // pub(crate) fn read_substream( + // &mut self, + // id: quinn_proto::StreamId, + // buf: &mut [u8], + // ) -> Result { + // let mut stream = self.connection.recv_stream(id); + // let mut chunks = stream.read(true)?; + // self.connection.read(id, buf).map(|n| { + // // `n` is `None` in case of EOF. + // // See https://github.com/quinn-rs/quinn/blob/9aa3bde3aa1319b2c743f792312508de9270b8c6/quinn/src/streams.rs#L367-L370 + // debug_assert_ne!(n, Some(0)); // Sanity check + // n.unwrap_or(0) + // }) + // } + + // /// Writes data to the given substream. Similar to the API of `std::io::Write`. + // /// + // /// If `Err(WriteError::Blocked)` is returned, then a [`ConnectionEvent::StreamWritable`] event + // /// will later be produced when the substream can be written to. A + // /// [`ConnectionEvent::StreamStopped`] event can also be emitted. + // pub(crate) fn write_substream( + // &mut self, + // id: quinn_proto::StreamId, + // buf: &[u8], + // ) -> Result { + // self.connection.write(id, buf) + // } /// Closes the given substream. /// /// [`Connection::write_substream`] must no longer be called. The substream is however still /// readable. /// - /// On success, a [`ConnectionEvent::StreamFinished`] event will later be produced when the + /// On success, a [`StreamEvent::Finished`] event will later be produced when the /// substream has been effectively closed. A [`ConnectionEvent::StreamStopped`] event can also /// be emitted. pub(crate) fn shutdown_substream( &mut self, id: quinn_proto::StreamId, ) -> Result<(), quinn_proto::FinishError> { - self.connection.finish(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. + self.connection.send_stream(id).finish() } /// Polls the connection for an event that happend on it. + #[tracing::instrument] pub(crate) fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { // Nothing more can be done if the connection is closed. // Return `Pending` without registering the waker, essentially freezing the task forever. @@ -270,7 +276,7 @@ impl Connection { // Poll the connection for packets to send on the UDP socket and try to send them on // `to_endpoint`. - while let Some(transmit) = self.connection.poll_transmit(now) { + while let Some(transmit) = self.connection.poll_transmit(now, 0) { let endpoint = self.endpoint.clone(); debug_assert!(self.pending_to_endpoint.is_none()); self.pending_to_endpoint = Some(Box::pin(async move { diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 1ac98f1efee..296ca3c5376 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,22 +28,28 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`background_task`] for a thorough description. -use crate::{connection::Connection, x509}; +use crate::{connection::Connection, tls}; use async_std::net::SocketAddr; use futures::{ channel::{mpsc, oneshot}, lock::Mutex, prelude::*, + stream::Stream, }; use libp2p_core::multiaddr::Multiaddr; use std::{ collections::{HashMap, VecDeque}, fmt, io, + pin::Pin, sync::{Arc, Weak}, - task::Poll, + task::{Context, Poll}, time::{Duration, Instant}, }; +use quinn_proto::{ + ClientConfig as QuinnClientConfig, + ServerConfig as QuinnServerConfig, +}; /// Represents the configuration for the [`Endpoint`]. #[derive(Debug, Clone)] @@ -63,19 +69,21 @@ impl Config { pub fn new( keypair: &libp2p_core::identity::Keypair, multiaddr: Multiaddr, - ) -> Result { + ) -> Result { let mut transport = quinn_proto::TransportConfig::default(); - transport.stream_window_uni(0).unwrap(); // Can only panic if value is out of range. + transport.max_concurrent_uni_streams(0u32.into()); // Can only panic if value is out of range. transport.datagram_receive_buffer_size(None); transport.keep_alive_interval(Some(Duration::from_millis(10))); let transport = Arc::new(transport); - let (client_tls_config, server_tls_config) = x509::make_tls_config(keypair)?; - let mut server_config = quinn_proto::ServerConfig::default(); - server_config.transport = transport.clone(); - server_config.crypto = Arc::new(server_tls_config); - let mut client_config = quinn_proto::ClientConfig::default(); + + let client_tls_config = tls::make_client_config(keypair).unwrap(); + let server_tls_config = tls::make_server_config(keypair).unwrap(); + + let mut server_config = QuinnServerConfig::with_crypto(Arc::new(server_tls_config)); + server_config.transport = Arc::clone(&transport); + + let mut client_config = QuinnClientConfig::new(Arc::new(client_tls_config)); client_config.transport = transport; - client_config.crypto = Arc::new(client_tls_config); Ok(Self { client_config, server_config: Arc::new(server_config), @@ -115,6 +123,7 @@ pub struct Endpoint { impl Endpoint { /// Builds a new `Endpoint`. + #[tracing::instrument(skip_all)] pub fn new(config: Config) -> Result, io::Error> { let local_socket_addr = match crate::transport::multiaddr_to_socketaddr(&config.multiaddr) { Ok(a) => a, @@ -205,6 +214,7 @@ impl Endpoint { /// /// Note that this method only *starts* the dialing. `Ok` is returned as soon as possible, even /// when the remote might end up being unreachable. + #[tracing::instrument] pub(crate) async fn dial( &self, addr: SocketAddr, @@ -235,6 +245,21 @@ impl Endpoint { .expect("background task has crashed") } + pub(crate) fn poll_incoming(&self, cx: &mut Context) -> Poll> { + // The `expect` below can panic if the background task has stopped. The background task + // can stop only if the `Endpoint` is destroyed or if the task itself panics. In other + // words, we panic here iff a panic has already happened somewhere else, which is a + // reasonable thing to do. + let mut connections_lock = self.new_connections.lock(); + let guard_poll = Pin::new(&mut connections_lock).poll(cx); + let mut guard = match guard_poll { + Poll::Ready(guard) => guard, + Poll::Pending => return Poll::Pending, + }; + let mut new_connections = &mut *guard; + Pin::new(&mut new_connections).poll_next(cx) + } + /// Asks the endpoint to send a UDP packet. /// /// Note that this method only queues the packet and returns as soon as the packet is in queue. @@ -408,6 +433,7 @@ enum ToEndpoint { /// guarantees that the [`Endpoint`], and therefore the background task, is properly kept alive /// for as long as any QUIC connection is open. /// +#[tracing::instrument(skip_all)] async fn background_task( config: Config, endpoint_weak: Weak, @@ -447,9 +473,10 @@ async fn background_task( // network interface is too busy, we back-pressure all of our internal // channels. // TODO: set ECN bits; there is no support for them in the ecosystem right now + tracing::trace_span!("udp_socket.send_to"); match udp_socket.send_to(&data, destination).await { Ok(n) if n == data.len() => {} - Ok(_) => log::error!( + Ok(_) => tracing::error!( "QUIC UDP socket violated expectation that packets are always fully \ transferred" ), @@ -458,7 +485,7 @@ async fn background_task( // printing a log message. The packet gets discarded in case of error, but we are // robust to packet losses and it is consequently not a logic error to process with // normal operations. - Err(err) => log::error!("Error while sending on QUIC UDP socket: {:?}", err), + Err(err) => tracing::error!("Error while sending on QUIC UDP socket: {:?}", err), } } @@ -553,7 +580,7 @@ async fn background_task( .expect("if queue is empty, the future above is always Pending; qed"); new_connections.start_send(elem) .expect("future is waken up only if poll_ready returned Ready; qed"); - endpoint.accept(); + //endpoint.accept(); } result = udp_socket.recv_from(&mut socket_recv_buffer).fuse() => { @@ -562,7 +589,7 @@ async fn background_task( // Errors on the socket are expected to never happen, and we handle them by // simply printing a log message. Err(err) => { - log::error!("Error while receive on QUIC UDP socket: {:?}", err); + tracing::error!("Error while receive on QUIC UDP socket: {:?}", err); continue; }, }; @@ -570,15 +597,16 @@ async fn background_task( // Received a UDP packet from the socket. debug_assert!(packet_len <= socket_recv_buffer.len()); let packet = From::from(&socket_recv_buffer[..packet_len]); + let local_ip = udp_socket.local_addr().ok().map(|a| a.ip()); // TODO: ECN bits aren't handled - match endpoint.handle(Instant::now(), packet_src, None, packet) { + match endpoint.handle(Instant::now(), packet_src, local_ip, None, packet) { None => {}, Some((connec_id, quinn_proto::DatagramEvent::ConnectionEvent(event))) => { // Event to send to an existing connection. if let Some(sender) = alive_connections.get_mut(&connec_id) { let _ = sender.send(event).await; // TODO: don't await here /!\ } else { - log::error!("State mismatch: event for closed connection"); + tracing::error!("State mismatch: event for closed connection"); } }, Some((connec_id, quinn_proto::DatagramEvent::NewConnection(connec))) => { diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 97dc27ef996..1c3a7531f6a 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -58,12 +58,12 @@ mod endpoint; mod error; mod muxer; mod upgrade; -mod x509; +mod tls; pub mod transport; pub use endpoint::{Config, Endpoint}; pub use error::Error; -pub use muxer::QuicMuxer; +pub use muxer::{QuicMuxer, Substream}; pub use transport::QuicTransport; pub use upgrade::Upgrade; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index f3477b9562b..8e6d8902db5 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -29,6 +29,8 @@ use std::{ task::{Context, Poll, Waker}, }; +pub type Substream = quinn_proto::StreamId; + /// State for a single opened QUIC connection. pub struct QuicMuxer { // Note: This could theoretically be an asynchronous future, in order to yield the current @@ -47,6 +49,8 @@ struct QuicMuxerInner { poll_substream_opened_waker: Option, /// Waker to wake if the connection is closed. poll_close_waker: Option, + /// Waker to wake if any event is happened. + poll_event_waker: Option, } /// State of a single substream. @@ -77,6 +81,7 @@ impl QuicMuxer { substreams: Default::default(), poll_substream_opened_waker: None, poll_close_waker: None, + poll_event_waker: None, }), } } @@ -95,7 +100,7 @@ impl StreamMuxer for QuicMuxer { while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { ConnectionEvent::Connected => { - log::error!("Unexpected Connected event on established QUIC connection"); + tracing::error!("Unexpected Connected event on established QUIC connection"); } ConnectionEvent::ConnectionLost(_) => { if let Some(waker) = inner.poll_close_waker.take() { @@ -149,6 +154,7 @@ impl StreamMuxer for QuicMuxer { inner.substreams.insert(substream, Default::default()); Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) } else { + inner.poll_event_waker = Some(cx.waker().clone()); Poll::Pending } } @@ -193,73 +199,141 @@ impl StreamMuxer for QuicMuxer { substream: &mut Self::Substream, buf: &[u8], ) -> Poll> { + use quinn_proto::{WriteError}; + let mut inner = self.inner.lock(); + let id = substream; - match inner.connection.write_substream(*substream, buf) { + match inner.connection.connection.send_stream(*id).write(buf) { Ok(bytes) => Poll::Ready(Ok(bytes)), - Err(quinn_proto::WriteError::Stopped(err_code)) => { - Poll::Ready(Err(Error::Reset(err_code))) - }, - Err(quinn_proto::WriteError::Blocked) => { - if let Some(substream) = inner.substreams.get_mut(substream) { - if !substream - .write_waker - .as_ref() - .map_or(false, |w| w.will_wake(cx.waker())) - { - substream.write_waker = Some(cx.waker().clone()); - } - } + Err(WriteError::Blocked) => { + let mut substream = inner.substreams.get_mut(id).expect("known substream; qed"); + substream.write_waker = Some(cx.waker().clone()); Poll::Pending } - Err(quinn_proto::WriteError::UnknownStream) => { - log::error!( - "The application used a connection that is already being \ - closed. This is a bug in the application or in libp2p." - ); - Poll::Pending + Err(WriteError::Stopped(_)) => Poll::Ready(Ok(0)), + Err(WriteError::UnknownStream) => { + Poll::Ready(Err(Self::Error::ExpiredStream)) } } + + // match inner.connection.write_substream(*substream, buf) { + // Ok(bytes) => Poll::Ready(Ok(bytes)), + // Err(quinn_proto::WriteError::Stopped(err_code)) => { + // Poll::Ready(Err(Error::Reset(err_code))) + // }, + // Err(quinn_proto::WriteError::Blocked) => { + // if let Some(substream) = inner.substreams.get_mut(substream) { + // if !substream + // .write_waker + // .as_ref() + // .map_or(false, |w| w.will_wake(cx.waker())) + // { + // substream.write_waker = Some(cx.waker().clone()); + // } + // } + // Poll::Pending + // } + // Err(quinn_proto::WriteError::UnknownStream) => { + // tracing::error!( + // "The application used a connection that is already being \ + // closed. This is a bug in the application or in libp2p." + // ); + // Poll::Pending + // } + // } } fn read_substream( &self, cx: &mut Context<'_>, substream: &mut Self::Substream, - buf: &mut [u8], + mut buf: &mut [u8], ) -> Poll> { - let mut inner = self.inner.lock(); + // let mut inner = self.inner.lock(); + + // match inner.connection.read_substream(*substream, buf) { + // Ok(bytes) => Poll::Ready(Ok(bytes)), + // Err(quinn_proto::ReadError::Blocked) => { + // if let Some(substream) = inner.substreams.get_mut(substream) { + // if !substream + // .read_waker + // .as_ref() + // .map_or(false, |w| w.will_wake(cx.waker())) + // { + // substream.read_waker = Some(cx.waker().clone()); + // } + // } + // Poll::Pending + // } + + // Err(quinn_proto::ReadError::Reset(err_code)) => { + // Poll::Ready(Err(Error::Reset(err_code))) + // }, + + // // `IllegalOrderedRead` happens if an unordered read followed with an ordered read are + // // performed. `libp2p-quic` never does any unordered read. + // Err(quinn_proto::ReadError::IllegalOrderedRead) => unreachable!(), + // Err(quinn_proto::ReadError::UnknownStream) => { + // tracing::error!( + // "The application used a connection that is already being \ + // closed. This is a bug in the application or in libp2p." + // ); + // Poll::Pending + // } + // } + + use quinn_proto::{ReadableError, ReadError}; + use std::io::Write; + + let id = *substream; - match inner.connection.read_substream(*substream, buf) { - Ok(bytes) => Poll::Ready(Ok(bytes)), - Err(quinn_proto::ReadError::Blocked) => { - if let Some(substream) = inner.substreams.get_mut(substream) { - if !substream - .read_waker - .as_ref() - .map_or(false, |w| w.will_wake(cx.waker())) - { - substream.read_waker = Some(cx.waker().clone()); - } + let mut inner = self.inner.lock(); + let mut stream = inner.connection.connection.recv_stream(id); + let mut chunks = match stream.read(true) { + Ok(chunks) => chunks, + Err(ReadableError::UnknownStream) => { + return Poll::Ready(Err(Self::Error::ExpiredStream)) + } + 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; } - Poll::Pending } - - Err(quinn_proto::ReadError::Reset(err_code)) => { - Poll::Ready(Err(Error::Reset(err_code))) - }, - - // `IllegalOrderedRead` happens if an unordered read followed with an ordered read are - // performed. `libp2p-quic` never does any unordered read. - Err(quinn_proto::ReadError::IllegalOrderedRead) => unreachable!(), - Err(quinn_proto::ReadError::UnknownStream) => { - log::error!( - "The application used a connection that is already being \ - closed. This is a bug in the application or in libp2p." - ); - Poll::Pending + } + if chunks.finalize().should_transmit() { + if let Some(waker) = inner.poll_event_waker.take() { + waker.wake(); } } + let substream = inner.substreams.get_mut(&id).expect("known substream; qed"); + if pending && bytes == 0 { + substream.read_waker = Some(cx.waker().clone()); + Poll::Pending + } else { + Poll::Ready(Ok(bytes)) + } } fn shutdown_substream( diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index a0934418f15..7a0036b901d 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -31,6 +31,7 @@ use libp2p_core::{ PeerId, Transport, }; use std::{net::SocketAddr, pin::Pin, sync::Arc}; +use std::task::{Poll, Context}; // We reexport the errors that are exposed in the API. // All of these types use one another. @@ -61,32 +62,34 @@ pub enum Error { impl Transport for QuicTransport { type Output = (PeerId, QuicMuxer); type Error = Error; - type Listener = Pin< - Box, Self::Error>> + Send>, - >; + // type Listener = Pin< + // Box, Self::Error>> + Send>, + // >; + type Listener = Self; type ListenerUpgrade = Upgrade; type Dial = Pin> + Send>>; fn listen_on(self, addr: Multiaddr) -> Result> { - // TODO: check address correctness - - // TODO: report the locally opened addresses - - Ok(stream::unfold((), move |()| { - let endpoint = self.0.clone(); - let addr = addr.clone(); - async move { - let connec = endpoint.next_incoming().await; - let remote_addr = socketaddr_to_multiaddr(&connec.remote_addr()); - let event = Ok(ListenerEvent::Upgrade { - upgrade: Upgrade::from_connection(connec), - local_addr: addr.clone(), // TODO: hack - remote_addr, - }); - Some((event, ())) - } - }) - .boxed()) + Ok(self) + // // TODO: check address correctness + + // // TODO: report the locally opened addresses + + // Ok(stream::unfold((), move |()| { + // let endpoint = self.0.clone(); + // let addr = addr.clone(); + // async move { + // let connec = endpoint.next_incoming().await; + // let remote_addr = socketaddr_to_multiaddr(&connec.remote_addr()); + // let event = Ok(ListenerEvent::Upgrade { + // upgrade: Upgrade::from_connection(connec), + // local_addr: addr.clone(), // TODO: hack + // remote_addr, + // }); + // Some((event, ())) + // } + // }) + // .boxed()) } fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { @@ -112,6 +115,32 @@ impl Transport for QuicTransport { } } + +impl Stream for QuicTransport { + type Item = Result, Error>; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let endpoint = self.0.clone(); + //let addr = addr.clone(); + + let connec = match endpoint.poll_incoming(cx) { + Poll::Ready(Some(conn)) => conn, + Poll::Ready(None) => return Poll::Ready(None), + Poll::Pending => return Poll::Pending, + }; + let remote_addr = socketaddr_to_multiaddr(&connec.remote_addr()); + let event = ListenerEvent::Upgrade { + upgrade: Upgrade::from_connection(connec), + local_addr: "/ip4/127.0.0.1/udp/0/quic".parse().unwrap(), // addr.clone(), // TODO: hack + remote_addr, + }; + Poll::Ready(Some(Ok(event))) + //Some((event, ())) + + //Poll::Pending + } +} + /// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns an error if the format /// of the multiaddr is wrong. pub(crate) fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result { diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 71f2d00b285..b86072cbd48 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -23,7 +23,7 @@ use crate::{ connection::{Connection, ConnectionEvent}, muxer::QuicMuxer, - transport, x509, + transport, tls, }; use futures::prelude::*; @@ -59,7 +59,9 @@ impl Future for Upgrade { loop { if let Some(mut certificates) = connection.peer_certificates() { - let peer_id = x509::extract_peerid_or_panic(certificates.next().unwrap().as_der()); // TODO: bad API + let cert = certificates.next().unwrap(); + let p2p_cert = tls::certificate::parse_certificate(cert.as_ref()).unwrap(); // TODO: bad API + let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index bf6f7f4008d..263088caebe 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -9,7 +9,9 @@ use libp2p::request_response::{ RequestResponseEvent, RequestResponseMessage, }; use libp2p::swarm::{Swarm, SwarmEvent}; -use libp2p_quic::QuicConfig; +use libp2p_core::muxing::StreamMuxerBox; +use libp2p::{Multiaddr, Transport}; +use libp2p_quic::{Config as QuicConfig, Endpoint as QuicEndpoint, QuicTransport}; use rand::RngCore; use std::{io, iter}; @@ -17,19 +19,24 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } +#[tracing::instrument] async fn create_swarm(keylog: bool) -> Result>> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); - let mut transport = QuicConfig::new(keypair); - transport - .transport - .max_idle_timeout(Some(quinn_proto::VarInt::from_u32(1_000u32).into())); - if keylog { - transport.enable_keylogger(); - } - let transport = transport - .listen_on("/ip4/127.0.0.1/udp/0/quic".parse()?) - .await? + let addr: Multiaddr = "/ip4/127.0.0.1/udp/0/quic".parse()?; + let config = QuicConfig::new(&keypair, addr.clone()).unwrap(); + let endpoint = QuicEndpoint::new(config).unwrap(); + let transport = QuicTransport(endpoint); + + // transport + // .transport + // .max_idle_timeout(Some(quinn_proto::VarInt::from_u32(1_000u32).into())); + // if keylog { + // transport.enable_keylogger(); + // } + + let transport = + Transport::map(transport, |(peer, muxer), _| (peer, StreamMuxerBox::new(muxer))) .boxed(); let protocols = iter::once((PingProtocol(), ProtocolSupport::Full)); From 99439ca6bf8f64d1831eed8dcf232f30b6afe96f Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 3 Feb 2022 17:07:42 +0300 Subject: [PATCH 033/218] Fix tests --- transports/quic/src/connection.rs | 8 ++-- transports/quic/src/endpoint.rs | 33 +++++++++---- transports/quic/src/muxer.rs | 1 + transports/quic/src/transport.rs | 77 ++++++++++++++++++++++--------- transports/quic/src/upgrade.rs | 50 ++++++++++++-------- transports/quic/tests/smoke.rs | 8 ++-- 6 files changed, 122 insertions(+), 55 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 7e05e05693e..f54b918c6ff 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -276,7 +276,8 @@ impl Connection { // Poll the connection for packets to send on the UDP socket and try to send them on // `to_endpoint`. - while let Some(transmit) = self.connection.poll_transmit(now, 0) { + // FIXME max_datagrams + while let Some(transmit) = self.connection.poll_transmit(now, 1) { let endpoint = self.endpoint.clone(); debug_assert!(self.pending_to_endpoint.is_none()); self.pending_to_endpoint = Some(Box::pin(async move { @@ -383,8 +384,9 @@ impl Connection { return Poll::Ready(ConnectionEvent::Connected); } quinn_proto::Event::HandshakeDataReady => { - debug_assert!(self.is_handshaking); - debug_assert!(self.connection.is_handshaking()); + if !self.is_handshaking { + tracing::error!("Got HandshakeDataReady while not handshaking"); + } } } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 296ca3c5376..8ee5ce09bce 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -119,6 +119,9 @@ pub struct Endpoint { /// Multiaddr of the local UDP socket passed in the configuration at initialization after it /// has potentially been modified to handle port number `0`. local_multiaddr: Multiaddr, + + // after bind(), the result is without port=0 + pub(crate) local_addr: SocketAddr, } impl Endpoint { @@ -126,8 +129,8 @@ impl Endpoint { #[tracing::instrument(skip_all)] pub fn new(config: Config) -> Result, io::Error> { let local_socket_addr = match crate::transport::multiaddr_to_socketaddr(&config.multiaddr) { - Ok(a) => a, - Err(()) => panic!(), // TODO: Err(TransportError::MultiaddrNotSupported(multiaddr)), + Some(a) => a, + None => panic!(), // TODO: Err(TransportError::MultiaddrNotSupported(multiaddr)), }; // NOT blocking, as per man:bind(2), as we pass an IP address. @@ -153,6 +156,7 @@ impl Endpoint { new_connections: Mutex::new(new_connections_rx), config: config.clone(), local_multiaddr: config.multiaddr.clone(), // TODO: no + local_addr: socket.local_addr()?, }); // TODO: just for testing, do proper task spawning @@ -245,19 +249,18 @@ impl Endpoint { .expect("background task has crashed") } + #[tracing::instrument] pub(crate) fn poll_incoming(&self, cx: &mut Context) -> Poll> { // The `expect` below can panic if the background task has stopped. The background task // can stop only if the `Endpoint` is destroyed or if the task itself panics. In other // words, we panic here iff a panic has already happened somewhere else, which is a // reasonable thing to do. let mut connections_lock = self.new_connections.lock(); - let guard_poll = Pin::new(&mut connections_lock).poll(cx); - let mut guard = match guard_poll { + let mut guard = match Pin::new(&mut connections_lock).poll(cx) { Poll::Ready(guard) => guard, Poll::Pending => return Poll::Pending, }; - let mut new_connections = &mut *guard; - Pin::new(&mut new_connections).poll_next(cx) + Pin::new(&mut *guard).poll_next(cx) } /// Asks the endpoint to send a UDP packet. @@ -491,11 +494,13 @@ async fn background_task( // The endpoint might request packets to be sent out. This is handled in priority to avoid // buffering up packets. + let span = tracing::trace_span!("endpoint.poll_transmit"); if let Some(packet) = endpoint.poll_transmit() { debug_assert!(next_packet_out.is_none()); next_packet_out = Some((packet.destination, packet.contents)); continue; } + drop(span); futures::select! { message = receiver.next() => { @@ -509,6 +514,7 @@ async fn background_task( // This `"l"` seems necessary because an empty string is an invalid domain // name. While we don't use domain names, the underlying rustls library // is based upon the assumption that we do. + tracing::trace!("endpoint.connect"); let (connection_id, connection) = match endpoint.connect(config.client_config.clone(), addr, "l") { Ok(c) => c, @@ -539,7 +545,13 @@ async fn background_task( if is_drained_event { alive_connections.remove(&connection_id); } - if let Some(event_back) = endpoint.handle_event(connection_id, event) { + + let span = tracing::trace_span!("endpoint.handle_event"); + let event_back = endpoint.handle_event(connection_id, event); + drop(span); + + if let Some(event_back) = event_back { + tracing::trace_span!("process event back"); debug_assert!(!is_drained_event); // TODO: don't await here /!\ alive_connections.get_mut(&connection_id).unwrap().send(event_back).await; @@ -599,7 +611,12 @@ async fn background_task( let packet = From::from(&socket_recv_buffer[..packet_len]); let local_ip = udp_socket.local_addr().ok().map(|a| a.ip()); // TODO: ECN bits aren't handled - match endpoint.handle(Instant::now(), packet_src, local_ip, None, packet) { + let span = tracing::trace_span!("endpoint.handle_event"); + let event = endpoint.handle(Instant::now(), packet_src, local_ip, None, packet); + drop(span); + + tracing::trace_span!("process endpoint event"); + match event { None => {}, Some((connec_id, quinn_proto::DatagramEvent::ConnectionEvent(event))) => { // Event to send to an existing connection. diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 8e6d8902db5..06b5ee704f5 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -93,6 +93,7 @@ impl StreamMuxer for QuicMuxer { type Error = Error; // TODO: what if called multiple times? register all wakers? + #[tracing::instrument] fn poll_event(&self, cx: &mut Context<'_>) -> Poll, Self::Error>> { // We use `poll_inbound` to perform the background processing of the entire connection. let mut inner = self.inner.lock(); diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 7a0036b901d..84903bf5075 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -46,7 +46,13 @@ pub use quinn_proto::{ /// > **Note**: This type is necessary because Rust unfortunately forbids implementing the /// > `Transport` trait directly on `Arc`. #[derive(Debug, Clone)] -pub struct QuicTransport(pub Arc); +pub struct QuicTransport(pub Arc, /* addr reported */ bool); // FIXME IfWatcher + +impl QuicTransport { + pub fn new(endpoint: Arc) -> Self { + Self(endpoint, false) + } +} /// Error that can happen on the transport. #[derive(Debug, thiserror::Error)] @@ -69,6 +75,7 @@ impl Transport for QuicTransport { type ListenerUpgrade = Upgrade; type Dial = Pin> + Send>>; + #[tracing::instrument] fn listen_on(self, addr: Multiaddr) -> Result> { Ok(self) // // TODO: check address correctness @@ -96,13 +103,17 @@ impl Transport for QuicTransport { panic!("not implemented") } + #[tracing::instrument] fn dial(self, addr: Multiaddr) -> Result> { - let socket_addr = if let Ok(socket_addr) = multiaddr_to_socketaddr(&addr) { - if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { - return Err(TransportError::MultiaddrNotSupported(addr)); - } + let socket_addr = if let Some(socket_addr) = multiaddr_to_socketaddr(&addr) { + // FIXME IfWatcher + // if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { + // tracing::error!("multiaddr not supported"); + // return Err(TransportError::MultiaddrNotSupported(addr)); + // } socket_addr } else { + tracing::error!("multiaddr not supported"); return Err(TransportError::MultiaddrNotSupported(addr)); }; @@ -119,8 +130,18 @@ impl Transport for QuicTransport { impl Stream for QuicTransport { type Item = Result, Error>; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + #[tracing::instrument(skip_all)] + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + tracing::trace!("QuicTransport::poll_next"); let endpoint = self.0.clone(); + + if !self.1 { // FIXME IfWatcher + self.1 = true; + let addr = socketaddr_to_multiaddr(&endpoint.local_addr); + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(addr)))); + } + + //let addr = addr.clone(); let connec = match endpoint.poll_incoming(cx) { @@ -141,26 +162,29 @@ impl Stream for QuicTransport { } } -/// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns an error if the format +/// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns None if the format /// of the multiaddr is wrong. -pub(crate) fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Result { +pub(crate) fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { let mut iter = addr.iter(); - let proto1 = iter.next().ok_or(())?; - let proto2 = iter.next().ok_or(())?; - let proto3 = iter.next().ok_or(())?; + let proto1 = iter.next()?; + let proto2 = iter.next()?; + let proto3 = iter.next()?; - if iter.next().is_some() { - return Err(()); + while let Some(proto) = iter.next() { + match proto { + Protocol::P2p(_) => {} // Ignore a `/p2p/...` prefix of possibly outer protocols, if present. + _ => return None, + } } match (proto1, proto2, proto3) { (Protocol::Ip4(ip), Protocol::Udp(port), Protocol::Quic) => { - Ok(SocketAddr::new(ip.into(), port)) + Some(SocketAddr::new(ip.into(), port)) } (Protocol::Ip6(ip), Protocol::Udp(port), Protocol::Quic) => { - Ok(SocketAddr::new(ip.into(), port)) + Some(SocketAddr::new(ip.into(), port)) } - _ => Err(()), + _ => None, } } @@ -178,7 +202,7 @@ 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() + multiaddr_to_socketaddr(&"/ip4/127.0.0.1/udp/1234".parse::().unwrap()).is_none() ); assert_eq!( @@ -187,7 +211,7 @@ fn multiaddr_to_udp_conversion() { .parse::() .unwrap() ), - Ok(SocketAddr::new( + Some(SocketAddr::new( IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 12345, )) @@ -198,14 +222,25 @@ fn multiaddr_to_udp_conversion() { .parse::() .unwrap() ), - Ok(SocketAddr::new( + Some(SocketAddr::new( IpAddr::V4(Ipv4Addr::new(255, 255, 255, 255)), 8080, )) ); + assert_eq!( + multiaddr_to_socketaddr( + &"/ip4/127.0.0.1/udp/55148/quic/p2p/12D3KooW9xk7Zp1gejwfwNpfm6L9zH5NL4Bx5rm94LRYJJHJuARZ" + .parse::() + .unwrap() + ), + Some(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), + 55148, + )) + ); assert_eq!( multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), - Ok(SocketAddr::new( + Some(SocketAddr::new( IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), 12345, )) @@ -216,7 +251,7 @@ fn multiaddr_to_udp_conversion() { .parse::() .unwrap() ), - Ok(SocketAddr::new( + Some(SocketAddr::new( IpAddr::V6(Ipv6Addr::new( 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, )), diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index b86072cbd48..0fcfc4336b8 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -51,33 +51,43 @@ impl Upgrade { impl Future for Upgrade { type Output = Result<(PeerId, QuicMuxer), transport::Error>; + #[tracing::instrument(skip_all)] fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let connection = match self.connection.as_mut() { - Some(c) => c, - None => panic!("Future polled after it has ended"), - }; + let connection = self.connection.as_mut() + .expect("Future polled after it has completed"); - loop { - if let Some(mut certificates) = connection.peer_certificates() { - let cert = certificates.next().unwrap(); - let p2p_cert = tls::certificate::parse_certificate(cert.as_ref()).unwrap(); // TODO: bad API + match Connection::poll_event(connection, cx) { + Poll::Pending => return Poll::Pending, + Poll::Ready(ConnectionEvent::Connected) => { + debug_assert!(!connection.is_handshaking()); + + let mut certificates = connection.peer_certificates() + .expect("connection got certificates because it passed TLS handshake; qed"); + let end_entity = certificates.next() + .expect("there should be exactly one certificate; qed"); + let end_entity_der = end_entity.as_ref(); + let p2p_cert = tls::certificate::parse_certificate(end_entity_der) + .expect("the certificate was validated during TLS handshake; qed"); let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } - - match Connection::poll_event(connection, cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(ConnectionEvent::Connected) => { - // `is_handshaking()` will return `false` at the next loop iteration. - continue; - } - Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { - return Poll::Ready(Err(transport::Error::Established(err))); - } - // TODO: enumerate the items and explain how they can't happen - _ => unreachable!(), + Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { + return Poll::Ready(Err(transport::Error::Established(err))); } + // Other items are: + // - StreamAvailable + // - StreamOpened + // - StreamReadable + // - StreamWritable + // - StreamFinished + // - StreamStopped + Poll::Ready(_) => { + // They can happen only after we finished handshake and connected to the peer. + // But for `Upgrade` we get `Connected` event, wrap connection into a muxer + // and pass it to the result Stream of muxers. + unreachable!() + }, } } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 263088caebe..f1b35cd2feb 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -24,10 +24,11 @@ async fn create_swarm(keylog: bool) -> Result>> let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let addr: Multiaddr = "/ip4/127.0.0.1/udp/0/quic".parse()?; - let config = QuicConfig::new(&keypair, addr.clone()).unwrap(); + let config = QuicConfig::new(&keypair, addr).unwrap(); let endpoint = QuicEndpoint::new(config).unwrap(); - let transport = QuicTransport(endpoint); + let transport = QuicTransport::new(endpoint); + // TODO: // transport // .transport // .max_idle_timeout(Some(quinn_proto::VarInt::from_u32(1_000u32).into())); @@ -42,13 +43,14 @@ async fn create_swarm(keylog: bool) -> Result>> let protocols = iter::once((PingProtocol(), ProtocolSupport::Full)); let cfg = RequestResponseConfig::default(); let behaviour = RequestResponse::new(PingCodec(), protocols, cfg); - tracing::info!("{}", peer_id); + tracing::info!(?peer_id); Ok(Swarm::new(transport, behaviour, peer_id)) } #[async_std::test] async fn smoke() -> Result<()> { tracing_subscriber::fmt() + .pretty() .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) .try_init() .ok(); From 35e67665cbc86f08e99354db5b25573c6bbc3a98 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 3 Feb 2022 17:32:13 +0300 Subject: [PATCH 034/218] Add Connection::remote_peer_id to extract peer_id --- transports/quic/src/connection.rs | 31 +++++++++++++++++++------------ transports/quic/src/upgrade.rs | 13 ++----------- 2 files changed, 21 insertions(+), 23 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index f54b918c6ff..5375e8c3b8f 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -37,6 +37,7 @@ use std::{ task::{Context, Poll}, time::Instant, }; +use libp2p_core::PeerId; /// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Upgrade`]. /// @@ -120,18 +121,6 @@ impl Connection { } } - /// Returns the certificates sent by the remote through the underlying TLS session. - /// Returns `None` if the connection is still handshaking. - // TODO: it seems to happen that is_handshaking is false but this returns None - pub(crate) fn peer_certificates( - &self, - ) -> Option> { - let session = self.connection.crypto_session(); - let identity = session.peer_identity()?; - let certs: Box> = identity.downcast().ok()?; - Some(certs.into_iter()) - } - /// Returns the address of the node we're connected to. // TODO: can change /!\ pub(crate) fn remote_addr(&self) -> SocketAddr { @@ -144,6 +133,24 @@ impl Connection { self.is_handshaking } + /// Returns the address of the node we're connected to. + /// Panics if the connection is still handshaking. + #[tracing::instrument(skip_all)] + pub(crate) fn remote_peer_id(&self) -> PeerId { + debug_assert!(!self.is_handshaking()); + let session = self.connection.crypto_session(); + let identity = session.peer_identity() + .expect("connection got identity because it passed TLS handshake; qed"); + let certificates: Box> = identity.downcast().ok() + .expect("we rely on rustls feature; qed"); + let end_entity = certificates.get(0) + .expect("there should be exactly one certificate; qed"); + let end_entity_der = end_entity.as_ref(); + let p2p_cert = crate::tls::certificate::parse_certificate(end_entity_der) + .expect("the certificate was validated during TLS handshake; qed"); + PeerId::from_public_key(&p2p_cert.extension.public_key) + } + /// If the connection is closed, returns why. If the connection is open, returns `None`. /// /// > **Note**: This method is also the main way to determine whether a connection is closed. diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 0fcfc4336b8..5f09ed37101 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -23,7 +23,7 @@ use crate::{ connection::{Connection, ConnectionEvent}, muxer::QuicMuxer, - transport, tls, + transport, }; use futures::prelude::*; @@ -59,16 +59,7 @@ impl Future for Upgrade { match Connection::poll_event(connection, cx) { Poll::Pending => return Poll::Pending, Poll::Ready(ConnectionEvent::Connected) => { - debug_assert!(!connection.is_handshaking()); - - let mut certificates = connection.peer_certificates() - .expect("connection got certificates because it passed TLS handshake; qed"); - let end_entity = certificates.next() - .expect("there should be exactly one certificate; qed"); - let end_entity_der = end_entity.as_ref(); - let p2p_cert = tls::certificate::parse_certificate(end_entity_der) - .expect("the certificate was validated during TLS handshake; qed"); - let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); + let peer_id = connection.remote_peer_id(); let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } From 15aa70bfaacad32fe4a7f6e4cf26595012345f6b Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 3 Feb 2022 17:36:49 +0300 Subject: [PATCH 035/218] Remove Connection::next_incoming in favor of poll_incoming --- transports/quic/src/endpoint.rs | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 8ee5ce09bce..d61c5ca0e60 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -237,18 +237,6 @@ impl Endpoint { } /// Tries to pop a new incoming connection from the queue. - pub(crate) async fn next_incoming(&self) -> Connection { - // The `expect` below can panic if the background task has stopped. The background task - // can stop only if the `Endpoint` is destroyed or if the task itself panics. In other - // words, we panic here iff a panic has already happened somewhere else, which is a - // reasonable thing to do. - let mut new_connections = self.new_connections.lock().await; - new_connections - .next() - .await - .expect("background task has crashed") - } - #[tracing::instrument] pub(crate) fn poll_incoming(&self, cx: &mut Context) -> Poll> { // The `expect` below can panic if the background task has stopped. The background task From 154b73d7bbf84135dd43b6c080c0cb7dcf230abb Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Mon, 7 Feb 2022 18:50:45 +0300 Subject: [PATCH 036/218] Add connection::local_addr to report it in ListenerEvent::Upgrade --- transports/quic/src/connection.rs | 16 ++++++++++++++++ transports/quic/src/transport.rs | 3 ++- 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 5375e8c3b8f..f4c72b037b4 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -121,6 +121,22 @@ impl Connection { } } + /// The local address which was used when the peer established the connection. + /// + /// Works for server connections only. + pub(crate) fn local_addr(&self) -> SocketAddr { + debug_assert_eq!(self.connection.side(), quinn_proto::Side::Server); + let endpoint_addr = self.endpoint.local_addr; + self.connection.local_ip() + .map(|ip| SocketAddr::new(ip, endpoint_addr.port())) + .unwrap_or_else(|| { + // In a normal case scenario this should not happen, because + // we get want to get a local addr for a server connection only. + tracing::error!("trying to get quinn::local_ip for a client"); + endpoint_addr + }) + } + /// Returns the address of the node we're connected to. // TODO: can change /!\ pub(crate) fn remote_addr(&self) -> SocketAddr { diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 84903bf5075..1ba46a29c25 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -149,10 +149,11 @@ impl Stream for QuicTransport { Poll::Ready(None) => return Poll::Ready(None), Poll::Pending => return Poll::Pending, }; + let local_addr = socketaddr_to_multiaddr(&connec.local_addr()); let remote_addr = socketaddr_to_multiaddr(&connec.remote_addr()); let event = ListenerEvent::Upgrade { upgrade: Upgrade::from_connection(connec), - local_addr: "/ip4/127.0.0.1/udp/0/quic".parse().unwrap(), // addr.clone(), // TODO: hack + local_addr, remote_addr, }; Poll::Ready(Some(Ok(event))) From 8bfcca2436ce19c2172f8bbf0ef2bf7650598270 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 15 Feb 2022 13:51:39 +0300 Subject: [PATCH 037/218] tmp: Swarm pool size = 1 --- swarm/src/lib.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 25539294d8a..cc93ab7d105 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -1276,6 +1276,7 @@ where self.network_config.or_else_with_executor(|| { match ThreadPoolBuilder::new() .name_prefix("libp2p-swarm-task-") + .pool_size(1) .create() { Ok(tp) => Some(Box::new(move |f| tp.spawn_ok(f))), From 30895127fb8e279fb86bb79a908def9598b14265 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 15 Feb 2022 15:39:35 +0300 Subject: [PATCH 038/218] Tmp commit --- transports/quic/Cargo.toml | 7 +- transports/quic/src/connection.rs | 25 +++- transports/quic/src/endpoint.rs | 56 +++++---- transports/quic/src/error.rs | 6 +- transports/quic/src/muxer.rs | 17 ++- transports/quic/src/transport.rs | 195 ++++++++++++++++++++++-------- transports/quic/tests/smoke.rs | 28 ++++- 7 files changed, 245 insertions(+), 89 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 9c78efaa1e5..5388e40b1f4 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -21,7 +21,7 @@ rcgen = { version = "0.8.11" } ring = { version = "0.16.20" } rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } thiserror = "1.0.26" -tracing = "0.1.29" +tracing = "0.1.30" udp-socket = "0.1.5" webpki = "0.22.0" x509-parser = "0.12.0" @@ -32,11 +32,12 @@ futures-timer = "*" [dev-dependencies] anyhow = "1.0.41" -async-std = { version = "1.9.0", features = ["attributes"] } +async-std = { version = "1.10.0", features = ["attributes"] } async-trait = "0.1.50" async-macros = "*" libp2p = { version = "0.42.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" +tracing-subscriber = {version = "0.3.8", features = ["env-filter"] } +tracing-flame = "0.2" \ No newline at end of file diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index f4c72b037b4..9dfe03c3c79 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -264,9 +264,13 @@ impl Connection { } // Process events that the endpoint has sent to us. + let span = tracing::trace_span!("connection.handle_event").entered(); loop { match Pin::new(&mut self.from_endpoint).poll_next(cx) { - Poll::Ready(Some(event)) => self.connection.handle_event(event), + Poll::Ready(Some(event)) => { + let _span = tracing::trace_span!("handle").entered(); + self.connection.handle_event(event) + }, Poll::Ready(None) => { debug_assert!(self.closed.is_none()); let err = Error::ClosedChannel; @@ -276,6 +280,7 @@ impl Connection { Poll::Pending => break, } } + drop(span); 'send_pending: loop { // Sending the pending event to the endpoint. If the endpoint is too busy, we just @@ -288,18 +293,21 @@ impl Connection { // However we don't deliver substream-related events to the user as long as // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. + let span = tracing::trace_span!("connection.send_pending").entered(); if let Some(pending_to_endpoint) = &mut self.pending_to_endpoint { match Future::poll(Pin::new(pending_to_endpoint), cx) { Poll::Pending => return Poll::Pending, Poll::Ready(()) => self.pending_to_endpoint = None, } } + drop(span); let now = Instant::now(); // Poll the connection for packets to send on the UDP socket and try to send them on // `to_endpoint`. // FIXME max_datagrams + let span = tracing::trace_span!("connection.poll_transmit").entered(); while let Some(transmit) = self.connection.poll_transmit(now, 1) { let endpoint = self.endpoint.clone(); debug_assert!(self.pending_to_endpoint.is_none()); @@ -311,9 +319,11 @@ impl Connection { })); continue 'send_pending; } + drop(span); // The connection also needs to be able to send control messages to the endpoint. This is // handled here, and we try to send them on `to_endpoint` as well. + let span = tracing::trace_span!("connection.poll_endpoint_events").entered(); while let Some(endpoint_event) = self.connection.poll_endpoint_events() { let endpoint = self.endpoint.clone(); let connection_id = self.connection_id; @@ -325,11 +335,13 @@ impl Connection { })); continue 'send_pending; } + drop(span); // Timeout system. // We break out of the following loop until if `poll_timeout()` returns `None` or if // polling `self.next_timeout` returns `Poll::Pending`. loop { + let _span = tracing::trace_span!("connection.timeout").entered(); if let Some(next_timeout) = &mut self.next_timeout { match Future::poll(Pin::new(next_timeout), cx) { Poll::Ready(()) => { @@ -351,6 +363,7 @@ impl Connection { } // The final step consists in handling the events related to the various substreams. + let _span = tracing::trace_span!("connection.poll").entered(); while let Some(event) = self.connection.poll() { match event { quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { @@ -376,6 +389,7 @@ impl Connection { }) => { // The `Stop` QUIC event is more or less similar to a `Reset`, except that // it applies only on the writing side of the pipe. + tracing::error!("stream stopped {}", id); return Poll::Ready(ConnectionEvent::StreamStopped(id)); } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { @@ -389,6 +403,7 @@ impl Connection { return Poll::Ready(ConnectionEvent::StreamOpened); } quinn_proto::Event::ConnectionLost { reason } => { + tracing::error!("connection lost {}", reason); debug_assert!(self.closed.is_none()); self.is_handshaking = false; let err = Error::Quinn(reason); @@ -429,9 +444,11 @@ impl fmt::Debug for Connection { impl Drop for Connection { fn drop(&mut self) { - // TODO: don't do that if already drained - // We send a message to the endpoint. - self.endpoint.report_quinn_event_non_block(self.connection_id, quinn_proto::EndpointEvent::drained()); + // TODO: + // if let Some(_) = self.closed.take() { + // We send a `Drained` message to the endpoint to clean endpoint's resources. + self.endpoint.report_quinn_event_non_block(self.connection_id, quinn_proto::EndpointEvent::drained()); + // } } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index d61c5ca0e60..2e54f2aa17a 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -239,15 +239,8 @@ impl Endpoint { /// Tries to pop a new incoming connection from the queue. #[tracing::instrument] pub(crate) fn poll_incoming(&self, cx: &mut Context) -> Poll> { - // The `expect` below can panic if the background task has stopped. The background task - // can stop only if the `Endpoint` is destroyed or if the task itself panics. In other - // words, we panic here iff a panic has already happened somewhere else, which is a - // reasonable thing to do. let mut connections_lock = self.new_connections.lock(); - let mut guard = match Pin::new(&mut connections_lock).poll(cx) { - Poll::Ready(guard) => guard, - Poll::Pending => return Poll::Pending, - }; + let mut guard = futures::ready!(Pin::new(&mut connections_lock).poll(cx)); Pin::new(&mut *guard).poll_next(cx) } @@ -256,6 +249,7 @@ impl Endpoint { /// Note that this method only queues the packet and returns as soon as the packet is in queue. /// There is no guarantee that the packet will actually be sent, but considering that this is /// a UDP packet, you cannot rely on the packet being delivered anyway. + #[tracing::instrument(skip_all)] pub(crate) async fn send_udp_packet( &self, destination: SocketAddr, @@ -277,11 +271,15 @@ impl Endpoint { /// /// If `event.is_drained()` is true, the event indicates that the connection no longer exists. /// This must therefore be the last event sent using this [`quinn_proto::ConnectionHandle`]. + #[tracing::instrument(skip_all)] pub(crate) async fn report_quinn_event( &self, connection_id: quinn_proto::ConnectionHandle, event: quinn_proto::EndpointEvent, ) { + if event.is_drained() { + tracing::error!("drained, {:?}", connection_id); + } self.to_endpoint .lock().await .send(ToEndpoint::ProcessConnectionEvent { @@ -297,11 +295,15 @@ impl Endpoint { /// /// This method bypasses back-pressure mechanisms and is meant to be called only from /// destructors, where waiting is not advisable. + #[tracing::instrument(skip_all)] pub(crate) fn report_quinn_event_non_block( &self, connection_id: quinn_proto::ConnectionHandle, event: quinn_proto::EndpointEvent, ) { + if event.is_drained() { + tracing::error!("drained, {:?}", connection_id); + } // We implement this by cloning the `mpsc::Sender`. Since each sender is guaranteed a slot // in the buffer, cloning the sender reserves the slot and sending thus always succeeds. let result = self.to_endpoint2 @@ -464,8 +466,8 @@ async fn background_task( // network interface is too busy, we back-pressure all of our internal // channels. // TODO: set ECN bits; there is no support for them in the ecosystem right now - tracing::trace_span!("udp_socket.send_to"); - match udp_socket.send_to(&data, destination).await { + let span = tracing::trace_span!("udp_socket.send_to"); + match udp_socket.send_to(&data, destination).instrument(span).await { Ok(n) if n == data.len() => {} Ok(_) => tracing::error!( "QUIC UDP socket violated expectation that packets are always fully \ @@ -482,16 +484,17 @@ async fn background_task( // The endpoint might request packets to be sent out. This is handled in priority to avoid // buffering up packets. - let span = tracing::trace_span!("endpoint.poll_transmit"); if let Some(packet) = endpoint.poll_transmit() { + tracing::trace!("endpoint.poll_transmit"); debug_assert!(next_packet_out.is_none()); next_packet_out = Some((packet.destination, packet.contents)); continue; } - drop(span); + + use tracing::Instrument; futures::select! { - message = receiver.next() => { + message = receiver.next().instrument(tracing::trace_span!("ToEndpoint message")).fuse() => { // Received a message from a different part of the code requesting us to // do something. match message { @@ -502,7 +505,6 @@ async fn background_task( // This `"l"` seems necessary because an empty string is an invalid domain // name. While we don't use domain names, the underlying rustls library // is based upon the assumption that we do. - tracing::trace!("endpoint.connect"); let (connection_id, connection) = match endpoint.connect(config.client_config.clone(), addr, "l") { Ok(c) => c, @@ -531,18 +533,21 @@ async fn background_task( // its ID can be reclaimed. let is_drained_event = event.is_drained(); if is_drained_event { + tracing::error!("ProcessConnectionEvent Drained : {:?}", connection_id); alive_connections.remove(&connection_id); } - let span = tracing::trace_span!("endpoint.handle_event"); + tracing::trace!("endpoint.handle_event"); let event_back = endpoint.handle_event(connection_id, event); - drop(span); if let Some(event_back) = event_back { - tracing::trace_span!("process event back"); + // tracing::trace_span!("process event back"); debug_assert!(!is_drained_event); - // TODO: don't await here /!\ - alive_connections.get_mut(&connection_id).unwrap().send(event_back).await; + if let Some(sender) = alive_connections.get_mut(&connection_id) { + let _ = sender.send(event_back).await; // TODO: don't await here /!\ + } else { + tracing::error!("event back: no such connection {:?}", connection_id); + } } } @@ -568,7 +573,9 @@ async fn background_task( let new_connections = &mut new_connections; future::poll_fn(move |cx| { if active { new_connections.poll_ready(cx) } else { Poll::Pending } - }).fuse() + }) + .instrument(tracing::trace_span!("readiness")) + .fuse() } => { if readiness.is_err() { // new_connections channel has been dropped, meaning that the endpoint has @@ -583,7 +590,7 @@ async fn background_task( //endpoint.accept(); } - result = udp_socket.recv_from(&mut socket_recv_buffer).fuse() => { + result = udp_socket.recv_from(&mut socket_recv_buffer).instrument(tracing::trace_span!("udp recv")).fuse() => { let (packet_len, packet_src) = match result { Ok(v) => v, // Errors on the socket are expected to never happen, and we handle them by @@ -599,15 +606,16 @@ async fn background_task( let packet = From::from(&socket_recv_buffer[..packet_len]); let local_ip = udp_socket.local_addr().ok().map(|a| a.ip()); // TODO: ECN bits aren't handled - let span = tracing::trace_span!("endpoint.handle_event"); + // let span = tracing::trace_span!("endpoint.handle_event"); let event = endpoint.handle(Instant::now(), packet_src, local_ip, None, packet); - drop(span); + // drop(span); - tracing::trace_span!("process endpoint event"); + // tracing::trace_span!("process endpoint event"); match event { None => {}, Some((connec_id, quinn_proto::DatagramEvent::ConnectionEvent(event))) => { // Event to send to an existing connection. + // tracing::trace_span!("send event"); if let Some(sender) = alive_connections.get_mut(&connec_id) { let _ = sender.send(event).await; // TODO: don't await here /!\ } else { diff --git a/transports/quic/src/error.rs b/transports/quic/src/error.rs index 9671950d4e3..14755fe165c 100644 --- a/transports/quic/src/error.rs +++ b/transports/quic/src/error.rs @@ -36,8 +36,8 @@ pub enum Error { #[error("Peer stopped receiving data: code {0}")] Stopped(quinn_proto::VarInt), /// Connection was prematurely closed - #[error("Connection was prematurely closed")] - ConnectionLost, + #[error("Connection was prematurely closed: {0}")] + ConnectionLost(crate::connection::Error), /// Error making the connection. #[error("Connection failure: {0}")] ConnectError(#[from] quinn_proto::ConnectError), @@ -79,7 +79,7 @@ impl From for io::Error { e @ Error::NetworkFailure | e @ Error::ConnectionClosing | e @ Error::ConnectError(_) => io::Error::new(ErrorKind::Other, e), - e @ Error::Stopped(_) | e @ Error::Reset(_) | e @ Error::ConnectionLost => { + e @ Error::Stopped(_) | e @ Error::Reset(_) | e @ Error::ConnectionLost(_) => { io::Error::new(ErrorKind::ConnectionAborted, e) } e @ Error::ExpiredStream => io::Error::new(ErrorKind::BrokenPipe, e), diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 06b5ee704f5..61705bf531c 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -103,10 +103,11 @@ impl StreamMuxer for QuicMuxer { ConnectionEvent::Connected => { tracing::error!("Unexpected Connected event on established QUIC connection"); } - ConnectionEvent::ConnectionLost(_) => { + ConnectionEvent::ConnectionLost(e) => { if let Some(waker) = inner.poll_close_waker.take() { waker.wake(); } + return Poll::Ready(Err(Error::ConnectionLost(e))) } ConnectionEvent::StreamOpened => { @@ -143,6 +144,8 @@ impl StreamMuxer for QuicMuxer { if let Some(waker) = substream.finished_waker.take() { waker.wake(); } + } else { + tracing::error!("no such substream={}", substream); } } ConnectionEvent::StreamAvailable => { @@ -165,6 +168,7 @@ impl StreamMuxer for QuicMuxer { } // TODO: what if called multiple times? register all wakers? + #[tracing::instrument(skip_all)] fn poll_outbound( &self, cx: &mut Context<'_>, @@ -194,6 +198,7 @@ impl StreamMuxer for QuicMuxer { fn destroy_outbound(&self, _: Self::OutboundSubstream) {} + #[tracing::instrument(skip_all)] fn write_substream( &self, cx: &mut Context<'_>, @@ -212,8 +217,12 @@ impl StreamMuxer for QuicMuxer { substream.write_waker = Some(cx.waker().clone()); Poll::Pending } - Err(WriteError::Stopped(_)) => Poll::Ready(Ok(0)), + Err(WriteError::Stopped(_)) => { + tracing::error!("write error Stopped: substream={}", id); + Poll::Ready(Ok(0)) + }, Err(WriteError::UnknownStream) => { + tracing::error!("write error UnknownStream substream={}", id); Poll::Ready(Err(Self::Error::ExpiredStream)) } } @@ -245,6 +254,7 @@ impl StreamMuxer for QuicMuxer { // } } + #[tracing::instrument(skip_all)] fn read_substream( &self, cx: &mut Context<'_>, @@ -294,6 +304,7 @@ impl StreamMuxer for QuicMuxer { let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(ReadableError::UnknownStream) => { + tracing::error!("read error UnknownStream: substream={}", id); return Poll::Ready(Err(Self::Error::ExpiredStream)) } Err(ReadableError::IllegalOrderedRead) => { @@ -337,6 +348,7 @@ impl StreamMuxer for QuicMuxer { } } + #[tracing::instrument(skip_all)] fn shutdown_substream( &self, cx: &mut Context<'_>, @@ -388,6 +400,7 @@ impl StreamMuxer for QuicMuxer { } // TODO: what if called multiple times? register all wakers? + #[tracing::instrument(skip_all)] fn close(&self, cx: &mut Context<'_>) -> Poll> { // StreamMuxer's `close` documentation mentions that it automatically implies `flush_all`. if let Poll::Pending = self.flush_all(cx)? { diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 1ba46a29c25..01226222e56 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -30,8 +30,11 @@ use libp2p_core::{ transport::{ListenerEvent, TransportError}, PeerId, Transport, }; -use std::{net::SocketAddr, pin::Pin, sync::Arc}; +use std::{net::{SocketAddr, IpAddr}, pin::Pin, sync::Arc}; use std::task::{Poll, Context}; +use if_watch::{IfEvent, IfWatcher}; +use futures::future::BoxFuture; +use futures::lock::Mutex; // We reexport the errors that are exposed in the API. // All of these types use one another. @@ -46,14 +49,55 @@ pub use quinn_proto::{ /// > **Note**: This type is necessary because Rust unfortunately forbids implementing the /// > `Transport` trait directly on `Arc`. #[derive(Debug, Clone)] -pub struct QuicTransport(pub Arc, /* addr reported */ bool); // FIXME IfWatcher +pub struct QuicTransport { + endpoint: Arc, + /// The IP addresses of network interfaces on which the listening socket + /// is accepting connections. + /// + /// If the listen socket listens on all interfaces, these may change over + /// time as interfaces become available or unavailable. + in_addr: InAddr, +} impl QuicTransport { pub fn new(endpoint: Arc) -> Self { - Self(endpoint, false) + let in_addr = if endpoint.local_addr.ip().is_unspecified() { + let watcher = IfWatch::Pending(IfWatcher::new().boxed()); + InAddr::Any { if_watch: Arc::new(Mutex::new(watcher)) } + } else { + InAddr::One { ip: Some(endpoint.local_addr.ip()) } + }; + Self { endpoint, in_addr } + } +} + +enum IfWatch { + Pending(BoxFuture<'static, std::io::Result>), + Ready(IfWatcher), +} + +impl std::fmt::Debug for IfWatch { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match *self { + IfWatch::Pending(_) => write!(f, "Pending"), + IfWatch::Ready(_) => write!(f, "Ready"), + } } } +/// The listening addresses of a `UdpSocket`. +#[derive(Clone, Debug)] +enum InAddr { + /// The socket accepts connections on a single interface. + One { + ip: Option, + }, + /// The socket accepts connections on all interfaces. + Any { + if_watch: Arc>, + }, +} + /// Error that can happen on the transport. #[derive(Debug, thiserror::Error)] pub enum Error { @@ -63,6 +107,9 @@ pub enum Error { /// Error after the remote has been reached. #[error("{0}")] Established(Libp2pQuicConnectionError), + /// Error while working with IfWatcher. + #[error("{0}")] + IfWatcher(std::io::Error), } impl Transport for QuicTransport { @@ -77,26 +124,9 @@ impl Transport for QuicTransport { #[tracing::instrument] fn listen_on(self, addr: Multiaddr) -> Result> { + multiaddr_to_socketaddr(&addr) + .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; Ok(self) - // // TODO: check address correctness - - // // TODO: report the locally opened addresses - - // Ok(stream::unfold((), move |()| { - // let endpoint = self.0.clone(); - // let addr = addr.clone(); - // async move { - // let connec = endpoint.next_incoming().await; - // let remote_addr = socketaddr_to_multiaddr(&connec.remote_addr()); - // let event = Ok(ListenerEvent::Upgrade { - // upgrade: Upgrade::from_connection(connec), - // local_addr: addr.clone(), // TODO: hack - // remote_addr, - // }); - // Some((event, ())) - // } - // }) - // .boxed()) } fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { @@ -118,7 +148,7 @@ impl Transport for QuicTransport { }; Ok(async move { - let connection = self.0.dial(socket_addr).await.map_err(Error::Reach)?; + let connection = self.endpoint.dial(socket_addr).await.map_err(Error::Reach)?; let final_connec = Upgrade::from_connection(connection).await?; Ok(final_connec) } @@ -131,35 +161,102 @@ impl Stream for QuicTransport { type Item = Result, Error>; #[tracing::instrument(skip_all)] - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll> { - tracing::trace!("QuicTransport::poll_next"); - let endpoint = self.0.clone(); + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let me = Pin::into_inner(self); + let endpoint = me.endpoint.as_ref(); - if !self.1 { // FIXME IfWatcher - self.1 = true; - let addr = socketaddr_to_multiaddr(&endpoint.local_addr); - return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(addr)))); + loop { + match &mut me.in_addr { + // If the listener is bound to a single interface, make sure the + // address is reported once. + InAddr::One { ip } => { + if let Some(ip) = ip.take() { + let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(ma)))); + } + }, + InAddr::Any { if_watch } => { + let mut ifwatch_lock = if_watch.lock(); + let mut ifwatch = futures::ready!(ifwatch_lock.poll_unpin(cx)); // TODO continue + match &mut *ifwatch { + // If we listen on all interfaces, wait for `if-watch` to be ready. + IfWatch::Pending(f) => { + // let mut f_lock = f.lock(); + // let mut f = futures::ready!(f_lock.poll_unpin(cx)); + match futures::ready!(f.poll_unpin(cx)) { + Ok(w) => { + *ifwatch = IfWatch::Ready(w); + continue; + } + Err(err) => { + tracing::debug! { + "Failed to begin observing interfaces: {:?}. Scheduling retry.", + err + }; + *ifwatch = IfWatch::Pending(IfWatcher::new().boxed()); + //me.pause = Some(Delay::new(me.sleep_on_error)); + return Poll::Ready(Some(Ok(ListenerEvent::Error(Error::IfWatcher(err))))); + } + } + }, + // Consume all events for up/down interface changes. + IfWatch::Ready(watch) => { + while let Poll::Ready(ev) = watch.poll_unpin(cx) { + match ev { + Ok(IfEvent::Up(inet)) => { + let ip = inet.addr(); + if endpoint.local_addr.is_ipv4() == ip.is_ipv4() + { + let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("New listen address: {}", ma); + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress( + ma, + )))); + } + } + Ok(IfEvent::Down(inet)) => { + let ip = inet.addr(); + if endpoint.local_addr.is_ipv4() == ip.is_ipv4() + { + let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("Expired listen address: {}", ma); + return Poll::Ready(Some(Ok( + ListenerEvent::AddressExpired(ma), + ))); + } + } + Err(err) => { + tracing::debug! { + "Failure polling interfaces: {:?}. Scheduling retry.", + err + }; + return Poll::Ready(Some(Ok(ListenerEvent::Error(Error::IfWatcher(err))))); + } + } + } + } + } + }, + } + break; } - - //let addr = addr.clone(); - - let connec = match endpoint.poll_incoming(cx) { - Poll::Ready(Some(conn)) => conn, - Poll::Ready(None) => return Poll::Ready(None), - Poll::Pending => return Poll::Pending, - }; - let local_addr = socketaddr_to_multiaddr(&connec.local_addr()); - let remote_addr = socketaddr_to_multiaddr(&connec.remote_addr()); - let event = ListenerEvent::Upgrade { - upgrade: Upgrade::from_connection(connec), - local_addr, - remote_addr, - }; - Poll::Ready(Some(Ok(event))) - //Some((event, ())) - - //Poll::Pending + let connection = match endpoint.poll_incoming(cx) { + Poll::Ready(Some(conn)) => conn, + Poll::Ready(None) => return Poll::Ready(None), + Poll::Pending => return Poll::Pending, + }; + let local_addr = socketaddr_to_multiaddr(&connection.local_addr()); + let remote_addr = socketaddr_to_multiaddr(&connection.remote_addr()); + let event = ListenerEvent::Upgrade { + upgrade: Upgrade::from_connection(connection), + local_addr, + remote_addr, + }; + Poll::Ready(Some(Ok(event))) } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index f1b35cd2feb..a46dfac98cc 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -47,13 +47,32 @@ async fn create_swarm(keylog: bool) -> Result>> Ok(Swarm::new(transport, behaviour, peer_id)) } -#[async_std::test] -async fn smoke() -> Result<()> { - tracing_subscriber::fmt() +fn setup_global_subscriber() -> impl Drop { + use tracing_flame::FlameLayer; + use tracing_subscriber::{prelude::*, fmt}; + + let filter_layer = tracing_subscriber::EnvFilter::from_default_env(); + + let fmt_format = tracing_subscriber::fmt::format() .pretty() - .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .with_thread_ids(false) + .without_time(); + let fmt_layer = fmt::Layer::default().event_format(fmt_format); + + let (flame_layer, _guard) = FlameLayer::with_file("./tracing.folded").unwrap(); + + tracing_subscriber::registry() + .with(filter_layer) + .with(fmt_layer) + .with(flame_layer) .try_init() .ok(); + _guard +} + +#[async_std::test] +async fn smoke() -> Result<()> { + let _guard = setup_global_subscriber(); log_panics::init(); let mut rng = rand::thread_rng(); @@ -66,6 +85,7 @@ async fn smoke() -> Result<()> { Some(SwarmEvent::NewListenAddr { address, .. }) => address, e => panic!("{:?}", e), }; + tracing::info!(?addr); let mut data = vec![0; 4096 * 10]; rng.fill_bytes(&mut data); From f7e52d0317eb39630eb3088d76e02f1963f7ef8a Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 15 Feb 2022 15:37:26 +0300 Subject: [PATCH 039/218] Fix InAddr::One to produce event only once, move InAddr into a separate mod --- transports/quic/src/in_addr.rs | 117 +++++++++++++++++++++ transports/quic/src/lib.rs | 1 + transports/quic/src/transport.rs | 173 ++++++++++--------------------- 3 files changed, 171 insertions(+), 120 deletions(-) create mode 100644 transports/quic/src/in_addr.rs diff --git a/transports/quic/src/in_addr.rs b/transports/quic/src/in_addr.rs new file mode 100644 index 00000000000..5a45b4c7fa7 --- /dev/null +++ b/transports/quic/src/in_addr.rs @@ -0,0 +1,117 @@ +use if_watch::{IfEvent, IfWatcher}; + +use futures::{ + future::{BoxFuture, FutureExt}, + lock::Mutex, + stream::{Stream, StreamExt}, +}; + +use std::{ + io::Result, + net::IpAddr, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +#[derive(Clone, Debug)] +pub(crate) struct InAddr(Arc>); + +impl InAddr { + pub(crate) fn new(ip: IpAddr) -> Self { + let inner = if ip.is_unspecified() { + let watcher = IfWatch::Pending(IfWatcher::new().boxed()); + InAddrInner::Any { if_watch: watcher } + } else { + InAddrInner::One { ip: Some(ip) } + }; + Self(Arc::new(Mutex::new(inner))) + } +} + +/// The listening addresses of a `UdpSocket`. +#[derive(Debug)] +enum InAddrInner { + /// The socket accepts connections on a single interface. + One { ip: Option }, + /// The socket accepts connections on all interfaces. + Any { if_watch: IfWatch }, +} + +enum IfWatch { + Pending(BoxFuture<'static, std::io::Result>), + Ready(IfWatcher), +} + +impl std::fmt::Debug for IfWatch { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + match *self { + IfWatch::Pending(_) => write!(f, "Pending"), + IfWatch::Ready(_) => write!(f, "Ready"), + } + } +} + +impl Stream for InAddr { + type Item = Result; + + #[tracing::instrument(skip_all)] + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let me = Pin::into_inner(self); + let mut lock = me.0.lock(); + let mut guard = futures::ready!(lock.poll_unpin(cx)); + let inner = &mut *guard; + + inner.poll_next_unpin(cx) + } +} + +impl Stream for InAddrInner { + type Item = Result; + + #[tracing::instrument(skip_all)] + fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + let me = Pin::into_inner(self); + loop { + match me { + // If the listener is bound to a single interface, make sure the + // address is reported once. + InAddrInner::One { ip } => { + if let Some(ip) = ip.take() { + return Poll::Ready(Some(Ok(IfEvent::Up(ip.into())))); + } + } + InAddrInner::Any { if_watch } => { + match &mut *if_watch { + // If we listen on all interfaces, wait for `if-watch` to be ready. + IfWatch::Pending(f) => match futures::ready!(f.poll_unpin(cx)) { + Ok(w) => { + *if_watch = IfWatch::Ready(w); + continue; + } + Err(err) => { + *if_watch = IfWatch::Pending(IfWatcher::new().boxed()); + return Poll::Ready(Some(Err(err))); + } + }, + // Consume all events for up/down interface changes. + IfWatch::Ready(watch) => { + while let Poll::Ready(ev) = watch.poll_unpin(cx) { + match ev { + Ok(event) => { + return Poll::Ready(Some(Ok(event))); + } + Err(err) => { + return Poll::Ready(Some(Err(err))); + } + } + } + } + } + } + } + break; + } + Poll::Pending + } +} diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 1c3a7531f6a..4994c2a2437 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -56,6 +56,7 @@ mod connection; mod endpoint; mod error; +mod in_addr; mod muxer; mod upgrade; mod tls; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 01226222e56..125c2efe926 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -22,19 +22,20 @@ //! //! Combines all the objects in the other modules to implement the trait. -use crate::{endpoint::Endpoint, muxer::QuicMuxer, upgrade::Upgrade}; +use crate::{endpoint::Endpoint, in_addr::InAddr, muxer::QuicMuxer, upgrade::Upgrade}; use futures::prelude::*; +use futures::stream::StreamExt; + +use if_watch::IfEvent; + use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, transport::{ListenerEvent, TransportError}, PeerId, Transport, }; -use std::{net::{SocketAddr, IpAddr}, pin::Pin, sync::Arc}; -use std::task::{Poll, Context}; -use if_watch::{IfEvent, IfWatcher}; -use futures::future::BoxFuture; -use futures::lock::Mutex; +use std::task::{Context, Poll}; +use std::{net::SocketAddr, pin::Pin, sync::Arc}; // We reexport the errors that are exposed in the API. // All of these types use one another. @@ -61,43 +62,11 @@ pub struct QuicTransport { impl QuicTransport { pub fn new(endpoint: Arc) -> Self { - let in_addr = if endpoint.local_addr.ip().is_unspecified() { - let watcher = IfWatch::Pending(IfWatcher::new().boxed()); - InAddr::Any { if_watch: Arc::new(Mutex::new(watcher)) } - } else { - InAddr::One { ip: Some(endpoint.local_addr.ip()) } - }; + let in_addr = InAddr::new(endpoint.local_addr.ip()); Self { endpoint, in_addr } } } -enum IfWatch { - Pending(BoxFuture<'static, std::io::Result>), - Ready(IfWatcher), -} - -impl std::fmt::Debug for IfWatch { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - match *self { - IfWatch::Pending(_) => write!(f, "Pending"), - IfWatch::Ready(_) => write!(f, "Ready"), - } - } -} - -/// The listening addresses of a `UdpSocket`. -#[derive(Clone, Debug)] -enum InAddr { - /// The socket accepts connections on a single interface. - One { - ip: Option, - }, - /// The socket accepts connections on all interfaces. - Any { - if_watch: Arc>, - }, -} - /// Error that can happen on the transport. #[derive(Debug, thiserror::Error)] pub enum Error { @@ -128,19 +97,18 @@ impl Transport for QuicTransport { .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; Ok(self) } - + fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { - panic!("not implemented") + panic!("not implemented") } #[tracing::instrument] fn dial(self, addr: Multiaddr) -> Result> { let socket_addr = if let Some(socket_addr) = multiaddr_to_socketaddr(&addr) { - // FIXME IfWatcher - // if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { - // tracing::error!("multiaddr not supported"); - // return Err(TransportError::MultiaddrNotSupported(addr)); - // } + if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { + tracing::error!("multiaddr not supported"); + return Err(TransportError::MultiaddrNotSupported(addr)); + } socket_addr } else { tracing::error!("multiaddr not supported"); @@ -148,7 +116,11 @@ impl Transport for QuicTransport { }; Ok(async move { - let connection = self.endpoint.dial(socket_addr).await.map_err(Error::Reach)?; + let connection = self + .endpoint + .dial(socket_addr) + .await + .map_err(Error::Reach)?; let final_connec = Upgrade::from_connection(connection).await?; Ok(final_connec) } @@ -156,7 +128,6 @@ impl Transport for QuicTransport { } } - impl Stream for QuicTransport { type Item = Result, Error>; @@ -165,83 +136,45 @@ impl Stream for QuicTransport { let me = Pin::into_inner(self); let endpoint = me.endpoint.as_ref(); - loop { - match &mut me.in_addr { - // If the listener is bound to a single interface, make sure the - // address is reported once. - InAddr::One { ip } => { - if let Some(ip) = ip.take() { - let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(ma)))); - } - }, - InAddr::Any { if_watch } => { - let mut ifwatch_lock = if_watch.lock(); - let mut ifwatch = futures::ready!(ifwatch_lock.poll_unpin(cx)); // TODO continue - match &mut *ifwatch { - // If we listen on all interfaces, wait for `if-watch` to be ready. - IfWatch::Pending(f) => { - // let mut f_lock = f.lock(); - // let mut f = futures::ready!(f_lock.poll_unpin(cx)); - match futures::ready!(f.poll_unpin(cx)) { - Ok(w) => { - *ifwatch = IfWatch::Ready(w); - continue; - } - Err(err) => { - tracing::debug! { - "Failed to begin observing interfaces: {:?}. Scheduling retry.", - err - }; - *ifwatch = IfWatch::Pending(IfWatcher::new().boxed()); - //me.pause = Some(Delay::new(me.sleep_on_error)); - return Poll::Ready(Some(Ok(ListenerEvent::Error(Error::IfWatcher(err))))); - } + // Poll for a next IfEvent + match me.in_addr.poll_next_unpin(cx) { + Poll::Ready(mut item) => { + if let Some(item) = item.take() { + // Consume all events for up/down interface changes. + match item { + Ok(IfEvent::Up(inet)) => { + let ip = inet.addr(); + if endpoint.local_addr.is_ipv4() == ip.is_ipv4() { + let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("New listen address: {}", ma); + return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(ma)))); } - }, - // Consume all events for up/down interface changes. - IfWatch::Ready(watch) => { - while let Poll::Ready(ev) = watch.poll_unpin(cx) { - match ev { - Ok(IfEvent::Up(inet)) => { - let ip = inet.addr(); - if endpoint.local_addr.is_ipv4() == ip.is_ipv4() - { - let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("New listen address: {}", ma); - return Poll::Ready(Some(Ok(ListenerEvent::NewAddress( - ma, - )))); - } - } - Ok(IfEvent::Down(inet)) => { - let ip = inet.addr(); - if endpoint.local_addr.is_ipv4() == ip.is_ipv4() - { - let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("Expired listen address: {}", ma); - return Poll::Ready(Some(Ok( - ListenerEvent::AddressExpired(ma), - ))); - } - } - Err(err) => { - tracing::debug! { - "Failure polling interfaces: {:?}. Scheduling retry.", - err - }; - return Poll::Ready(Some(Ok(ListenerEvent::Error(Error::IfWatcher(err))))); - } - } + } + Ok(IfEvent::Down(inet)) => { + let ip = inet.addr(); + if endpoint.local_addr.is_ipv4() == ip.is_ipv4() { + let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("Expired listen address: {}", ma); + return Poll::Ready(Some(Ok(ListenerEvent::AddressExpired(ma)))); } } + Err(err) => { + tracing::debug! { + "Failure polling interfaces: {:?}.", + err + }; + return Poll::Ready(Some(Ok(ListenerEvent::Error(Error::IfWatcher( + err, + ))))); + } } - }, + } + } + Poll::Pending => { + // continue polling endpoint } - break; } let connection = match endpoint.poll_incoming(cx) { From 3bdcb4ae194ba4743f524997457efd883dca5313 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Sat, 26 Feb 2022 20:40:38 +0300 Subject: [PATCH 040/218] Add more logs --- transports/quic/src/muxer.rs | 27 +++++++++++++++++++++++++-- transports/quic/src/upgrade.rs | 4 +++- 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 61705bf531c..96b13eb0a45 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -93,12 +93,21 @@ impl StreamMuxer for QuicMuxer { type Error = Error; // TODO: what if called multiple times? register all wakers? - #[tracing::instrument] + #[tracing::instrument(skip_all)] fn poll_event(&self, cx: &mut Context<'_>) -> Poll, Self::Error>> { + //tracing::info!("here"); // We use `poll_inbound` to perform the background processing of the entire connection. let mut inner = self.inner.lock(); + let span = if inner.connection.connection.side().is_client() { + tracing::info_span!("client") + } else { + tracing::info_span!("server") + }; + let _enter = span.entered(); + while let Poll::Ready(event) = inner.connection.poll_event(cx) { + tracing::info!(?event); match event { ConnectionEvent::Connected => { tracing::error!("Unexpected Connected event on established QUIC connection"); @@ -149,21 +158,26 @@ impl StreamMuxer for QuicMuxer { } } ConnectionEvent::StreamAvailable => { + tracing::info!("StreamAvailable"); // Handled below. } } } if let Some(substream) = inner.connection.pop_incoming_substream() { + tracing::info!("pop_incoming_substream"); inner.substreams.insert(substream, Default::default()); Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) } else { + tracing::info!("set poll_event_waker"); inner.poll_event_waker = Some(cx.waker().clone()); Poll::Pending } } + #[tracing::instrument(skip_all)] fn open_outbound(&self) -> Self::OutboundSubstream { + tracing::info!("open_outbound"); () } @@ -177,9 +191,10 @@ impl StreamMuxer for QuicMuxer { // Note: this implementation makes it possible to poll the same `Self::OutboundSubstream` // over and over again and get new substreams. Using the API this way is invalid and would // normally result in a panic, but we decide to just ignore this problem. - + tracing::info!("poll_outbound"); let mut inner = self.inner.lock(); if let Some(substream) = inner.connection.pop_outgoing_substream() { + tracing::info!(?inner.poll_substream_opened_waker, "poll_outbound substream"); inner.substreams.insert(substream, Default::default()); return Poll::Ready(Ok(substream)); } @@ -190,6 +205,7 @@ impl StreamMuxer for QuicMuxer { .as_ref() .map_or(false, |w| w.will_wake(cx.waker())) { + tracing::info!("set poll_substream_opened_waker"); inner.poll_substream_opened_waker = Some(cx.waker().clone()); } @@ -208,6 +224,10 @@ impl StreamMuxer for QuicMuxer { use quinn_proto::{WriteError}; let mut inner = self.inner.lock(); + + let side = inner.connection.connection.side(); + tracing::info!(?side, ?substream, "write_substream"); + let id = substream; match inner.connection.connection.send_stream(*id).write(buf) { @@ -300,6 +320,8 @@ impl StreamMuxer for QuicMuxer { let id = *substream; let mut inner = self.inner.lock(); + let side = inner.connection.connection.side(); + tracing::info!(?side, ?id, "read_substream"); let mut stream = inner.connection.connection.recv_stream(id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, @@ -356,6 +378,7 @@ impl StreamMuxer for QuicMuxer { ) -> Poll> { let mut inner = self.inner.lock(); let inner = &mut *inner; + tracing::info!(?inner.connection.connection, "shutdown_substream"); let mut substream_state = inner.substreams.get_mut(substream) .expect("invalid StreamMuxer::shutdown_substream API usage"); diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 5f09ed37101..3207d86e124 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -56,7 +56,9 @@ impl Future for Upgrade { let connection = self.connection.as_mut() .expect("Future polled after it has completed"); - match Connection::poll_event(connection, cx) { + let event = Connection::poll_event(connection, cx); + tracing::info!(?event); + match event { Poll::Pending => return Poll::Pending, Poll::Ready(ConnectionEvent::Connected) => { let peer_id = connection.remote_peer_id(); From 86c41293003924f3dc416e99802b4579416e1d67 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 6 Apr 2022 23:00:33 +0300 Subject: [PATCH 041/218] Fixing disconnects --- transports/quic/Cargo.toml | 5 +- transports/quic/src/connection.rs | 58 +++++++---- transports/quic/src/endpoint.rs | 9 +- transports/quic/src/muxer.rs | 145 +++++++++++++++++++++++---- transports/quic/tests/smoke.rs | 157 +++++++++++++++++++++++++++++- 5 files changed, 330 insertions(+), 44 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 5388e40b1f4..3fd92f4e188 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -16,7 +16,7 @@ futures = "0.3.15" if-watch = "0.2.2" libp2p-core = { version = "0.31.0", path = "../../core" } parking_lot = "0.11.1" -quinn-proto = { version = "0.8.0", default-features = false, features = ["tls-rustls"] } +quinn-proto = { version = "0.8.2", default-features = false, features = ["tls-rustls"] } rcgen = { version = "0.8.11" } ring = { version = "0.16.20" } rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } @@ -40,4 +40,5 @@ log-panics = "2.0.0" rand = "0.8.4" rand_core = "0.5.1" tracing-subscriber = {version = "0.3.8", features = ["env-filter"] } -tracing-flame = "0.2" \ No newline at end of file +tracing-flame = "0.2" +quickcheck = "1" diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 9dfe03c3c79..0fe09b44d35 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -183,6 +183,7 @@ impl Connection { // support this. self.connection .close(Instant::now(), From::from(0u32), Default::default()); + self.endpoint.report_quinn_event_non_block(self.connection_id, quinn_proto::EndpointEvent::drained()); } /// Pops a new substream opened by the remote. @@ -254,6 +255,16 @@ impl Connection { self.connection.send_stream(id).finish() } + // pub(crate) fn flush_substream( + // &mut self, + // id: quinn_proto::StreamId, + // ) -> Result<(), quinn_proto::FinishError> { + // let mut substream = self.connection.send_stream(id); + // if let Ok(_) = substream.stopped() { + + // } + // } + /// Polls the connection for an event that happend on it. #[tracing::instrument] pub(crate) fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { @@ -267,11 +278,12 @@ impl Connection { let span = tracing::trace_span!("connection.handle_event").entered(); loop { match Pin::new(&mut self.from_endpoint).poll_next(cx) { - Poll::Ready(Some(event)) => { + Poll::Ready(Some(event)) => { let _span = tracing::trace_span!("handle").entered(); self.connection.handle_event(event) }, Poll::Ready(None) => { + tracing::error!("connection handle event should close connection"); debug_assert!(self.closed.is_none()); let err = Error::ClosedChannel; self.closed = Some(err.clone()); @@ -321,22 +333,6 @@ impl Connection { } drop(span); - // The connection also needs to be able to send control messages to the endpoint. This is - // handled here, and we try to send them on `to_endpoint` as well. - let span = tracing::trace_span!("connection.poll_endpoint_events").entered(); - while let Some(endpoint_event) = self.connection.poll_endpoint_events() { - let endpoint = self.endpoint.clone(); - let connection_id = self.connection_id; - debug_assert!(self.pending_to_endpoint.is_none()); - self.pending_to_endpoint = Some(Box::pin(async move { - endpoint - .report_quinn_event(connection_id, endpoint_event) - .await; - })); - continue 'send_pending; - } - drop(span); - // Timeout system. // We break out of the following loop until if `poll_timeout()` returns `None` or if // polling `self.next_timeout` returns `Poll::Pending`. @@ -362,6 +358,22 @@ impl Connection { } } + // The connection also needs to be able to send control messages to the endpoint. This is + // handled here, and we try to send them on `to_endpoint` as well. + let span = tracing::trace_span!("connection.poll_endpoint_events").entered(); + while let Some(endpoint_event) = self.connection.poll_endpoint_events() { + let endpoint = self.endpoint.clone(); + let connection_id = self.connection_id; + debug_assert!(self.pending_to_endpoint.is_none()); + self.pending_to_endpoint = Some(Box::pin(async move { + endpoint + .report_quinn_event(connection_id, endpoint_event) + .await; + })); + continue 'send_pending; + } + drop(span); + // The final step consists in handling the events related to the various substreams. let _span = tracing::trace_span!("connection.poll").entered(); while let Some(event) = self.connection.poll() { @@ -389,7 +401,7 @@ impl Connection { }) => { // The `Stop` QUIC event is more or less similar to a `Reset`, except that // it applies only on the writing side of the pipe. - tracing::error!("stream stopped {}", id); + tracing::error!("StreamEvent::Stopped {}", id); return Poll::Ready(ConnectionEvent::StreamStopped(id)); } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { @@ -403,16 +415,20 @@ impl Connection { return Poll::Ready(ConnectionEvent::StreamOpened); } quinn_proto::Event::ConnectionLost { reason } => { + tracing::error!("connection poll should close connection"); tracing::error!("connection lost {}", reason); debug_assert!(self.closed.is_none()); self.is_handshaking = false; let err = Error::Quinn(reason); self.closed = Some(err.clone()); + // self.connection + // .close(Instant::now(), From::from(0u32), Default::default()); return Poll::Ready(ConnectionEvent::ConnectionLost(err)); } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id, }) => { + tracing::error!("StreamEvent::StreamFinished {}", id); return Poll::Ready(ConnectionEvent::StreamFinished(id)); } quinn_proto::Event::Connected => { @@ -444,11 +460,15 @@ impl fmt::Debug for Connection { impl Drop for Connection { fn drop(&mut self) { + let is_closed = self.connection.is_closed(); + let is_drained = self.connection.is_drained(); + if !is_drained { // TODO: // if let Some(_) = self.closed.take() { // We send a `Drained` message to the endpoint to clean endpoint's resources. - self.endpoint.report_quinn_event_non_block(self.connection_id, quinn_proto::EndpointEvent::drained()); + self.close(); // } + } } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 2e54f2aa17a..f66f7547ed4 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -528,7 +528,14 @@ async fn background_task( // A connection wants to notify the endpoint of something. Some(ToEndpoint::ProcessConnectionEvent { connection_id, event }) => { - debug_assert!(alive_connections.contains_key(&connection_id)); + let has_key = alive_connections.contains_key(&connection_id); + tracing::error!( + has_key, + is_drained_event = event.is_drained()); + if !has_key { + continue; + } + //debug_assert!(alive_connections.contains_key(&connection_id)); // We "drained" event indicates that the connection no longer exists and // its ID can be reclaimed. let is_drained_event = event.is_drained(); diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 96b13eb0a45..8224501f154 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -54,7 +54,7 @@ struct QuicMuxerInner { } /// State of a single substream. -#[derive(Default)] +#[derive(Default, Clone)] struct SubstreamState { /// Waker to wake if the substream becomes readable or stopped. read_waker: Option, @@ -107,15 +107,30 @@ impl StreamMuxer for QuicMuxer { let _enter = span.entered(); while let Poll::Ready(event) = inner.connection.poll_event(cx) { - tracing::info!(?event); + // tracing::info!(?event); match event { ConnectionEvent::Connected => { tracing::error!("Unexpected Connected event on established QUIC connection"); } ConnectionEvent::ConnectionLost(e) => { + tracing::error!(?e, "ConnectionLost"); + // for (_, substream) in inner.substreams.iter_mut() { + // substream.finished = true; + // // } + // if let Some(waker) = substream.read_waker.take() { + // waker.wake(); + // } + // if let Some(waker) = substream.write_waker.take() { + // waker.wake(); + // } + // if let Some(waker) = substream.finished_waker.take() { + // waker.wake(); + // } + // } if let Some(waker) = inner.poll_close_waker.take() { waker.wake(); } + inner.connection.close(); return Poll::Ready(Err(Error::ConnectionLost(e))) } @@ -140,10 +155,11 @@ impl StreamMuxer for QuicMuxer { } ConnectionEvent::StreamFinished(substream) | ConnectionEvent::StreamStopped(substream) => { + tracing::info!(?event); if let Some(substream) = inner.substreams.get_mut(&substream) { - if let ConnectionEvent::StreamFinished(_) = event { + // if let ConnectionEvent::StreamFinished(_) = event { substream.finished = true; - } + // } if let Some(waker) = substream.read_waker.take() { waker.wake(); } @@ -158,7 +174,7 @@ impl StreamMuxer for QuicMuxer { } } ConnectionEvent::StreamAvailable => { - tracing::info!("StreamAvailable"); + tracing::info!(?event); // Handled below. } } @@ -169,7 +185,7 @@ impl StreamMuxer for QuicMuxer { inner.substreams.insert(substream, Default::default()); Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) } else { - tracing::info!("set poll_event_waker"); + //tracing::info!("set poll_event_waker"); inner.poll_event_waker = Some(cx.waker().clone()); Poll::Pending } @@ -177,7 +193,7 @@ impl StreamMuxer for QuicMuxer { #[tracing::instrument(skip_all)] fn open_outbound(&self) -> Self::OutboundSubstream { - tracing::info!("open_outbound"); + tracing::error!("open_outbound"); () } @@ -214,7 +230,7 @@ impl StreamMuxer for QuicMuxer { fn destroy_outbound(&self, _: Self::OutboundSubstream) {} - #[tracing::instrument(skip_all)] + #[tracing::instrument(skip(self, cx, buf), ret)] fn write_substream( &self, cx: &mut Context<'_>, @@ -226,7 +242,7 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); let side = inner.connection.connection.side(); - tracing::info!(?side, ?substream, "write_substream"); + //tracing::info!(?side, ?substream, "write_substream"); let id = substream; @@ -274,7 +290,7 @@ impl StreamMuxer for QuicMuxer { // } } - #[tracing::instrument(skip_all)] + #[tracing::instrument(skip(self, cx, buf), ret)] fn read_substream( &self, cx: &mut Context<'_>, @@ -320,14 +336,23 @@ impl StreamMuxer for QuicMuxer { let id = *substream; let mut inner = self.inner.lock(); + let side = inner.connection.connection.side(); - tracing::info!(?side, ?id, "read_substream"); + //tracing::info!(?side, ?id, "read_substream"); + + let substream_state = inner.substreams.get_mut(substream) + .expect("invalid StreamMuxer::read_substream API usage"); + if substream_state.finished { + return Poll::Ready(Ok(0)) + } + let mut stream = inner.connection.connection.recv_stream(id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(ReadableError::UnknownStream) => { tracing::error!("read error UnknownStream: substream={}", id); - return Poll::Ready(Err(Self::Error::ExpiredStream)) + return Poll::Ready(Ok(0)) + // return Poll::Ready(Err(Self::Error::ExpiredStream)) } Err(ReadableError::IllegalOrderedRead) => { panic!("Illegal ordered read can only happen if `stream.read(false)` is used."); @@ -344,19 +369,24 @@ impl StreamMuxer for QuicMuxer { buf.write_all(&chunk.bytes).expect("enough buffer space"); bytes += chunk.bytes.len(); } - Ok(None) => break, + Ok(None) => { + tracing::error!("No more data"); + break + }, Err(ReadError::Reset(error_code)) => { - tracing::debug!("substream {} was reset with error code {}", id, error_code); + tracing::error!("substream {} was reset with error code {}", id, error_code); bytes = 0; break; } Err(ReadError::Blocked) => { + tracing::error!("Blocked"); pending = true; break; } } } if chunks.finalize().should_transmit() { + tracing::error!("read should transmit"); if let Some(waker) = inner.poll_event_waker.take() { waker.wake(); } @@ -370,7 +400,7 @@ impl StreamMuxer for QuicMuxer { } } - #[tracing::instrument(skip_all)] + #[tracing::instrument(skip(self, cx), ret)] fn shutdown_substream( &self, cx: &mut Context<'_>, @@ -378,7 +408,7 @@ impl StreamMuxer for QuicMuxer { ) -> Poll> { let mut inner = self.inner.lock(); let inner = &mut *inner; - tracing::info!(?inner.connection.connection, "shutdown_substream"); + tracing::error!(?inner.connection.connection, "shutdown_substream"); let mut substream_state = inner.substreams.get_mut(substream) .expect("invalid StreamMuxer::shutdown_substream API usage"); @@ -388,6 +418,7 @@ impl StreamMuxer for QuicMuxer { match inner.connection.shutdown_substream(*substream) { Ok(()) => { + substream_state.finished = true; match substream_state.finished_waker { Some(ref w) if w.will_wake(cx.waker()) => {}, _ => substream_state.finished_waker = Some(cx.waker().clone()), @@ -403,9 +434,27 @@ impl StreamMuxer for QuicMuxer { } } + #[tracing::instrument(skip(self), ret)] fn destroy_substream(&self, substream: Self::Substream) { + tracing::error!(?substream, "destroy_substream"); let mut inner = self.inner.lock(); - inner.substreams.remove(&substream); + if let Some(mut substream) = inner.substreams.remove(&substream) { + // // if let ConnectionEvent::StreamFinished(_) = event { + // substream.finished = true; + // // } + // if let Some(waker) = substream.read_waker.take() { + // tracing::error!("read_waker"); + // waker.wake(); + // } + // if let Some(waker) = substream.write_waker.take() { + // tracing::error!("write_waker"); + // waker.wake(); + // } + // if let Some(waker) = substream.finished_waker.take() { + // tracing::error!("finished_waker"); + // waker.wake(); + // } + } } fn flush_substream( @@ -417,14 +466,34 @@ impl StreamMuxer for QuicMuxer { } // TODO: what if called multiple times? register all wakers? + #[tracing::instrument(skip_all, ret)] fn flush_all(&self, _cx: &mut Context<'_>) -> Poll> { - // TODO: call poll_transmit() and stuff - Poll::Ready(Ok(())) + return Poll::Ready(Ok(())); + let mut inner = self.inner.lock(); + let inner = &mut *inner; + if inner.substreams.is_empty() { + return Poll::Ready(Ok(())) + } + for substream in inner.substreams.keys() { + match inner.connection.shutdown_substream(*substream) { + Ok(()) => { + }, + Err(quinn_proto::FinishError::Stopped(err)) => return Poll::Ready(Err(Error::Reset(err))), + Err(quinn_proto::FinishError::UnknownStream) => { + // Illegal usage of the API. + debug_assert!(false); + return Poll::Ready(Err(Error::ExpiredStream)) + }, + } + } + return Poll::Pending } // TODO: what if called multiple times? register all wakers? - #[tracing::instrument(skip_all)] + #[tracing::instrument(skip_all, ret)] fn close(&self, cx: &mut Context<'_>) -> Poll> { + // return Poll::Ready(Ok(())); + tracing::error!("muxer_close"); // StreamMuxer's `close` documentation mentions that it automatically implies `flush_all`. if let Poll::Pending = self.flush_all(cx)? { return Poll::Pending; @@ -433,7 +502,41 @@ impl StreamMuxer for QuicMuxer { // TODO: poll if closed or something let mut inner = self.inner.lock(); - //self.connection.close(); + + // if inner.connection.connection.is_drained() { + // return Poll::Ready(Ok(())) + // } + + // if inner.substreams.is_empty() { + // let connection = &mut inner.connection; + // if !connection.connection.is_closed() { + // connection.close(); + // if let Some(waker) = inner.poll_event_waker.take() { + // waker.wake(); + // } + // } else { + + // } + // tracing::error!( + // is_closed = inner.connection.connection.is_closed(), + // is_drained = inner.connection.connection.is_drained() + // ); + // while let Poll::Ready(event) = inner.connection.poll_event(cx) { + // match event { + // ConnectionEvent::ConnectionLost(_) => { + // return Poll::Ready(Ok(())) + // }, + // _ => { + + // } + // } + // } + // // return Poll::Ready(Ok(())) + // } else { + // for substream in inner.substreams.clone().keys() { + // inner.connection.shutdown_substream(*substream); + // } + // } // Register `cx.waker()` as being woken up if the connection closes. if !inner diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index a46dfac98cc..dcbcaff0d68 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -8,13 +8,17 @@ use libp2p::request_response::{ ProtocolName, ProtocolSupport, RequestResponse, RequestResponseCodec, RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, }; -use libp2p::swarm::{Swarm, SwarmEvent}; +use libp2p::swarm::{Swarm, SwarmBuilder, SwarmEvent}; use libp2p_core::muxing::StreamMuxerBox; use libp2p::{Multiaddr, Transport}; use libp2p_quic::{Config as QuicConfig, Endpoint as QuicEndpoint, QuicTransport}; use rand::RngCore; use std::{io, iter}; +use futures::task::Spawn; +use std::num::NonZeroU8; +use std::time::Duration; + fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } @@ -315,3 +319,154 @@ async fn dial_failure() -> Result<()> { Ok(()) } + +#[test] +fn concurrent_connections_and_streams() { + use futures::executor::block_on; + use quickcheck::*; + + tracing_subscriber::fmt() + //.pretty() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .try_init() + .ok(); + log_panics::init(); + + #[tracing::instrument] + fn prop(number_listeners: NonZeroU8, number_streams: NonZeroU8) -> TestResult { + tracing::info!("entered"); + let (number_listeners, number_streams): (u8, u8) = + (number_listeners.into(), number_streams.into()); + if number_listeners > 10 || number_streams > 10 { + return TestResult::discard(); + } + + let mut pool = futures::executor::LocalPool::default(); + let mut data = vec![0; 4096 * 10]; + rand::thread_rng().fill_bytes(&mut data); + let mut listeners = vec![]; + + // Spawn the listener nodes. + for _ in 0..number_listeners { + let mut listener = block_on(create_swarm(true)).unwrap(); + Swarm::listen_on(&mut listener, "/ip4/127.0.0.1/udp/0/quic".parse().unwrap()).unwrap(); + + // Wait to listen on address. + let addr = match block_on(listener.next()) { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + }; + + listeners.push((*listener.local_peer_id(), addr)); + + pool.spawner() + .spawn_obj( + async move { + loop { + match listener.next().await { + Some(SwarmEvent::ConnectionEstablished { .. }) => { + tracing::info!("listener ConnectionEstablished"); + } + Some(SwarmEvent::IncomingConnection { .. }) => { + tracing::info!("listener IncomingConnection"); + } + Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { + message: + RequestResponseMessage::Request { + request: Ping(ping), + channel, + .. + }, + .. + })) => { + tracing::info!("listener got Message"); + listener + .behaviour_mut() + .send_response(channel, Pong(ping)) + .unwrap(); + } + Some(SwarmEvent::Behaviour( + RequestResponseEvent::ResponseSent { .. }, + )) => { + tracing::info!("listener ResponseSent"); + } + Some(SwarmEvent::ConnectionClosed { .. }) => {} + Some(e) => { + tracing::info!(?e, "listener"); + } // panic!("{:?}", e), + None => { + panic!("listener stopped"); + } + } + } + } + .boxed() + .into(), + ) + .unwrap(); + } + + let mut dialer = block_on(create_swarm(true)).unwrap(); + + // For each listener node start `number_streams` requests. + for (listener_peer_id, listener_addr) in &listeners { + dialer + .behaviour_mut() + .add_address(&listener_peer_id, listener_addr.clone()); + + dialer.dial(listener_peer_id.clone()).unwrap(); + //dialer + // .behaviour_mut() + // .send_request(&listener_peer_id, Ping(data.clone())); + } + + // Wait for responses to each request. + pool.run_until(async { + let mut num_responses = 0; + loop { + match dialer.next().await { + Some(SwarmEvent::Dialing(_)) => { + tracing::info!("dialer Dialing"); + } + Some(SwarmEvent::ConnectionEstablished { peer_id, .. }) => { + tracing::info!("dialer Connection established"); + for _ in 0..number_streams { + dialer + .behaviour_mut() + .send_request(&peer_id, Ping(data.clone())); + } + } + Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { + message: + RequestResponseMessage::Response { + response: Pong(pong), + .. + }, + .. + })) => { + tracing::info!("dialer got Message"); + num_responses += 1; + assert_eq!(data, pong); + let should_be = number_listeners as usize * (number_streams) as usize; + tracing::info!(?num_responses, ?should_be); + if num_responses == should_be { + break; + } + } + Some(SwarmEvent::ConnectionClosed { .. }) => { + tracing::info!("dialer ConnectionClosed"); + } + e => { + tracing::info!(?e, "dialer"); + } // panic!("{:?}", e), + } + } + }); + + TestResult::passed() + } + + prop(NonZeroU8::new(1).unwrap(), NonZeroU8::new(2).unwrap()); + + // QuickCheck::new().quickcheck(prop as fn(_, _) -> _); +} From 4c4c5f39f3e9637dbe8ce190489a658291ab6e8f Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 6 Apr 2022 23:04:12 +0300 Subject: [PATCH 042/218] close muxer from the client side --- transports/quic/src/connection.rs | 1 + transports/quic/src/muxer.rs | 74 +++++++++++++++---------------- transports/quic/tests/smoke.rs | 4 +- 3 files changed, 39 insertions(+), 40 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 0fe09b44d35..2fdc00eeff9 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -421,6 +421,7 @@ impl Connection { self.is_handshaking = false; let err = Error::Quinn(reason); self.closed = Some(err.clone()); + // self.close(); // self.connection // .close(Instant::now(), From::from(0u32), Default::default()); return Poll::Ready(ConnectionEvent::ConnectionLost(err)); diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 8224501f154..1497de53f61 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -131,7 +131,7 @@ impl StreamMuxer for QuicMuxer { waker.wake(); } inner.connection.close(); - return Poll::Ready(Err(Error::ConnectionLost(e))) + // return Poll::Ready(Err(Error::ConnectionLost(e))) } ConnectionEvent::StreamOpened => { @@ -340,11 +340,11 @@ impl StreamMuxer for QuicMuxer { let side = inner.connection.connection.side(); //tracing::info!(?side, ?id, "read_substream"); - let substream_state = inner.substreams.get_mut(substream) - .expect("invalid StreamMuxer::read_substream API usage"); - if substream_state.finished { - return Poll::Ready(Ok(0)) - } + // let substream_state = inner.substreams.get_mut(substream) + // .expect("invalid StreamMuxer::read_substream API usage"); + // if substream_state.finished { + // return Poll::Ready(Ok(0)) + // } let mut stream = inner.connection.connection.recv_stream(id); let mut chunks = match stream.read(true) { @@ -503,40 +503,40 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); - // if inner.connection.connection.is_drained() { - // return Poll::Ready(Ok(())) - // } + if inner.connection.connection.is_drained() { + return Poll::Ready(Ok(())) + } - // if inner.substreams.is_empty() { - // let connection = &mut inner.connection; - // if !connection.connection.is_closed() { - // connection.close(); - // if let Some(waker) = inner.poll_event_waker.take() { - // waker.wake(); - // } - // } else { + if inner.substreams.is_empty() { + let connection = &mut inner.connection; + if !connection.connection.is_closed() { + connection.close(); + if let Some(waker) = inner.poll_event_waker.take() { + waker.wake(); + } + } else { - // } - // tracing::error!( - // is_closed = inner.connection.connection.is_closed(), - // is_drained = inner.connection.connection.is_drained() - // ); - // while let Poll::Ready(event) = inner.connection.poll_event(cx) { - // match event { - // ConnectionEvent::ConnectionLost(_) => { - // return Poll::Ready(Ok(())) - // }, - // _ => { + } + tracing::error!( + is_closed = inner.connection.connection.is_closed(), + is_drained = inner.connection.connection.is_drained() + ); + while let Poll::Ready(event) = inner.connection.poll_event(cx) { + match event { + ConnectionEvent::ConnectionLost(_) => { + return Poll::Ready(Ok(())) + }, + _ => { - // } - // } - // } - // // return Poll::Ready(Ok(())) - // } else { - // for substream in inner.substreams.clone().keys() { - // inner.connection.shutdown_substream(*substream); - // } - // } + } + } + } + // return Poll::Ready(Ok(())) + } else { + for substream in inner.substreams.clone().keys() { + inner.connection.shutdown_substream(*substream); + } + } // Register `cx.waker()` as being woken up if the connection closes. if !inner diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index dcbcaff0d68..f0e0826e8e6 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -466,7 +466,5 @@ fn concurrent_connections_and_streams() { TestResult::passed() } - prop(NonZeroU8::new(1).unwrap(), NonZeroU8::new(2).unwrap()); - - // QuickCheck::new().quickcheck(prop as fn(_, _) -> _); + QuickCheck::new().quickcheck(prop as fn(_, _) -> _); } From ebaf5acd8cda9e1cf4b114ebb72a0e0e69b05dd1 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 6 Apr 2022 23:26:02 +0300 Subject: [PATCH 043/218] cargo fmt --- transports/quic/src/connection.rs | 34 +++++++++------- transports/quic/src/endpoint.rs | 31 ++++++++------- transports/quic/src/lib.rs | 2 +- transports/quic/src/muxer.rs | 65 ++++++++++++++++--------------- transports/quic/src/upgrade.rs | 6 ++- transports/quic/tests/smoke.rs | 11 +++--- 6 files changed, 80 insertions(+), 69 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 2fdc00eeff9..bba301cbe88 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -29,6 +29,7 @@ use crate::endpoint::Endpoint; use futures::{channel::mpsc, prelude::*}; +use libp2p_core::PeerId; use std::{ fmt, net::SocketAddr, @@ -37,7 +38,6 @@ use std::{ task::{Context, Poll}, time::Instant, }; -use libp2p_core::PeerId; /// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Upgrade`]. /// @@ -127,7 +127,8 @@ impl Connection { pub(crate) fn local_addr(&self) -> SocketAddr { debug_assert_eq!(self.connection.side(), quinn_proto::Side::Server); let endpoint_addr = self.endpoint.local_addr; - self.connection.local_ip() + self.connection + .local_ip() .map(|ip| SocketAddr::new(ip, endpoint_addr.port())) .unwrap_or_else(|| { // In a normal case scenario this should not happen, because @@ -155,11 +156,15 @@ impl Connection { pub(crate) fn remote_peer_id(&self) -> PeerId { debug_assert!(!self.is_handshaking()); let session = self.connection.crypto_session(); - let identity = session.peer_identity() + let identity = session + .peer_identity() .expect("connection got identity because it passed TLS handshake; qed"); - let certificates: Box> = identity.downcast().ok() + let certificates: Box> = identity + .downcast() + .ok() .expect("we rely on rustls feature; qed"); - let end_entity = certificates.get(0) + let end_entity = certificates + .get(0) .expect("there should be exactly one certificate; qed"); let end_entity_der = end_entity.as_ref(); let p2p_cert = crate::tls::certificate::parse_certificate(end_entity_der) @@ -183,7 +188,10 @@ impl Connection { // support this. self.connection .close(Instant::now(), From::from(0u32), Default::default()); - self.endpoint.report_quinn_event_non_block(self.connection_id, quinn_proto::EndpointEvent::drained()); + self.endpoint.report_quinn_event_non_block( + self.connection_id, + quinn_proto::EndpointEvent::drained(), + ); } /// Pops a new substream opened by the remote. @@ -281,7 +289,7 @@ impl Connection { Poll::Ready(Some(event)) => { let _span = tracing::trace_span!("handle").entered(); self.connection.handle_event(event) - }, + } Poll::Ready(None) => { tracing::error!("connection handle event should close connection"); debug_assert!(self.closed.is_none()); @@ -388,7 +396,7 @@ impl Connection { // We don't use datagrams or unidirectional streams. If these events // happen, it is by some code not compatible with libp2p-quic. self.connection - .close(Instant::now(), From::from(0u32), Default::default()); + .close(Instant::now(), From::from(0u32), Default::default()); } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { return Poll::Ready(ConnectionEvent::StreamReadable(id)); @@ -426,9 +434,7 @@ impl Connection { // .close(Instant::now(), From::from(0u32), Default::default()); return Poll::Ready(ConnectionEvent::ConnectionLost(err)); } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { - id, - }) => { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { tracing::error!("StreamEvent::StreamFinished {}", id); return Poll::Ready(ConnectionEvent::StreamFinished(id)); } @@ -464,11 +470,11 @@ impl Drop for Connection { let is_closed = self.connection.is_closed(); let is_drained = self.connection.is_drained(); if !is_drained { - // TODO: - // if let Some(_) = self.closed.take() { + // TODO: + // if let Some(_) = self.closed.take() { // We send a `Drained` message to the endpoint to clean endpoint's resources. self.close(); - // } + // } } } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index f66f7547ed4..7c0122b69c1 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -38,6 +38,7 @@ use futures::{ stream::Stream, }; use libp2p_core::multiaddr::Multiaddr; +use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; use std::{ collections::{HashMap, VecDeque}, fmt, io, @@ -46,10 +47,6 @@ use std::{ task::{Context, Poll}, time::{Duration, Instant}, }; -use quinn_proto::{ - ClientConfig as QuinnClientConfig, - ServerConfig as QuinnServerConfig, -}; /// Represents the configuration for the [`Endpoint`]. #[derive(Debug, Clone)] @@ -71,7 +68,7 @@ impl Config { multiaddr: Multiaddr, ) -> Result { let mut transport = quinn_proto::TransportConfig::default(); - transport.max_concurrent_uni_streams(0u32.into()); // Can only panic if value is out of range. + transport.max_concurrent_uni_streams(0u32.into()); // Can only panic if value is out of range. transport.datagram_receive_buffer_size(None); transport.keep_alive_interval(Some(Duration::from_millis(10))); let transport = Arc::new(transport); @@ -229,7 +226,8 @@ impl Endpoint { // reasonable thing to do. let (tx, rx) = oneshot::channel(); self.to_endpoint - .lock().await + .lock() + .await .send(ToEndpoint::Dial { addr, result: tx }) .await .expect("background task has crashed"); @@ -250,14 +248,11 @@ impl Endpoint { /// There is no guarantee that the packet will actually be sent, but considering that this is /// a UDP packet, you cannot rely on the packet being delivered anyway. #[tracing::instrument(skip_all)] - pub(crate) async fn send_udp_packet( - &self, - destination: SocketAddr, - data: impl Into>, - ) { + pub(crate) async fn send_udp_packet(&self, destination: SocketAddr, data: impl Into>) { let _ = self .to_endpoint - .lock().await + .lock() + .await .send(ToEndpoint::SendUdpPacket { destination, data: data.into(), @@ -281,7 +276,8 @@ impl Endpoint { tracing::error!("drained, {:?}", connection_id); } self.to_endpoint - .lock().await + .lock() + .await .send(ToEndpoint::ProcessConnectionEvent { connection_id, event, @@ -306,7 +302,8 @@ impl Endpoint { } // We implement this by cloning the `mpsc::Sender`. Since each sender is guaranteed a slot // in the buffer, cloning the sender reserves the slot and sending thus always succeeds. - let result = self.to_endpoint2 + let result = self + .to_endpoint2 .clone() .try_send(ToEndpoint::ProcessConnectionEvent { connection_id, @@ -467,7 +464,11 @@ async fn background_task( // channels. // TODO: set ECN bits; there is no support for them in the ecosystem right now let span = tracing::trace_span!("udp_socket.send_to"); - match udp_socket.send_to(&data, destination).instrument(span).await { + match udp_socket + .send_to(&data, destination) + .instrument(span) + .await + { Ok(n) if n == data.len() => {} Ok(_) => tracing::error!( "QUIC UDP socket violated expectation that packets are always fully \ diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 4994c2a2437..8a5c929c7ed 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -58,8 +58,8 @@ mod endpoint; mod error; mod in_addr; mod muxer; -mod upgrade; mod tls; +mod upgrade; pub mod transport; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 1497de53f61..28780ad02db 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -94,7 +94,10 @@ impl StreamMuxer for QuicMuxer { // TODO: what if called multiple times? register all wakers? #[tracing::instrument(skip_all)] - fn poll_event(&self, cx: &mut Context<'_>) -> Poll, Self::Error>> { + fn poll_event( + &self, + cx: &mut Context<'_>, + ) -> Poll, Self::Error>> { //tracing::info!("here"); // We use `poll_inbound` to perform the background processing of the entire connection. let mut inner = self.inner.lock(); @@ -153,12 +156,12 @@ impl StreamMuxer for QuicMuxer { } } } - ConnectionEvent::StreamFinished(substream) | - ConnectionEvent::StreamStopped(substream) => { + ConnectionEvent::StreamFinished(substream) + | ConnectionEvent::StreamStopped(substream) => { tracing::info!(?event); if let Some(substream) = inner.substreams.get_mut(&substream) { // if let ConnectionEvent::StreamFinished(_) = event { - substream.finished = true; + substream.finished = true; // } if let Some(waker) = substream.read_waker.take() { waker.wake(); @@ -237,7 +240,7 @@ impl StreamMuxer for QuicMuxer { substream: &mut Self::Substream, buf: &[u8], ) -> Poll> { - use quinn_proto::{WriteError}; + use quinn_proto::WriteError; let mut inner = self.inner.lock(); @@ -256,7 +259,7 @@ impl StreamMuxer for QuicMuxer { Err(WriteError::Stopped(_)) => { tracing::error!("write error Stopped: substream={}", id); Poll::Ready(Ok(0)) - }, + } Err(WriteError::UnknownStream) => { tracing::error!("write error UnknownStream substream={}", id); Poll::Ready(Err(Self::Error::ExpiredStream)) @@ -330,7 +333,7 @@ impl StreamMuxer for QuicMuxer { // } // } - use quinn_proto::{ReadableError, ReadError}; + use quinn_proto::{ReadError, ReadableError}; use std::io::Write; let id = *substream; @@ -345,13 +348,13 @@ impl StreamMuxer for QuicMuxer { // if substream_state.finished { // return Poll::Ready(Ok(0)) // } - + let mut stream = inner.connection.connection.recv_stream(id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(ReadableError::UnknownStream) => { tracing::error!("read error UnknownStream: substream={}", id); - return Poll::Ready(Ok(0)) + return Poll::Ready(Ok(0)); // return Poll::Ready(Err(Self::Error::ExpiredStream)) } Err(ReadableError::IllegalOrderedRead) => { @@ -371,8 +374,8 @@ impl StreamMuxer for QuicMuxer { } Ok(None) => { tracing::error!("No more data"); - break - }, + break; + } Err(ReadError::Reset(error_code)) => { tracing::error!("substream {} was reset with error code {}", id, error_code); bytes = 0; @@ -410,27 +413,29 @@ impl StreamMuxer for QuicMuxer { let inner = &mut *inner; tracing::error!(?inner.connection.connection, "shutdown_substream"); - let mut substream_state = inner.substreams.get_mut(substream) + let mut substream_state = inner + .substreams + .get_mut(substream) .expect("invalid StreamMuxer::shutdown_substream API usage"); if substream_state.finished { - return Poll::Ready(Ok(())) + return Poll::Ready(Ok(())); } match inner.connection.shutdown_substream(*substream) { Ok(()) => { substream_state.finished = true; match substream_state.finished_waker { - Some(ref w) if w.will_wake(cx.waker()) => {}, + Some(ref w) if w.will_wake(cx.waker()) => {} _ => substream_state.finished_waker = Some(cx.waker().clone()), } Poll::Pending - }, + } Err(quinn_proto::FinishError::Stopped(err)) => Poll::Ready(Err(Error::Reset(err))), Err(quinn_proto::FinishError::UnknownStream) => { // Illegal usage of the API. debug_assert!(false); Poll::Ready(Err(Error::ExpiredStream)) - }, + } } } @@ -472,21 +477,22 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); let inner = &mut *inner; if inner.substreams.is_empty() { - return Poll::Ready(Ok(())) + return Poll::Ready(Ok(())); } for substream in inner.substreams.keys() { match inner.connection.shutdown_substream(*substream) { - Ok(()) => { - }, - Err(quinn_proto::FinishError::Stopped(err)) => return Poll::Ready(Err(Error::Reset(err))), + Ok(()) => {} + Err(quinn_proto::FinishError::Stopped(err)) => { + return Poll::Ready(Err(Error::Reset(err))) + } Err(quinn_proto::FinishError::UnknownStream) => { // Illegal usage of the API. debug_assert!(false); - return Poll::Ready(Err(Error::ExpiredStream)) - }, + return Poll::Ready(Err(Error::ExpiredStream)); + } } } - return Poll::Pending + return Poll::Pending; } // TODO: what if called multiple times? register all wakers? @@ -504,9 +510,9 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); if inner.connection.connection.is_drained() { - return Poll::Ready(Ok(())) + return Poll::Ready(Ok(())); } - + if inner.substreams.is_empty() { let connection = &mut inner.connection; if !connection.connection.is_closed() { @@ -515,7 +521,6 @@ impl StreamMuxer for QuicMuxer { waker.wake(); } } else { - } tracing::error!( is_closed = inner.connection.connection.is_closed(), @@ -523,12 +528,8 @@ impl StreamMuxer for QuicMuxer { ); while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { - ConnectionEvent::ConnectionLost(_) => { - return Poll::Ready(Ok(())) - }, - _ => { - - } + ConnectionEvent::ConnectionLost(_) => return Poll::Ready(Ok(())), + _ => {} } } // return Poll::Ready(Ok(())) diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 3207d86e124..cf329dd2332 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -53,7 +53,9 @@ impl Future for Upgrade { #[tracing::instrument(skip_all)] fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let connection = self.connection.as_mut() + let connection = self + .connection + .as_mut() .expect("Future polled after it has completed"); let event = Connection::poll_event(connection, cx); @@ -80,7 +82,7 @@ impl Future for Upgrade { // But for `Upgrade` we get `Connected` event, wrap connection into a muxer // and pass it to the result Stream of muxers. unreachable!() - }, + } } } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index f0e0826e8e6..70fc5bcd07e 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -9,8 +9,8 @@ use libp2p::request_response::{ RequestResponseEvent, RequestResponseMessage, }; use libp2p::swarm::{Swarm, SwarmBuilder, SwarmEvent}; -use libp2p_core::muxing::StreamMuxerBox; use libp2p::{Multiaddr, Transport}; +use libp2p_core::muxing::StreamMuxerBox; use libp2p_quic::{Config as QuicConfig, Endpoint as QuicEndpoint, QuicTransport}; use rand::RngCore; use std::{io, iter}; @@ -40,9 +40,10 @@ async fn create_swarm(keylog: bool) -> Result>> // transport.enable_keylogger(); // } - let transport = - Transport::map(transport, |(peer, muxer), _| (peer, StreamMuxerBox::new(muxer))) - .boxed(); + let transport = Transport::map(transport, |(peer, muxer), _| { + (peer, StreamMuxerBox::new(muxer)) + }) + .boxed(); let protocols = iter::once((PingProtocol(), ProtocolSupport::Full)); let cfg = RequestResponseConfig::default(); @@ -53,7 +54,7 @@ async fn create_swarm(keylog: bool) -> Result>> fn setup_global_subscriber() -> impl Drop { use tracing_flame::FlameLayer; - use tracing_subscriber::{prelude::*, fmt}; + use tracing_subscriber::{fmt, prelude::*}; let filter_layer = tracing_subscriber::EnvFilter::from_default_env(); From d6228ca250d76c374fccb74507d1c98acd5a502f Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 7 Apr 2022 15:15:07 +0300 Subject: [PATCH 044/218] Prettify code and get rid of tracings --- transports/quic/src/connection.rs | 34 ----- transports/quic/src/endpoint.rs | 35 +----- transports/quic/src/muxer.rs | 202 +++++++----------------------- transports/quic/src/upgrade.rs | 2 - 4 files changed, 46 insertions(+), 227 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index bba301cbe88..c410826b927 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -99,7 +99,6 @@ impl Connection { /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of /// its methods has ever been called. Failure to comply might lead to logic errors and panics. // TODO: maybe abstract `to_endpoint` more and make it generic? dunno - #[tracing::instrument(skip_all)] pub(crate) fn from_quinn_connection( endpoint: Arc, connection: quinn_proto::Connection, @@ -152,7 +151,6 @@ impl Connection { /// Returns the address of the node we're connected to. /// Panics if the connection is still handshaking. - #[tracing::instrument(skip_all)] pub(crate) fn remote_peer_id(&self) -> PeerId { debug_assert!(!self.is_handshaking()); let session = self.connection.crypto_session(); @@ -263,18 +261,7 @@ impl Connection { self.connection.send_stream(id).finish() } - // pub(crate) fn flush_substream( - // &mut self, - // id: quinn_proto::StreamId, - // ) -> Result<(), quinn_proto::FinishError> { - // let mut substream = self.connection.send_stream(id); - // if let Ok(_) = substream.stopped() { - - // } - // } - /// Polls the connection for an event that happend on it. - #[tracing::instrument] pub(crate) fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { // Nothing more can be done if the connection is closed. // Return `Pending` without registering the waker, essentially freezing the task forever. @@ -283,15 +270,12 @@ impl Connection { } // Process events that the endpoint has sent to us. - let span = tracing::trace_span!("connection.handle_event").entered(); loop { match Pin::new(&mut self.from_endpoint).poll_next(cx) { Poll::Ready(Some(event)) => { - let _span = tracing::trace_span!("handle").entered(); self.connection.handle_event(event) } Poll::Ready(None) => { - tracing::error!("connection handle event should close connection"); debug_assert!(self.closed.is_none()); let err = Error::ClosedChannel; self.closed = Some(err.clone()); @@ -300,7 +284,6 @@ impl Connection { Poll::Pending => break, } } - drop(span); 'send_pending: loop { // Sending the pending event to the endpoint. If the endpoint is too busy, we just @@ -313,21 +296,18 @@ impl Connection { // However we don't deliver substream-related events to the user as long as // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. - let span = tracing::trace_span!("connection.send_pending").entered(); if let Some(pending_to_endpoint) = &mut self.pending_to_endpoint { match Future::poll(Pin::new(pending_to_endpoint), cx) { Poll::Pending => return Poll::Pending, Poll::Ready(()) => self.pending_to_endpoint = None, } } - drop(span); let now = Instant::now(); // Poll the connection for packets to send on the UDP socket and try to send them on // `to_endpoint`. // FIXME max_datagrams - let span = tracing::trace_span!("connection.poll_transmit").entered(); while let Some(transmit) = self.connection.poll_transmit(now, 1) { let endpoint = self.endpoint.clone(); debug_assert!(self.pending_to_endpoint.is_none()); @@ -339,13 +319,11 @@ impl Connection { })); continue 'send_pending; } - drop(span); // Timeout system. // We break out of the following loop until if `poll_timeout()` returns `None` or if // polling `self.next_timeout` returns `Poll::Pending`. loop { - let _span = tracing::trace_span!("connection.timeout").entered(); if let Some(next_timeout) = &mut self.next_timeout { match Future::poll(Pin::new(next_timeout), cx) { Poll::Ready(()) => { @@ -368,7 +346,6 @@ impl Connection { // The connection also needs to be able to send control messages to the endpoint. This is // handled here, and we try to send them on `to_endpoint` as well. - let span = tracing::trace_span!("connection.poll_endpoint_events").entered(); while let Some(endpoint_event) = self.connection.poll_endpoint_events() { let endpoint = self.endpoint.clone(); let connection_id = self.connection_id; @@ -380,10 +357,8 @@ impl Connection { })); continue 'send_pending; } - drop(span); // The final step consists in handling the events related to the various substreams. - let _span = tracing::trace_span!("connection.poll").entered(); while let Some(event) = self.connection.poll() { match event { quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { @@ -409,7 +384,6 @@ impl Connection { }) => { // The `Stop` QUIC event is more or less similar to a `Reset`, except that // it applies only on the writing side of the pipe. - tracing::error!("StreamEvent::Stopped {}", id); return Poll::Ready(ConnectionEvent::StreamStopped(id)); } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { @@ -423,8 +397,6 @@ impl Connection { return Poll::Ready(ConnectionEvent::StreamOpened); } quinn_proto::Event::ConnectionLost { reason } => { - tracing::error!("connection poll should close connection"); - tracing::error!("connection lost {}", reason); debug_assert!(self.closed.is_none()); self.is_handshaking = false; let err = Error::Quinn(reason); @@ -435,7 +407,6 @@ impl Connection { return Poll::Ready(ConnectionEvent::ConnectionLost(err)); } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { - tracing::error!("StreamEvent::StreamFinished {}", id); return Poll::Ready(ConnectionEvent::StreamFinished(id)); } quinn_proto::Event::Connected => { @@ -467,14 +438,9 @@ impl fmt::Debug for Connection { impl Drop for Connection { fn drop(&mut self) { - let is_closed = self.connection.is_closed(); let is_drained = self.connection.is_drained(); if !is_drained { - // TODO: - // if let Some(_) = self.closed.take() { - // We send a `Drained` message to the endpoint to clean endpoint's resources. self.close(); - // } } } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 7c0122b69c1..ee8aef4f766 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -123,7 +123,6 @@ pub struct Endpoint { impl Endpoint { /// Builds a new `Endpoint`. - #[tracing::instrument(skip_all)] pub fn new(config: Config) -> Result, io::Error> { let local_socket_addr = match crate::transport::multiaddr_to_socketaddr(&config.multiaddr) { Some(a) => a, @@ -215,7 +214,6 @@ impl Endpoint { /// /// Note that this method only *starts* the dialing. `Ok` is returned as soon as possible, even /// when the remote might end up being unreachable. - #[tracing::instrument] pub(crate) async fn dial( &self, addr: SocketAddr, @@ -235,7 +233,6 @@ impl Endpoint { } /// Tries to pop a new incoming connection from the queue. - #[tracing::instrument] pub(crate) fn poll_incoming(&self, cx: &mut Context) -> Poll> { let mut connections_lock = self.new_connections.lock(); let mut guard = futures::ready!(Pin::new(&mut connections_lock).poll(cx)); @@ -247,7 +244,6 @@ impl Endpoint { /// Note that this method only queues the packet and returns as soon as the packet is in queue. /// There is no guarantee that the packet will actually be sent, but considering that this is /// a UDP packet, you cannot rely on the packet being delivered anyway. - #[tracing::instrument(skip_all)] pub(crate) async fn send_udp_packet(&self, destination: SocketAddr, data: impl Into>) { let _ = self .to_endpoint @@ -266,15 +262,11 @@ impl Endpoint { /// /// If `event.is_drained()` is true, the event indicates that the connection no longer exists. /// This must therefore be the last event sent using this [`quinn_proto::ConnectionHandle`]. - #[tracing::instrument(skip_all)] pub(crate) async fn report_quinn_event( &self, connection_id: quinn_proto::ConnectionHandle, event: quinn_proto::EndpointEvent, ) { - if event.is_drained() { - tracing::error!("drained, {:?}", connection_id); - } self.to_endpoint .lock() .await @@ -291,15 +283,11 @@ impl Endpoint { /// /// This method bypasses back-pressure mechanisms and is meant to be called only from /// destructors, where waiting is not advisable. - #[tracing::instrument(skip_all)] pub(crate) fn report_quinn_event_non_block( &self, connection_id: quinn_proto::ConnectionHandle, event: quinn_proto::EndpointEvent, ) { - if event.is_drained() { - tracing::error!("drained, {:?}", connection_id); - } // We implement this by cloning the `mpsc::Sender`. Since each sender is guaranteed a slot // in the buffer, cloning the sender reserves the slot and sending thus always succeeds. let result = self @@ -423,7 +411,6 @@ enum ToEndpoint { /// guarantees that the [`Endpoint`], and therefore the background task, is properly kept alive /// for as long as any QUIC connection is open. /// -#[tracing::instrument(skip_all)] async fn background_task( config: Config, endpoint_weak: Weak, @@ -463,10 +450,8 @@ async fn background_task( // network interface is too busy, we back-pressure all of our internal // channels. // TODO: set ECN bits; there is no support for them in the ecosystem right now - let span = tracing::trace_span!("udp_socket.send_to"); match udp_socket .send_to(&data, destination) - .instrument(span) .await { Ok(n) if n == data.len() => {} @@ -486,16 +471,13 @@ async fn background_task( // The endpoint might request packets to be sent out. This is handled in priority to avoid // buffering up packets. if let Some(packet) = endpoint.poll_transmit() { - tracing::trace!("endpoint.poll_transmit"); debug_assert!(next_packet_out.is_none()); next_packet_out = Some((packet.destination, packet.contents)); continue; } - use tracing::Instrument; - futures::select! { - message = receiver.next().instrument(tracing::trace_span!("ToEndpoint message")).fuse() => { + message = receiver.next().fuse() => { // Received a message from a different part of the code requesting us to // do something. match message { @@ -530,9 +512,6 @@ async fn background_task( // A connection wants to notify the endpoint of something. Some(ToEndpoint::ProcessConnectionEvent { connection_id, event }) => { let has_key = alive_connections.contains_key(&connection_id); - tracing::error!( - has_key, - is_drained_event = event.is_drained()); if !has_key { continue; } @@ -541,20 +520,17 @@ async fn background_task( // its ID can be reclaimed. let is_drained_event = event.is_drained(); if is_drained_event { - tracing::error!("ProcessConnectionEvent Drained : {:?}", connection_id); alive_connections.remove(&connection_id); } - tracing::trace!("endpoint.handle_event"); let event_back = endpoint.handle_event(connection_id, event); if let Some(event_back) = event_back { - // tracing::trace_span!("process event back"); debug_assert!(!is_drained_event); if let Some(sender) = alive_connections.get_mut(&connection_id) { let _ = sender.send(event_back).await; // TODO: don't await here /!\ } else { - tracing::error!("event back: no such connection {:?}", connection_id); + tracing::error!("State mismatch: event for closed connection"); } } } @@ -582,7 +558,6 @@ async fn background_task( future::poll_fn(move |cx| { if active { new_connections.poll_ready(cx) } else { Poll::Pending } }) - .instrument(tracing::trace_span!("readiness")) .fuse() } => { if readiness.is_err() { @@ -598,7 +573,7 @@ async fn background_task( //endpoint.accept(); } - result = udp_socket.recv_from(&mut socket_recv_buffer).instrument(tracing::trace_span!("udp recv")).fuse() => { + result = udp_socket.recv_from(&mut socket_recv_buffer).fuse() => { let (packet_len, packet_src) = match result { Ok(v) => v, // Errors on the socket are expected to never happen, and we handle them by @@ -614,16 +589,12 @@ async fn background_task( let packet = From::from(&socket_recv_buffer[..packet_len]); let local_ip = udp_socket.local_addr().ok().map(|a| a.ip()); // TODO: ECN bits aren't handled - // let span = tracing::trace_span!("endpoint.handle_event"); let event = endpoint.handle(Instant::now(), packet_src, local_ip, None, packet); - // drop(span); - // tracing::trace_span!("process endpoint event"); match event { None => {}, Some((connec_id, quinn_proto::DatagramEvent::ConnectionEvent(event))) => { // Event to send to an existing connection. - // tracing::trace_span!("send event"); if let Some(sender) = alive_connections.get_mut(&connec_id) { let _ = sender.send(event).await; // TODO: don't await here /!\ } else { diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 28780ad02db..00b33f53db7 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -62,6 +62,7 @@ struct SubstreamState { write_waker: Option, /// True if the substream has been closed. finished: bool, + stopped: bool, /// Waker to wake if the substream becomes closed or stopped. finished_waker: Option, } @@ -92,44 +93,19 @@ impl StreamMuxer for QuicMuxer { type Substream = quinn_proto::StreamId; type Error = Error; - // TODO: what if called multiple times? register all wakers? - #[tracing::instrument(skip_all)] fn poll_event( &self, cx: &mut Context<'_>, ) -> Poll, Self::Error>> { - //tracing::info!("here"); - // We use `poll_inbound` to perform the background processing of the entire connection. + // We use `poll_event` to perform the background processing of the entire connection. let mut inner = self.inner.lock(); - let span = if inner.connection.connection.side().is_client() { - tracing::info_span!("client") - } else { - tracing::info_span!("server") - }; - let _enter = span.entered(); - while let Poll::Ready(event) = inner.connection.poll_event(cx) { - // tracing::info!(?event); match event { ConnectionEvent::Connected => { tracing::error!("Unexpected Connected event on established QUIC connection"); } ConnectionEvent::ConnectionLost(e) => { - tracing::error!(?e, "ConnectionLost"); - // for (_, substream) in inner.substreams.iter_mut() { - // substream.finished = true; - // // } - // if let Some(waker) = substream.read_waker.take() { - // waker.wake(); - // } - // if let Some(waker) = substream.write_waker.take() { - // waker.wake(); - // } - // if let Some(waker) = substream.finished_waker.take() { - // waker.wake(); - // } - // } if let Some(waker) = inner.poll_close_waker.take() { waker.wake(); } @@ -156,47 +132,55 @@ impl StreamMuxer for QuicMuxer { } } } - ConnectionEvent::StreamFinished(substream) - | ConnectionEvent::StreamStopped(substream) => { - tracing::info!(?event); + ConnectionEvent::StreamFinished(substream) => { if let Some(substream) = inner.substreams.get_mut(&substream) { // if let ConnectionEvent::StreamFinished(_) = event { substream.finished = true; // } - if let Some(waker) = substream.read_waker.take() { - waker.wake(); - } - if let Some(waker) = substream.write_waker.take() { - waker.wake(); - } + // if let Some(waker) = substream.read_waker.take() { + // waker.wake(); + // } + // if let Some(waker) = substream.write_waker.take() { + // waker.wake(); + // } if let Some(waker) = substream.finished_waker.take() { waker.wake(); } - } else { - tracing::error!("no such substream={}", substream); + } + } + ConnectionEvent::StreamStopped(substream) => { + if let Some(substream) = inner.substreams.get_mut(&substream) { + substream.stopped = true; + // if let ConnectionEvent::StreamFinished(_) = event { + // substream.finished = true; + // } + // if let Some(waker) = substream.read_waker.take() { + // waker.wake(); + // } + // if let Some(waker) = substream.write_waker.take() { + // waker.wake(); + // } + // if let Some(waker) = substream.finished_waker.take() { + // waker.wake(); + // } } } ConnectionEvent::StreamAvailable => { - tracing::info!(?event); // Handled below. } } } if let Some(substream) = inner.connection.pop_incoming_substream() { - tracing::info!("pop_incoming_substream"); inner.substreams.insert(substream, Default::default()); Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) } else { - //tracing::info!("set poll_event_waker"); inner.poll_event_waker = Some(cx.waker().clone()); Poll::Pending } } - #[tracing::instrument(skip_all)] fn open_outbound(&self) -> Self::OutboundSubstream { - tracing::error!("open_outbound"); () } @@ -210,10 +194,8 @@ impl StreamMuxer for QuicMuxer { // Note: this implementation makes it possible to poll the same `Self::OutboundSubstream` // over and over again and get new substreams. Using the API this way is invalid and would // normally result in a panic, but we decide to just ignore this problem. - tracing::info!("poll_outbound"); let mut inner = self.inner.lock(); if let Some(substream) = inner.connection.pop_outgoing_substream() { - tracing::info!(?inner.poll_substream_opened_waker, "poll_outbound substream"); inner.substreams.insert(substream, Default::default()); return Poll::Ready(Ok(substream)); } @@ -224,7 +206,6 @@ impl StreamMuxer for QuicMuxer { .as_ref() .map_or(false, |w| w.will_wake(cx.waker())) { - tracing::info!("set poll_substream_opened_waker"); inner.poll_substream_opened_waker = Some(cx.waker().clone()); } @@ -233,7 +214,6 @@ impl StreamMuxer for QuicMuxer { fn destroy_outbound(&self, _: Self::OutboundSubstream) {} - #[tracing::instrument(skip(self, cx, buf), ret)] fn write_substream( &self, cx: &mut Context<'_>, @@ -244,9 +224,6 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); - let side = inner.connection.connection.side(); - //tracing::info!(?side, ?substream, "write_substream"); - let id = substream; match inner.connection.connection.send_stream(*id).write(buf) { @@ -257,11 +234,9 @@ impl StreamMuxer for QuicMuxer { Poll::Pending } Err(WriteError::Stopped(_)) => { - tracing::error!("write error Stopped: substream={}", id); Poll::Ready(Ok(0)) } Err(WriteError::UnknownStream) => { - tracing::error!("write error UnknownStream substream={}", id); Poll::Ready(Err(Self::Error::ExpiredStream)) } } @@ -293,46 +268,12 @@ impl StreamMuxer for QuicMuxer { // } } - #[tracing::instrument(skip(self, cx, buf), ret)] fn read_substream( &self, cx: &mut Context<'_>, substream: &mut Self::Substream, mut buf: &mut [u8], ) -> Poll> { - // let mut inner = self.inner.lock(); - - // match inner.connection.read_substream(*substream, buf) { - // Ok(bytes) => Poll::Ready(Ok(bytes)), - // Err(quinn_proto::ReadError::Blocked) => { - // if let Some(substream) = inner.substreams.get_mut(substream) { - // if !substream - // .read_waker - // .as_ref() - // .map_or(false, |w| w.will_wake(cx.waker())) - // { - // substream.read_waker = Some(cx.waker().clone()); - // } - // } - // Poll::Pending - // } - - // Err(quinn_proto::ReadError::Reset(err_code)) => { - // Poll::Ready(Err(Error::Reset(err_code))) - // }, - - // // `IllegalOrderedRead` happens if an unordered read followed with an ordered read are - // // performed. `libp2p-quic` never does any unordered read. - // Err(quinn_proto::ReadError::IllegalOrderedRead) => unreachable!(), - // Err(quinn_proto::ReadError::UnknownStream) => { - // tracing::error!( - // "The application used a connection that is already being \ - // closed. This is a bug in the application or in libp2p." - // ); - // Poll::Pending - // } - // } - use quinn_proto::{ReadError, ReadableError}; use std::io::Write; @@ -340,21 +281,18 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); - let side = inner.connection.connection.side(); - //tracing::info!(?side, ?id, "read_substream"); - - // let substream_state = inner.substreams.get_mut(substream) - // .expect("invalid StreamMuxer::read_substream API usage"); - // if substream_state.finished { - // return Poll::Ready(Ok(0)) - // } + let substream_state = inner.substreams.get_mut(substream) + .expect("invalid StreamMuxer::read_substream API usage"); + if substream_state.stopped { + return Poll::Ready(Ok(0)) + } let mut stream = inner.connection.connection.recv_stream(id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(ReadableError::UnknownStream) => { - tracing::error!("read error UnknownStream: substream={}", id); - return Poll::Ready(Ok(0)); + return Poll::Ready(Ok(0)); // FIXME This is a hack, + // a rust client should close substream correctly // return Poll::Ready(Err(Self::Error::ExpiredStream)) } Err(ReadableError::IllegalOrderedRead) => { @@ -373,7 +311,6 @@ impl StreamMuxer for QuicMuxer { bytes += chunk.bytes.len(); } Ok(None) => { - tracing::error!("No more data"); break; } Err(ReadError::Reset(error_code)) => { @@ -382,14 +319,12 @@ impl StreamMuxer for QuicMuxer { break; } Err(ReadError::Blocked) => { - tracing::error!("Blocked"); pending = true; break; } } } if chunks.finalize().should_transmit() { - tracing::error!("read should transmit"); if let Some(waker) = inner.poll_event_waker.take() { waker.wake(); } @@ -403,7 +338,6 @@ impl StreamMuxer for QuicMuxer { } } - #[tracing::instrument(skip(self, cx), ret)] fn shutdown_substream( &self, cx: &mut Context<'_>, @@ -411,7 +345,6 @@ impl StreamMuxer for QuicMuxer { ) -> Poll> { let mut inner = self.inner.lock(); let inner = &mut *inner; - tracing::error!(?inner.connection.connection, "shutdown_substream"); let mut substream_state = inner .substreams @@ -423,11 +356,7 @@ impl StreamMuxer for QuicMuxer { match inner.connection.shutdown_substream(*substream) { Ok(()) => { - substream_state.finished = true; - match substream_state.finished_waker { - Some(ref w) if w.will_wake(cx.waker()) => {} - _ => substream_state.finished_waker = Some(cx.waker().clone()), - } + substream_state.finished_waker = Some(cx.waker().clone()); Poll::Pending } Err(quinn_proto::FinishError::Stopped(err)) => Poll::Ready(Err(Error::Reset(err))), @@ -439,67 +368,27 @@ impl StreamMuxer for QuicMuxer { } } - #[tracing::instrument(skip(self), ret)] fn destroy_substream(&self, substream: Self::Substream) { - tracing::error!(?substream, "destroy_substream"); let mut inner = self.inner.lock(); - if let Some(mut substream) = inner.substreams.remove(&substream) { - // // if let ConnectionEvent::StreamFinished(_) = event { - // substream.finished = true; - // // } - // if let Some(waker) = substream.read_waker.take() { - // tracing::error!("read_waker"); - // waker.wake(); - // } - // if let Some(waker) = substream.write_waker.take() { - // tracing::error!("write_waker"); - // waker.wake(); - // } - // if let Some(waker) = substream.finished_waker.take() { - // tracing::error!("finished_waker"); - // waker.wake(); - // } - } + inner.substreams.remove(&substream); } fn flush_substream( &self, - cx: &mut Context<'_>, + _: &mut Context<'_>, _: &mut Self::Substream, ) -> Poll> { - self.flush_all(cx) + // quinn doesn't support flushing, calling close will flush all substreams. + Poll::Ready(Ok(())) } - // TODO: what if called multiple times? register all wakers? - #[tracing::instrument(skip_all, ret)] - fn flush_all(&self, _cx: &mut Context<'_>) -> Poll> { - return Poll::Ready(Ok(())); - let mut inner = self.inner.lock(); - let inner = &mut *inner; - if inner.substreams.is_empty() { - return Poll::Ready(Ok(())); - } - for substream in inner.substreams.keys() { - match inner.connection.shutdown_substream(*substream) { - Ok(()) => {} - Err(quinn_proto::FinishError::Stopped(err)) => { - return Poll::Ready(Err(Error::Reset(err))) - } - Err(quinn_proto::FinishError::UnknownStream) => { - // Illegal usage of the API. - debug_assert!(false); - return Poll::Ready(Err(Error::ExpiredStream)); - } - } - } - return Poll::Pending; + fn flush_all(&self, _: &mut Context<'_>) -> Poll> { + // quinn doesn't support flushing, calling close will flush all substreams. + Poll::Ready(Ok(())) } - // TODO: what if called multiple times? register all wakers? - #[tracing::instrument(skip_all, ret)] fn close(&self, cx: &mut Context<'_>) -> Poll> { // return Poll::Ready(Ok(())); - tracing::error!("muxer_close"); // StreamMuxer's `close` documentation mentions that it automatically implies `flush_all`. if let Poll::Pending = self.flush_all(cx)? { return Poll::Pending; @@ -540,13 +429,8 @@ impl StreamMuxer for QuicMuxer { } // Register `cx.waker()` as being woken up if the connection closes. - if !inner - .poll_close_waker - .as_ref() - .map_or(false, |w| w.will_wake(cx.waker())) - { - inner.poll_close_waker = Some(cx.waker().clone()); - } + inner.poll_close_waker = Some(cx.waker().clone()); + Poll::Pending } } diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index cf329dd2332..840ba8f4232 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -51,7 +51,6 @@ impl Upgrade { impl Future for Upgrade { type Output = Result<(PeerId, QuicMuxer), transport::Error>; - #[tracing::instrument(skip_all)] fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let connection = self .connection @@ -59,7 +58,6 @@ impl Future for Upgrade { .expect("Future polled after it has completed"); let event = Connection::poll_event(connection, cx); - tracing::info!(?event); match event { Poll::Pending => return Poll::Pending, Poll::Ready(ConnectionEvent::Connected) => { From 728c2ba00b51ca9842bc710edc7675c77151eb2b Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 7 Apr 2022 15:16:48 +0300 Subject: [PATCH 045/218] cargo fmt --- transports/quic/src/connection.rs | 4 +--- transports/quic/src/endpoint.rs | 5 +---- transports/quic/src/muxer.rs | 16 +++++++--------- 3 files changed, 9 insertions(+), 16 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index c410826b927..769e5790cee 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -272,9 +272,7 @@ impl Connection { // Process events that the endpoint has sent to us. loop { match Pin::new(&mut self.from_endpoint).poll_next(cx) { - Poll::Ready(Some(event)) => { - self.connection.handle_event(event) - } + Poll::Ready(Some(event)) => self.connection.handle_event(event), Poll::Ready(None) => { debug_assert!(self.closed.is_none()); let err = Error::ClosedChannel; diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index ee8aef4f766..5dadf05447d 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -450,10 +450,7 @@ async fn background_task( // network interface is too busy, we back-pressure all of our internal // channels. // TODO: set ECN bits; there is no support for them in the ecosystem right now - match udp_socket - .send_to(&data, destination) - .await - { + match udp_socket.send_to(&data, destination).await { Ok(n) if n == data.len() => {} Ok(_) => tracing::error!( "QUIC UDP socket violated expectation that packets are always fully \ diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 00b33f53db7..269e6a82dab 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -233,12 +233,8 @@ impl StreamMuxer for QuicMuxer { substream.write_waker = Some(cx.waker().clone()); Poll::Pending } - Err(WriteError::Stopped(_)) => { - Poll::Ready(Ok(0)) - } - Err(WriteError::UnknownStream) => { - Poll::Ready(Err(Self::Error::ExpiredStream)) - } + Err(WriteError::Stopped(_)) => Poll::Ready(Ok(0)), + Err(WriteError::UnknownStream) => Poll::Ready(Err(Self::Error::ExpiredStream)), } // match inner.connection.write_substream(*substream, buf) { @@ -281,10 +277,12 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); - let substream_state = inner.substreams.get_mut(substream) + let substream_state = inner + .substreams + .get_mut(substream) .expect("invalid StreamMuxer::read_substream API usage"); if substream_state.stopped { - return Poll::Ready(Ok(0)) + return Poll::Ready(Ok(0)); } let mut stream = inner.connection.connection.recv_stream(id); @@ -293,7 +291,7 @@ impl StreamMuxer for QuicMuxer { Err(ReadableError::UnknownStream) => { return Poll::Ready(Ok(0)); // FIXME This is a hack, // a rust client should close substream correctly - // return Poll::Ready(Err(Self::Error::ExpiredStream)) + // return Poll::Ready(Err(Self::Error::ExpiredStream)) } Err(ReadableError::IllegalOrderedRead) => { panic!("Illegal ordered read can only happen if `stream.read(false)` is used."); From deca16a3aec7bd609a0b4577ba27dd4f8528f182 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 7 Apr 2022 15:28:44 +0300 Subject: [PATCH 046/218] retrict concurrent_connections_and_streams to 3:8 params --- transports/quic/tests/smoke.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 70fc5bcd07e..08134ae9d59 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -467,5 +467,7 @@ fn concurrent_connections_and_streams() { TestResult::passed() } - QuickCheck::new().quickcheck(prop as fn(_, _) -> _); + prop(NonZeroU8::new(3).unwrap(), NonZeroU8::new(8).unwrap()); + + // QuickCheck::new().quickcheck(prop as fn(_, _) -> _); } From 8cdc476ef8c0f097391216c5a2d9bbaff6ad902e Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Mon, 9 May 2022 16:58:36 +0300 Subject: [PATCH 047/218] Polish the code --- transports/quic/Cargo.toml | 22 ++-- transports/quic/src/connection.rs | 29 ++---- transports/quic/src/endpoint.rs | 18 ++-- transports/quic/src/in_addr.rs | 26 +++-- transports/quic/src/muxer.rs | 166 ++++++++++++++---------------- transports/quic/src/tls/mod.rs | 2 +- transports/quic/src/transport.rs | 7 +- transports/quic/src/upgrade.rs | 6 +- 8 files changed, 123 insertions(+), 153 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 84421f9b74c..d6e210e0cf1 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -10,35 +10,25 @@ license = "MIT" [dependencies] async-global-executor = "2.0.2" async-io = "1.6.0" -bytes = "1.0.1" -der-parser = { version = "6.0.0", default-features = false } futures = "0.3.15" -if-watch = "0.2.2" +if-watch = "1.0.0" libp2p-core = { version = "0.32.0", path = "../../core" } -parking_lot = "0.11.1" +parking_lot = "0.12.0" quinn-proto = { version = "0.8.2", default-features = false, features = ["tls-rustls"] } -rcgen = { version = "0.8.11" } -ring = { version = "0.16.20" } +rcgen = "0.9.2" +ring = "0.16.20" rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } thiserror = "1.0.26" tracing = "0.1.30" -udp-socket = "0.1.5" webpki = "0.22.0" -x509-parser = "0.12.0" -yasna = { version = "0.4.0" } - -async-std = "*" -futures-timer = "*" +x509-parser = "0.13.0" +yasna = "0.5.0" [dev-dependencies] anyhow = "1.0.41" async-std = { version = "1.10.0", features = ["attributes"] } async-trait = "0.1.50" -async-macros = "*" libp2p = { version = "0.43.0", default-features = false, features = ["request-response"], path = "../.." } -log-panics = "2.0.0" rand = "0.8.4" -rand_core = "0.5.1" tracing-subscriber = {version = "0.3.8", features = ["env-filter"] } -tracing-flame = "0.2" quickcheck = "1" diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 769e5790cee..1dad313ab52 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -28,6 +28,7 @@ use crate::endpoint::Endpoint; +use async_io::Timer; use futures::{channel::mpsc, prelude::*}; use libp2p_core::PeerId; use std::{ @@ -58,7 +59,7 @@ pub(crate) struct Connection { /// the endpoint. connection_id: quinn_proto::ConnectionHandle, /// `Future` that triggers at the `Instant` that `self.connection.poll_timeout()` indicates. - next_timeout: Option, + next_timeout: Option, /// In other to avoid race conditions where a "connected" event happens if we were not /// handshaking, we cache whether the connection is handshaking and only set this to true @@ -157,10 +158,8 @@ impl Connection { let identity = session .peer_identity() .expect("connection got identity because it passed TLS handshake; qed"); - let certificates: Box> = identity - .downcast() - .ok() - .expect("we rely on rustls feature; qed"); + let certificates: Box> = + identity.downcast().expect("we rely on rustls feature; qed"); let end_entity = certificates .get(0) .expect("there should be exactly one certificate; qed"); @@ -170,14 +169,6 @@ impl Connection { PeerId::from_public_key(&p2p_cert.extension.public_key) } - /// If the connection is closed, returns why. If the connection is open, returns `None`. - /// - /// > **Note**: This method is also the main way to determine whether a connection is closed. - pub(crate) fn close_reason(&self) -> Option<&Error> { - debug_assert!(!self.is_handshaking); - self.closed.as_ref() - } - /// Start closing the connection. A [`ConnectionEvent::ConnectionLost`] event will be /// produced in the future. pub(crate) fn close(&mut self) { @@ -306,7 +297,7 @@ impl Connection { // Poll the connection for packets to send on the UDP socket and try to send them on // `to_endpoint`. // FIXME max_datagrams - while let Some(transmit) = self.connection.poll_transmit(now, 1) { + if let Some(transmit) = self.connection.poll_transmit(now, 1) { let endpoint = self.endpoint.clone(); debug_assert!(self.pending_to_endpoint.is_none()); self.pending_to_endpoint = Some(Box::pin(async move { @@ -324,8 +315,8 @@ impl Connection { loop { if let Some(next_timeout) = &mut self.next_timeout { match Future::poll(Pin::new(next_timeout), cx) { - Poll::Ready(()) => { - self.connection.handle_timeout(now); + Poll::Ready(when) => { + self.connection.handle_timeout(when); self.next_timeout = None; } Poll::Pending => break, @@ -334,8 +325,8 @@ impl Connection { if when <= now { self.connection.handle_timeout(now); } else { - let delay = when - now; - self.next_timeout = Some(futures_timer::Delay::new(delay)); + //let delay = when - now; + self.next_timeout = Some(Timer::at(when)); } } else { break; @@ -344,7 +335,7 @@ impl Connection { // The connection also needs to be able to send control messages to the endpoint. This is // handled here, and we try to send them on `to_endpoint` as well. - while let Some(endpoint_event) = self.connection.poll_endpoint_events() { + if let Some(endpoint_event) = self.connection.poll_endpoint_events() { let endpoint = self.endpoint.clone(); let connection_id = self.connection_id; debug_assert!(self.pending_to_endpoint.is_none()); diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 5dadf05447d..1d6ef640be5 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -30,7 +30,8 @@ use crate::{connection::Connection, tls}; -use async_std::net::SocketAddr; +use std::net::{SocketAddr, UdpSocket}; + use futures::{ channel::{mpsc, oneshot}, lock::Mutex, @@ -85,7 +86,7 @@ impl Config { client_config, server_config: Arc::new(server_config), endpoint_config: Default::default(), - multiaddr: multiaddr, + multiaddr, }) } } @@ -117,7 +118,7 @@ pub struct Endpoint { /// has potentially been modified to handle port number `0`. local_multiaddr: Multiaddr, - // after bind(), the result is without port=0 + // The real addr the endpoint bound to. pub(crate) local_addr: SocketAddr, } @@ -156,13 +157,14 @@ impl Endpoint { }); // TODO: just for testing, do proper task spawning - async_std::task::spawn(background_task( + async_global_executor::spawn(background_task( config.clone(), Arc::downgrade(&endpoint), - async_std::net::UdpSocket::from(socket), + async_io::Async::::new(socket)?, new_connections_tx, to_endpoint_rx.fuse(), - )); + )) + .detach(); Ok(endpoint) @@ -414,7 +416,7 @@ enum ToEndpoint { async fn background_task( config: Config, endpoint_weak: Weak, - udp_socket: async_std::net::UdpSocket, + udp_socket: async_io::Async, mut new_connections: mpsc::Sender, mut receiver: stream::Fuse>, ) { @@ -584,7 +586,7 @@ async fn background_task( // Received a UDP packet from the socket. debug_assert!(packet_len <= socket_recv_buffer.len()); let packet = From::from(&socket_recv_buffer[..packet_len]); - let local_ip = udp_socket.local_addr().ok().map(|a| a.ip()); + let local_ip = udp_socket.get_ref().local_addr().ok().map(|a| a.ip()); // TODO: ECN bits aren't handled let event = endpoint.handle(Instant::now(), packet_src, local_ip, None, packet); diff --git a/transports/quic/src/in_addr.rs b/transports/quic/src/in_addr.rs index 5a45b4c7fa7..113b91f46d5 100644 --- a/transports/quic/src/in_addr.rs +++ b/transports/quic/src/in_addr.rs @@ -9,19 +9,25 @@ use futures::{ use std::{ io::Result, net::IpAddr, + ops::DerefMut, pin::Pin, sync::Arc, task::{Context, Poll}, }; +/// Watches for interface changes. #[derive(Clone, Debug)] pub(crate) struct InAddr(Arc>); impl InAddr { + /// If ip is specified then only one `IfEvent::Up` with IpNet(ip)/32 will be generated. + /// If ip is unspecified then `IfEvent::Up/Down` events will be generated for all interfaces. pub(crate) fn new(ip: IpAddr) -> Self { let inner = if ip.is_unspecified() { let watcher = IfWatch::Pending(IfWatcher::new().boxed()); - InAddrInner::Any { if_watch: watcher } + InAddrInner::Any { + if_watch: Box::new(watcher), + } } else { InAddrInner::One { ip: Some(ip) } }; @@ -35,12 +41,12 @@ enum InAddrInner { /// The socket accepts connections on a single interface. One { ip: Option }, /// The socket accepts connections on all interfaces. - Any { if_watch: IfWatch }, + Any { if_watch: Box }, } enum IfWatch { Pending(BoxFuture<'static, std::io::Result>), - Ready(IfWatcher), + Ready(Box), } impl std::fmt::Debug for IfWatch { @@ -55,7 +61,6 @@ impl std::fmt::Debug for IfWatch { impl Stream for InAddr { type Item = Result; - #[tracing::instrument(skip_all)] fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { let me = Pin::into_inner(self); let mut lock = me.0.lock(); @@ -69,7 +74,6 @@ impl Stream for InAddr { impl Stream for InAddrInner { type Item = Result; - #[tracing::instrument(skip_all)] fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { let me = Pin::into_inner(self); loop { @@ -82,21 +86,21 @@ impl Stream for InAddrInner { } } InAddrInner::Any { if_watch } => { - match &mut *if_watch { + match if_watch.deref_mut() { // If we listen on all interfaces, wait for `if-watch` to be ready. IfWatch::Pending(f) => match futures::ready!(f.poll_unpin(cx)) { - Ok(w) => { - *if_watch = IfWatch::Ready(w); + Ok(watcher) => { + *if_watch = Box::new(IfWatch::Ready(Box::new(watcher))); continue; } Err(err) => { - *if_watch = IfWatch::Pending(IfWatcher::new().boxed()); + *if_watch = Box::new(IfWatch::Pending(IfWatcher::new().boxed())); return Poll::Ready(Some(Err(err))); } }, // Consume all events for up/down interface changes. - IfWatch::Ready(watch) => { - while let Poll::Ready(ev) = watch.poll_unpin(cx) { + IfWatch::Ready(watcher) => { + if let Poll::Ready(ev) = watcher.poll_unpin(cx) { match ev { Ok(event) => { return Poll::Ready(Some(Ok(event))); diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 269e6a82dab..23bec33add0 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -24,7 +24,7 @@ use crate::error::Error; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ - collections::HashMap, + collections::{HashMap, VecDeque}, fmt, task::{Context, Poll, Waker}, }; @@ -45,8 +45,8 @@ struct QuicMuxerInner { connection: Connection, /// State of all the substreams that the muxer reports as open. substreams: HashMap, - /// Waker to wake if a new outgoing substream is opened. - poll_substream_opened_waker: Option, + /// A FIFO of wakers to wake if a new outgoing substream is opened. + pending_substreams: VecDeque, /// Waker to wake if the connection is closed. poll_close_waker: Option, /// Waker to wake if any event is happened. @@ -60,8 +60,9 @@ struct SubstreamState { read_waker: Option, /// Waker to wake if the substream becomes writable or stopped. write_waker: Option, - /// True if the substream has been closed. + /// True if the substream has been finished. finished: bool, + /// True if the substream has been stopped. stopped: bool, /// Waker to wake if the substream becomes closed or stopped. finished_waker: Option, @@ -80,7 +81,7 @@ impl QuicMuxer { inner: Mutex::new(QuicMuxerInner { connection, substreams: Default::default(), - poll_substream_opened_waker: None, + pending_substreams: Default::default(), poll_close_waker: None, poll_event_waker: None, }), @@ -93,6 +94,18 @@ impl StreamMuxer for QuicMuxer { type Substream = quinn_proto::StreamId; type Error = Error; + /// Polls for a connection-wide event. + /// + /// This function behaves the same as a `Stream`. + /// + /// If `Pending` is returned, then the current task will be notified once the muxer + /// is ready to be polled, similar to the API of `Stream::poll()`. + /// Only the latest task that was used to call this method may be notified. + /// + /// It is permissible and common to use this method to perform background + /// work, such as processing incoming packets and polling timers. + /// + /// An error can be generated if the connection has been closed. fn poll_event( &self, cx: &mut Context<'_>, @@ -105,16 +118,15 @@ impl StreamMuxer for QuicMuxer { ConnectionEvent::Connected => { tracing::error!("Unexpected Connected event on established QUIC connection"); } - ConnectionEvent::ConnectionLost(e) => { + ConnectionEvent::ConnectionLost(_) => { if let Some(waker) = inner.poll_close_waker.take() { waker.wake(); } inner.connection.close(); - // return Poll::Ready(Err(Error::ConnectionLost(e))) } ConnectionEvent::StreamOpened => { - if let Some(waker) = inner.poll_substream_opened_waker.take() { + if let Some(waker) = inner.pending_substreams.pop_front() { waker.wake(); } } @@ -134,15 +146,7 @@ impl StreamMuxer for QuicMuxer { } ConnectionEvent::StreamFinished(substream) => { if let Some(substream) = inner.substreams.get_mut(&substream) { - // if let ConnectionEvent::StreamFinished(_) = event { substream.finished = true; - // } - // if let Some(waker) = substream.read_waker.take() { - // waker.wake(); - // } - // if let Some(waker) = substream.write_waker.take() { - // waker.wake(); - // } if let Some(waker) = substream.finished_waker.take() { waker.wake(); } @@ -151,18 +155,6 @@ impl StreamMuxer for QuicMuxer { ConnectionEvent::StreamStopped(substream) => { if let Some(substream) = inner.substreams.get_mut(&substream) { substream.stopped = true; - // if let ConnectionEvent::StreamFinished(_) = event { - // substream.finished = true; - // } - // if let Some(waker) = substream.read_waker.take() { - // waker.wake(); - // } - // if let Some(waker) = substream.write_waker.take() { - // waker.wake(); - // } - // if let Some(waker) = substream.finished_waker.take() { - // waker.wake(); - // } } } ConnectionEvent::StreamAvailable => { @@ -180,40 +172,50 @@ impl StreamMuxer for QuicMuxer { } } - fn open_outbound(&self) -> Self::OutboundSubstream { - () - } + /// Opens a new outgoing substream, and produces the equivalent to a future that will be + /// resolved when it becomes available. + /// + /// We provide the same handler to poll it by multiple tasks, which is done as a FIFO + /// queue via `poll_outbound`. + fn open_outbound(&self) -> Self::OutboundSubstream {} - // TODO: what if called multiple times? register all wakers? - #[tracing::instrument(skip_all)] + /// Polls the outbound substream. + /// + /// If `Pending` is returned, then the current task will be notified once the substream + /// is ready to be polled, similar to the API of `Future::poll()`. fn poll_outbound( &self, cx: &mut Context<'_>, _: &mut Self::OutboundSubstream, ) -> Poll> { - // Note: this implementation makes it possible to poll the same `Self::OutboundSubstream` - // over and over again and get new substreams. Using the API this way is invalid and would - // normally result in a panic, but we decide to just ignore this problem. let mut inner = self.inner.lock(); if let Some(substream) = inner.connection.pop_outgoing_substream() { inner.substreams.insert(substream, Default::default()); - return Poll::Ready(Ok(substream)); - } - - // Register `cx.waker()` as having to be woken up once a substream is available. - if !inner - .poll_substream_opened_waker - .as_ref() - .map_or(false, |w| w.will_wake(cx.waker())) - { - inner.poll_substream_opened_waker = Some(cx.waker().clone()); + Poll::Ready(Ok(substream)) + } else { + inner.pending_substreams.push_back(cx.waker().clone()); + Poll::Pending } - - Poll::Pending } - fn destroy_outbound(&self, _: Self::OutboundSubstream) {} + /// Destroys an outbound substream future. Use this after the outbound substream has finished, + /// or if you want to interrupt it. + fn destroy_outbound(&self, _: Self::OutboundSubstream) { + // Do nothing because we don't know which waker should be destroyed. + // TODO `Self::OutboundSubstream` -> autoincrement id. + } + /// Write data to a substream. The behaviour is the same as `futures::AsyncWrite::poll_write`. + /// + /// If `Pending` is returned, then the current task will be notified once the substream + /// is ready to be read. For each individual substream, only the latest task that was used to + /// call this method may be notified. + /// + /// Calling `write_substream` does not guarantee that data will arrive to the remote. To + /// ensure that, you should call `flush_substream`. + /// + /// It is incorrect to call this method on a substream if you called `shutdown_substream` on + /// this substream earlier. fn write_substream( &self, cx: &mut Context<'_>, @@ -233,37 +235,28 @@ impl StreamMuxer for QuicMuxer { substream.write_waker = Some(cx.waker().clone()); Poll::Pending } - Err(WriteError::Stopped(_)) => Poll::Ready(Ok(0)), - Err(WriteError::UnknownStream) => Poll::Ready(Err(Self::Error::ExpiredStream)), + Err(WriteError::Stopped(err_code)) => Poll::Ready(Err(Error::Reset(err_code))), + Err(WriteError::UnknownStream) => { + tracing::error!( + "The application used a connection that is already being \ + closed. This is a bug in the application or in libp2p." + ); + Poll::Pending + } } - - // match inner.connection.write_substream(*substream, buf) { - // Ok(bytes) => Poll::Ready(Ok(bytes)), - // Err(quinn_proto::WriteError::Stopped(err_code)) => { - // Poll::Ready(Err(Error::Reset(err_code))) - // }, - // Err(quinn_proto::WriteError::Blocked) => { - // if let Some(substream) = inner.substreams.get_mut(substream) { - // if !substream - // .write_waker - // .as_ref() - // .map_or(false, |w| w.will_wake(cx.waker())) - // { - // substream.write_waker = Some(cx.waker().clone()); - // } - // } - // Poll::Pending - // } - // Err(quinn_proto::WriteError::UnknownStream) => { - // tracing::error!( - // "The application used a connection that is already being \ - // closed. This is a bug in the application or in libp2p." - // ); - // Poll::Pending - // } - // } } + /// Reads data from a substream. The behaviour is the same as `futures::AsyncRead::poll_read`. + /// + /// If `Pending` is returned, then the current task will be notified once the substream + /// is ready to be read. However, for each individual substream, only the latest task that + /// was used to call this method may be notified. + /// + /// If `Async::Ready(0)` is returned, the substream has been closed by the remote and should + /// no longer be read afterwards. + /// + /// An error can be generated if the connection has been closed, or if a protocol misbehaviour + /// happened. fn read_substream( &self, cx: &mut Context<'_>, @@ -386,14 +379,11 @@ impl StreamMuxer for QuicMuxer { } fn close(&self, cx: &mut Context<'_>) -> Poll> { - // return Poll::Ready(Ok(())); // StreamMuxer's `close` documentation mentions that it automatically implies `flush_all`. - if let Poll::Pending = self.flush_all(cx)? { + if self.flush_all(cx)?.is_pending() { return Poll::Pending; } - // TODO: poll if closed or something - let mut inner = self.inner.lock(); if inner.connection.connection.is_drained() { @@ -409,20 +399,16 @@ impl StreamMuxer for QuicMuxer { } } else { } - tracing::error!( - is_closed = inner.connection.connection.is_closed(), - is_drained = inner.connection.connection.is_drained() - ); while let Poll::Ready(event) = inner.connection.poll_event(cx) { - match event { - ConnectionEvent::ConnectionLost(_) => return Poll::Ready(Ok(())), - _ => {} + if let ConnectionEvent::ConnectionLost(_) = event { + return Poll::Ready(Ok(())); } } - // return Poll::Ready(Ok(())) } else { for substream in inner.substreams.clone().keys() { - inner.connection.shutdown_substream(*substream); + if let Err(e) = inner.connection.shutdown_substream(*substream) { + tracing::error!("substream finish error on muxer close: {}", e); + } } } diff --git a/transports/quic/src/tls/mod.rs b/transports/quic/src/tls/mod.rs index 5183ef57955..78b31a7d4e9 100644 --- a/transports/quic/src/tls/mod.rs +++ b/transports/quic/src/tls/mod.rs @@ -96,7 +96,7 @@ pub fn make_server_config( fn make_cert_key( keypair: &libp2p_core::identity::Keypair, ) -> Result<(rustls::Certificate, rustls::PrivateKey), ConfigError> { - let cert = certificate::make_certificate(&keypair)?; + let cert = certificate::make_certificate(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/transport.rs b/transports/quic/src/transport.rs index 2d17da2a652..858578fe310 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -91,7 +91,6 @@ impl Transport for QuicTransport { type ListenerUpgrade = Upgrade; type Dial = Pin> + Send>>; - #[tracing::instrument] fn listen_on(self, addr: Multiaddr) -> Result> { multiaddr_to_socketaddr(&addr) .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; @@ -102,7 +101,6 @@ impl Transport for QuicTransport { Some(observed.clone()) } - #[tracing::instrument] fn dial(self, addr: Multiaddr) -> Result> { let socket_addr = if let Some(socket_addr) = multiaddr_to_socketaddr(&addr) { if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { @@ -139,7 +137,6 @@ impl Transport for QuicTransport { impl Stream for QuicTransport { type Item = Result, Error>; - #[tracing::instrument(skip_all)] fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { let me = Pin::into_inner(self); let endpoint = me.endpoint.as_ref(); @@ -186,7 +183,7 @@ impl Stream for QuicTransport { } let connection = match endpoint.poll_incoming(cx) { - Poll::Ready(Some(conn)) => conn, + Poll::Ready(Some(connection)) => connection, Poll::Ready(None) => return Poll::Ready(None), Poll::Pending => return Poll::Pending, }; @@ -209,7 +206,7 @@ pub(crate) fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { let proto2 = iter.next()?; let proto3 = iter.next()?; - while let Some(proto) = iter.next() { + for proto in iter { match proto { Protocol::P2p(_) => {} // Ignore a `/p2p/...` prefix of possibly outer protocols, if present. _ => return None, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 840ba8f4232..4114c8c2df2 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -59,14 +59,14 @@ impl Future for Upgrade { let event = Connection::poll_event(connection, cx); match event { - Poll::Pending => return Poll::Pending, + Poll::Pending => Poll::Pending, Poll::Ready(ConnectionEvent::Connected) => { let peer_id = connection.remote_peer_id(); let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); - return Poll::Ready(Ok((peer_id, muxer))); + Poll::Ready(Ok((peer_id, muxer))) } Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { - return Poll::Ready(Err(transport::Error::Established(err))); + Poll::Ready(Err(transport::Error::Established(err))) } // Other items are: // - StreamAvailable From 6bd30414f4ba391738a9b0a20b0f896055e0d03d Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 12 May 2022 00:58:59 +0300 Subject: [PATCH 048/218] Remove old comments --- transports/quic/src/endpoint.rs | 46 +-------------------------------- 1 file changed, 1 insertion(+), 45 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 1d6ef640be5..3d7c94810c2 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -158,7 +158,7 @@ impl Endpoint { // TODO: just for testing, do proper task spawning async_global_executor::spawn(background_task( - config.clone(), + config, Arc::downgrade(&endpoint), async_io::Async::::new(socket)?, new_connections_tx, @@ -167,49 +167,6 @@ impl Endpoint { .detach(); Ok(endpoint) - - // TODO: IP address stuff - /*if socket_addr.ip().is_unspecified() { - info!("returning all local IPs for unspecified address"); - let suffixes = [Protocol::Udp(socket_addr.port()), Protocol::Quic]; - let local_addresses = - host_addresses(&suffixes).map_err(|e| TransportError::Other(Error::IO(e)))?; - for (_, _, address) in local_addresses { - info!("sending address {:?}", address); - new_connections - .unbounded_send(ListenerEvent::NewAddress(address)) - .expect("we have a reference to the peer, so this will not fail; qed") - } - } else { - info!("sending address {:?}", multiaddr); - new_connections - .unbounded_send(ListenerEvent::NewAddress(multiaddr.clone())) - .expect("we have a reference to the peer, so this will not fail; qed"); - } - - if socket_addr.ip().is_unspecified() { - debug!("returning all local IPs for unspecified address"); - let local_addresses = - host_addresses(&[Protocol::Udp(socket_addr.port()), Protocol::Quic]) - .map_err(|e| TransportError::Other(Error::IO(e)))?; - for i in local_addresses { - info!("sending address {:?}", i.2); - reference - .new_connections - .unbounded_send(ListenerEvent::NewAddress(i.2)) - .expect("we have a reference to the peer, so this will not fail; qed") - } - } else { - info!("sending address {:?}", multiaddr); - reference - .new_connections - .unbounded_send(ListenerEvent::NewAddress(multiaddr)) - .expect("we have a reference to the peer, so this will not fail; qed"); - } - - let endpoint = EndpointRef { reference, channel }; - let join_handle = spawn(endpoint.clone()); - Ok((Self(endpoint), join_handle))*/ } /// Asks the endpoint to start dialing the given address. @@ -514,7 +471,6 @@ async fn background_task( if !has_key { continue; } - //debug_assert!(alive_connections.contains_key(&connection_id)); // We "drained" event indicates that the connection no longer exists and // its ID can be reclaimed. let is_drained_event = event.is_drained(); From af909b1490046d026d99061102e31f250c3fd37e Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 12 May 2022 01:46:31 +0300 Subject: [PATCH 049/218] Fix smoke test --- transports/quic/tests/smoke.rs | 49 +++++++--------------------------- 1 file changed, 10 insertions(+), 39 deletions(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 08134ae9d59..92ec9fccbe7 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -52,33 +52,17 @@ async fn create_swarm(keylog: bool) -> Result>> Ok(Swarm::new(transport, behaviour, peer_id)) } -fn setup_global_subscriber() -> impl Drop { - use tracing_flame::FlameLayer; - use tracing_subscriber::{fmt, prelude::*}; - +fn setup_global_subscriber() { let filter_layer = tracing_subscriber::EnvFilter::from_default_env(); - - let fmt_format = tracing_subscriber::fmt::format() - .pretty() - .with_thread_ids(false) - .without_time(); - let fmt_layer = fmt::Layer::default().event_format(fmt_format); - - let (flame_layer, _guard) = FlameLayer::with_file("./tracing.folded").unwrap(); - - tracing_subscriber::registry() - .with(filter_layer) - .with(fmt_layer) - .with(flame_layer) + tracing_subscriber::fmt() + .with_env_filter(filter_layer) .try_init() .ok(); - _guard } #[async_std::test] async fn smoke() -> Result<()> { - let _guard = setup_global_subscriber(); - log_panics::init(); + setup_global_subscriber(); let mut rng = rand::thread_rng(); let mut a = create_swarm(true).await?; @@ -281,11 +265,7 @@ impl RequestResponseCodec for PingCodec { #[async_std::test] async fn dial_failure() -> Result<()> { - tracing_subscriber::fmt() - .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) - .try_init() - .ok(); - log_panics::init(); + setup_global_subscriber(); let mut a = create_swarm(false).await?; let mut b = create_swarm(true).await?; @@ -326,16 +306,10 @@ fn concurrent_connections_and_streams() { use futures::executor::block_on; use quickcheck::*; - tracing_subscriber::fmt() - //.pretty() - .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) - .try_init() - .ok(); - log_panics::init(); + setup_global_subscriber(); #[tracing::instrument] fn prop(number_listeners: NonZeroU8, number_streams: NonZeroU8) -> TestResult { - tracing::info!("entered"); let (number_listeners, number_streams): (u8, u8) = (number_listeners.into(), number_streams.into()); if number_listeners > 10 || number_streams > 10 { @@ -393,8 +367,8 @@ fn concurrent_connections_and_streams() { } Some(SwarmEvent::ConnectionClosed { .. }) => {} Some(e) => { - tracing::info!(?e, "listener"); - } // panic!("{:?}", e), + panic!("unexpected event {:?}", e); + } None => { panic!("listener stopped"); } @@ -416,9 +390,6 @@ fn concurrent_connections_and_streams() { .add_address(&listener_peer_id, listener_addr.clone()); dialer.dial(listener_peer_id.clone()).unwrap(); - //dialer - // .behaviour_mut() - // .send_request(&listener_peer_id, Ping(data.clone())); } // Wait for responses to each request. @@ -458,8 +429,8 @@ fn concurrent_connections_and_streams() { tracing::info!("dialer ConnectionClosed"); } e => { - tracing::info!(?e, "dialer"); - } // panic!("{:?}", e), + panic!("unexpected event {:?}", e); + } } } }); From ce034491c3d2cbbe0bfaa2a46230775e3cafa5e2 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 18 May 2022 16:50:21 +0300 Subject: [PATCH 050/218] Fix several warnings --- transports/quic/src/endpoint.rs | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 3d7c94810c2..c3c23d00998 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -111,14 +111,8 @@ pub struct Endpoint { /// guaranteed a slot in the messages buffer. to_endpoint2: mpsc::Sender, - /// Configuration passed at initialization. - // TODO: remove? - config: Config, - /// Multiaddr of the local UDP socket passed in the configuration at initialization after it + /// Socketaddr of the local UDP socket passed in the configuration at initialization after it /// has potentially been modified to handle port number `0`. - local_multiaddr: Multiaddr, - - // The real addr the endpoint bound to. pub(crate) local_addr: SocketAddr, } @@ -151,8 +145,6 @@ impl Endpoint { to_endpoint: Mutex::new(to_endpoint_tx), to_endpoint2, new_connections: Mutex::new(new_connections_rx), - config: config.clone(), - local_multiaddr: config.multiaddr.clone(), // TODO: no local_addr: socket.local_addr()?, }); From b2a34e4ffac36493938f070a62302735fb436788 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 8 Jun 2022 15:58:09 +0300 Subject: [PATCH 051/218] Remove tmp change in swarm pool_size --- swarm/src/lib.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/swarm/src/lib.rs b/swarm/src/lib.rs index 8f7065ac608..5cc69030274 100644 --- a/swarm/src/lib.rs +++ b/swarm/src/lib.rs @@ -1337,7 +1337,6 @@ where self.pool_config.or_else_with_executor(|| { match ThreadPoolBuilder::new() .name_prefix("libp2p-swarm-task-") - .pool_size(1) .create() { Ok(tp) => Some(Box::new(move |f| tp.spawn_ok(f))), From 38c2fd59ad06d0f74d79d3812747f479b8dc8504 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 8 Jun 2022 16:00:52 +0300 Subject: [PATCH 052/218] Set libp2p-quic to v0.7 --- Cargo.toml | 2 +- transports/quic/Cargo.toml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 3d352a3ab2f..19ba5c8b5b9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -109,7 +109,7 @@ smallvec = "1.6.1" libp2p-deflate = { version = "0.32.0", path = "transports/deflate", optional = true } libp2p-dns = { version = "0.32.1", path = "transports/dns", optional = true, default-features = false } libp2p-mdns = { version = "0.36.0", path = "protocols/mdns", optional = true } -libp2p-quic = { version = "0.6.0", path = "transports/quic", optional = true } +libp2p-quic = { version = "0.7.0", path = "transports/quic", optional = true } libp2p-tcp = { version = "0.32.0", path = "transports/tcp", default-features = false, optional = true } libp2p-websocket = { version = "0.34.0", path = "transports/websocket", optional = true } diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 9574616988f..1694043644b 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -1,8 +1,8 @@ [package] name = "libp2p-quic" -version = "0.6.0" +version = "0.7.0" authors = ["David Craven ", "Parity Technologies "] -edition = "2018" +edition = "2021" description = "TLS based QUIC transport implementation for libp2p" repository = "https://github.com/libp2p/rust-libp2p" license = "MIT" From ef8decf03c895d39946d8af83b60c5cb5768794b Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 8 Jun 2022 16:02:08 +0300 Subject: [PATCH 053/218] Remove old comments --- transports/quic/src/connection.rs | 33 ------------------------------- 1 file changed, 33 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 1dad313ab52..5f02c75fdbc 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -202,39 +202,6 @@ impl Connection { self.connection.streams().open(quinn_proto::Dir::Bi) } - // /// Reads data from the given substream. Similar to the API of `std::io::Read`. - // /// - // /// If `Err(ReadError::Blocked)` is returned, then a [`ConnectionEvent::StreamReadable`] event - // /// will later be produced when the substream has readable data. A - // /// [`ConnectionEvent::StreamStopped`] event can also be emitted. - // pub(crate) fn read_substream( - // &mut self, - // id: quinn_proto::StreamId, - // buf: &mut [u8], - // ) -> Result { - // let mut stream = self.connection.recv_stream(id); - // let mut chunks = stream.read(true)?; - // self.connection.read(id, buf).map(|n| { - // // `n` is `None` in case of EOF. - // // See https://github.com/quinn-rs/quinn/blob/9aa3bde3aa1319b2c743f792312508de9270b8c6/quinn/src/streams.rs#L367-L370 - // debug_assert_ne!(n, Some(0)); // Sanity check - // n.unwrap_or(0) - // }) - // } - - // /// Writes data to the given substream. Similar to the API of `std::io::Write`. - // /// - // /// If `Err(WriteError::Blocked)` is returned, then a [`ConnectionEvent::StreamWritable`] event - // /// will later be produced when the substream can be written to. A - // /// [`ConnectionEvent::StreamStopped`] event can also be emitted. - // pub(crate) fn write_substream( - // &mut self, - // id: quinn_proto::StreamId, - // buf: &[u8], - // ) -> Result { - // self.connection.write(id, buf) - // } - /// Closes the given substream. /// /// [`Connection::write_substream`] must no longer be called. The substream is however still From e55c0cfda8c6b31dd54b53d7ebaceca6e9fcc3b6 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 8 Jun 2022 16:37:03 +0300 Subject: [PATCH 054/218] Simplify the code in timeout system --- transports/quic/src/connection.rs | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 5f02c75fdbc..58e8569111a 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -288,16 +288,12 @@ impl Connection { } Poll::Pending => break, } - } else if let Some(when) = self.connection.poll_timeout() { - if when <= now { - self.connection.handle_timeout(now); - } else { - //let delay = when - now; - self.next_timeout = Some(Timer::at(when)); - } - } else { - break; } + if let Some(when) = self.connection.poll_timeout() { + self.next_timeout = Some(Timer::at(when)); + continue; + } + break; } // The connection also needs to be able to send control messages to the endpoint. This is From e937ce63c20ccbfd333ba51e05842cf3d6ecc1e2 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 8 Jun 2022 16:40:10 +0300 Subject: [PATCH 055/218] Remove additional recursion_limit --- transports/quic/src/lib.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 8a5c929c7ed..de2487a75ca 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -18,8 +18,6 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -#![recursion_limit = "1024"] - //! Implementation of the libp2p `Transport` and `StreamMuxer` traits for QUIC. //! //! # Usage From ffafd4933eb1388e824d10bf43c77c19676017e9 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Tue, 21 Jun 2022 15:16:00 +0300 Subject: [PATCH 056/218] Fix upgrade bugs --- transports/quic/src/muxer.rs | 12 +----------- transports/quic/src/transport.rs | 13 +++++++------ 2 files changed, 8 insertions(+), 17 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 23bec33add0..82e5a6fd2b2 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -373,17 +373,7 @@ impl StreamMuxer for QuicMuxer { Poll::Ready(Ok(())) } - fn flush_all(&self, _: &mut Context<'_>) -> Poll> { - // quinn doesn't support flushing, calling close will flush all substreams. - Poll::Ready(Ok(())) - } - - fn close(&self, cx: &mut Context<'_>) -> Poll> { - // StreamMuxer's `close` documentation mentions that it automatically implies `flush_all`. - if self.flush_all(cx)?.is_pending() { - return Poll::Pending; - } - + fn poll_close(&self, cx: &mut Context<'_>) -> Poll> { let mut inner = self.inner.lock(); if inner.connection.connection.is_drained() { diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 858578fe310..3d63087ba25 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -91,17 +91,17 @@ impl Transport for QuicTransport { type ListenerUpgrade = Upgrade; type Dial = Pin> + Send>>; - fn listen_on(self, addr: Multiaddr) -> Result> { + fn listen_on(&mut self, addr: Multiaddr) -> Result> { multiaddr_to_socketaddr(&addr) .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; - Ok(self) + Ok(self.clone()) } fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { Some(observed.clone()) } - fn dial(self, addr: Multiaddr) -> Result> { + fn dial(&mut self, addr: Multiaddr) -> Result> { let socket_addr = if let Some(socket_addr) = multiaddr_to_socketaddr(&addr) { if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { tracing::error!("multiaddr not supported"); @@ -113,9 +113,10 @@ impl Transport for QuicTransport { return Err(TransportError::MultiaddrNotSupported(addr)); }; + let endpoint = self.endpoint.clone(); + Ok(async move { - let connection = self - .endpoint + let connection = endpoint .dial(socket_addr) .await .map_err(Error::Reach)?; @@ -125,7 +126,7 @@ impl Transport for QuicTransport { .boxed()) } - fn dial_as_listener(self, addr: Multiaddr) -> Result> { + fn dial_as_listener(&mut self, addr: Multiaddr) -> Result> { // TODO: As the listener of a QUIC hole punch, we need to send a random UDP packet to the // `addr`. See DCUtR specification below. // From 772162b4abe4bf21e4e48683990720c3a590ef48 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 22 Jun 2022 15:33:52 +0300 Subject: [PATCH 057/218] cargo fmt --- transports/quic/src/transport.rs | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 3d63087ba25..6c06c38a6ff 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -91,7 +91,10 @@ impl Transport for QuicTransport { type ListenerUpgrade = Upgrade; type Dial = Pin> + Send>>; - fn listen_on(&mut self, addr: Multiaddr) -> Result> { + fn listen_on( + &mut self, + addr: Multiaddr, + ) -> Result> { multiaddr_to_socketaddr(&addr) .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; Ok(self.clone()) @@ -116,17 +119,17 @@ impl Transport for QuicTransport { let endpoint = self.endpoint.clone(); Ok(async move { - let connection = endpoint - .dial(socket_addr) - .await - .map_err(Error::Reach)?; + let connection = endpoint.dial(socket_addr).await.map_err(Error::Reach)?; let final_connec = Upgrade::from_connection(connection).await?; Ok(final_connec) } .boxed()) } - fn dial_as_listener(&mut self, addr: Multiaddr) -> Result> { + fn dial_as_listener( + &mut self, + addr: Multiaddr, + ) -> Result> { // TODO: As the listener of a QUIC hole punch, we need to send a random UDP packet to the // `addr`. See DCUtR specification below. // From 4851552f2ca8e47b535a1cb790b4501102fbb298 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 22 Jun 2022 15:41:08 +0300 Subject: [PATCH 058/218] Fix intradocs --- 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 c3c23d00998..9bf87105939 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -104,7 +104,7 @@ pub struct Endpoint { /// This is protected by a futures-friendly `Mutex`, meaning that receiving a connection is /// done in two steps: locking this mutex, and grabbing the next element on the `Receiver`. /// The only consequence of this `Mutex` is that multiple simultaneous calls to - /// [`Endpoint::next_incoming`] are serialized. + /// [`Endpoint::poll_incoming`] are serialized. new_connections: Mutex>, /// Copy of [`Endpoint::to_endpoint`], except not behind a `Mutex`. Used if we want to be From c4c07a2e223de40d21c8bf7113049862f57e908b Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Wed, 22 Jun 2022 16:46:37 +0300 Subject: [PATCH 059/218] Fix intradocs --- transports/quic/src/connection.rs | 10 ++++------ transports/quic/src/endpoint.rs | 6 +++--- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 58e8569111a..f91cd9af127 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -204,10 +204,10 @@ impl Connection { /// Closes the given substream. /// - /// [`Connection::write_substream`] must no longer be called. The substream is however still + /// `write_substream` must no longer be called. The substream is however still /// readable. /// - /// On success, a [`StreamEvent::Finished`] event will later be produced when the + /// On success, a [`quinn_proto::StreamEvent::Finished`] event will later be produced when the /// substream has been effectively closed. A [`ConnectionEvent::StreamStopped`] event can also /// be emitted. pub(crate) fn shutdown_substream( @@ -413,11 +413,9 @@ pub(crate) enum ConnectionEvent { /// `None`. After this event has been generated, this method is guaranteed to return `Some`. StreamOpened, - /// Generated after [`Connection::read_substream`] has been called and has returned a - /// `Blocked` error. + /// Generated after `read_substream` has returned a `Blocked` error. StreamReadable(quinn_proto::StreamId), - /// Generated after [`Connection::write_substream`] has been called and has returned a - /// `Blocked` error. + /// Generated after `write_substream` has returned a `Blocked` error. StreamWritable(quinn_proto::StreamId), /// Generated after [`Connection::shutdown_substream`] has been called. diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 9bf87105939..6d0d3ceaa94 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -328,7 +328,7 @@ enum ToEndpoint { /// [`quinn_proto::Endpoint`]. /// /// Unfortunately, this design has the consequence that, on the network layer, we will accept a -/// certain number of incoming connections even if [`Endpoint::next_incoming`] is never even +/// certain number of incoming connections even if [`Endpoint::poll_incoming`] is never even /// called. The `quinn-proto` library doesn't provide any way to not accept incoming connections /// apart from filling the accept buffer. /// @@ -344,8 +344,8 @@ enum ToEndpoint { /// In an ideal world, we would handle a background-task-to-connection channel being full by /// dropping UDP packets destined to this connection, as a way to back-pressure the remote. /// Unfortunately, the `quinn-proto` library doesn't provide any way for us to know which -/// connection a UDP packet is destined for before it has been turned into a [`ConnectionEvent`], -/// and because these [`ConnectionEvent`]s are sometimes used to synchronize the states of the +/// connection a UDP packet is destined for before it has been turned into a `ConnectionEvent`, +/// and because these `ConnectionEvent`s are sometimes used to synchronize the states of the /// endpoint and connection, it would be a logic error to silently drop them. /// /// We handle this tricky situation by simply killing connections as soon as their associated From cb92ae5d2b29a35033dd3fa06ca68bda95195fe3 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 7 Jul 2022 16:16:22 +0300 Subject: [PATCH 060/218] Impl AsyncRead/AsyncWrite for Substream --- transports/quic/src/lib.rs | 2 +- transports/quic/src/muxer.rs | 378 ++++++++++++++++++----------------- 2 files changed, 198 insertions(+), 182 deletions(-) diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index de2487a75ca..116d459bcd6 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -63,6 +63,6 @@ pub mod transport; pub use endpoint::{Config, Endpoint}; pub use error::Error; -pub use muxer::{QuicMuxer, Substream}; +pub use muxer::QuicMuxer; pub use transport::QuicTransport; pub use upgrade::Upgrade; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 82e5a6fd2b2..96898a01c2d 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -21,29 +21,30 @@ use crate::connection::{Connection, ConnectionEvent}; use crate::error::Error; +use futures::{AsyncRead, AsyncWrite}; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ collections::{HashMap, VecDeque}, - fmt, + fmt, io, + pin::Pin, + sync::{Arc, Weak}, task::{Context, Poll, Waker}, }; -pub type Substream = quinn_proto::StreamId; - /// State for a single opened QUIC connection. pub struct QuicMuxer { // Note: This could theoretically be an asynchronous future, in order to yield the current // task if a task running in parallel is already holding the lock. However, using asynchronous // mutexes without async/await is extremely tedious and maybe not worth the effort. - inner: Mutex, + inner: Arc>, } /// Mutex-protected fields of [`QuicMuxer`]. struct QuicMuxerInner { /// Inner connection object that yields events. connection: Connection, - /// State of all the substreams that the muxer reports as open. + // /// State of all the substreams that the muxer reports as open. substreams: HashMap, /// A FIFO of wakers to wake if a new outgoing substream is opened. pending_substreams: VecDeque, @@ -78,20 +79,201 @@ impl QuicMuxer { assert!(!connection.is_handshaking()); QuicMuxer { - inner: Mutex::new(QuicMuxerInner { - connection, + inner: Arc::new(Mutex::new(QuicMuxerInner { + connection: connection, substreams: Default::default(), pending_substreams: Default::default(), poll_close_waker: None, poll_event_waker: None, - }), + })), + } + } +} + +pub struct Substream { + id: quinn_proto::StreamId, + muxer: Weak>, +} + +impl Substream { + fn new(id: quinn_proto::StreamId, muxer: Arc>) -> Self { + Self { + id, + muxer: Arc::downgrade(&muxer), + } + } +} + +impl Drop for Substream { + fn drop(&mut self) { + if let Some(muxer) = self.muxer.upgrade() { + let mut muxer = muxer.lock(); + muxer.substreams.remove(&self.id); + } + } +} + +impl AsyncRead for Substream { + fn poll_read( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + mut buf: &mut [u8], + ) -> Poll> { + use quinn_proto::{ReadError, ReadableError}; + use std::io::Write; + + let muxer = self + .muxer + .upgrade() + .expect("StreamMuxer::read_substream: muxer is dead"); + let mut muxer = muxer.lock(); + + let substream_state = muxer + .substreams + .get(&self.id) + .expect("invalid Substream::poll_read API usage"); + if substream_state.stopped { + return Poll::Ready(Ok(0)); + } + + let mut stream = muxer.connection.connection.recv_stream(self.id); + let mut chunks = match stream.read(true) { + Ok(chunks) => chunks, + Err(ReadableError::UnknownStream) => { + return Poll::Ready(Ok(0)); // FIXME This is a hack, + // a rust client should close substream correctly + // return Poll::Ready(Err(Self::Error::ExpiredStream)) + } + 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::error!( + "substream {} was reset with error code {}", + self.id, + error_code + ); + bytes = 0; + break; + } + Err(ReadError::Blocked) => { + pending = true; + break; + } + } + } + if chunks.finalize().should_transmit() { + if let Some(waker) = muxer.poll_event_waker.take() { + waker.wake(); + } + } + if pending && bytes == 0 { + let mut substream_state = muxer + .substreams + .get_mut(&self.id) + .expect("known substream; qed"); + substream_state.read_waker = Some(cx.waker().clone()); + Poll::Pending + } else { + Poll::Ready(Ok(bytes)) + } + } +} + +impl AsyncWrite for Substream { + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + use quinn_proto::WriteError; + + let muxer = self + .muxer + .upgrade() + .expect("Substream::poll_write: muxer is dead"); + let mut muxer = muxer.lock(); + + match muxer.connection.connection.send_stream(self.id).write(buf) { + Ok(bytes) => Poll::Ready(Ok(bytes)), + Err(WriteError::Blocked) => { + let mut substream = muxer + .substreams + .get_mut(&self.id) + .expect("known substream; qed"); + substream.write_waker = Some(cx.waker().clone()); + Poll::Pending + } + Err(err @ WriteError::Stopped(_)) => { + Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) + } + Err(WriteError::UnknownStream) => { + tracing::error!( + "The application used a connection that is already being \ + closed. This is a bug in the application or in libp2p." + ); + Poll::Pending + } + } + } + + fn poll_flush(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + // quinn doesn't support flushing, calling close will flush all substreams. + Poll::Ready(Ok(())) + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let muxer = self + .muxer + .upgrade() + .expect("Substream::poll_close: muxer is dead"); + let mut muxer = muxer.lock(); + let muxer = &mut *muxer; + + let mut substream_state = muxer + .substreams + .get_mut(&self.id) + .expect("invalid Substream::poll_close API usage"); + if substream_state.finished { + return Poll::Ready(Ok(())); + } + + match muxer.connection.shutdown_substream(self.id) { + Ok(()) => { + substream_state.finished_waker = Some(cx.waker().clone()); + Poll::Pending + } + Err(err @ quinn_proto::FinishError::Stopped(_)) => { + Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) + } + Err(quinn_proto::FinishError::UnknownStream) => { + // Illegal usage of the API. + debug_assert!(false); + Poll::Ready(Ok(())) + // Poll::Ready(Err(Error::ExpiredStream)) FIXME + } } } } impl StreamMuxer for QuicMuxer { type OutboundSubstream = (); - type Substream = quinn_proto::StreamId; + type Substream = Substream; type Error = Error; /// Polls for a connection-wide event. @@ -163,8 +345,9 @@ impl StreamMuxer for QuicMuxer { } } - if let Some(substream) = inner.connection.pop_incoming_substream() { - inner.substreams.insert(substream, Default::default()); + if let Some(substream_id) = inner.connection.pop_incoming_substream() { + inner.substreams.insert(substream_id, Default::default()); + let substream = Substream::new(substream_id, self.inner.clone()); Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) } else { inner.poll_event_waker = Some(cx.waker().clone()); @@ -189,8 +372,9 @@ impl StreamMuxer for QuicMuxer { _: &mut Self::OutboundSubstream, ) -> Poll> { let mut inner = self.inner.lock(); - if let Some(substream) = inner.connection.pop_outgoing_substream() { - inner.substreams.insert(substream, Default::default()); + if let Some(substream_id) = inner.connection.pop_outgoing_substream() { + inner.substreams.insert(substream_id, Default::default()); + let substream = Substream::new(substream_id, self.inner.clone()); Poll::Ready(Ok(substream)) } else { inner.pending_substreams.push_back(cx.waker().clone()); @@ -205,174 +389,6 @@ impl StreamMuxer for QuicMuxer { // TODO `Self::OutboundSubstream` -> autoincrement id. } - /// Write data to a substream. The behaviour is the same as `futures::AsyncWrite::poll_write`. - /// - /// If `Pending` is returned, then the current task will be notified once the substream - /// is ready to be read. For each individual substream, only the latest task that was used to - /// call this method may be notified. - /// - /// Calling `write_substream` does not guarantee that data will arrive to the remote. To - /// ensure that, you should call `flush_substream`. - /// - /// It is incorrect to call this method on a substream if you called `shutdown_substream` on - /// this substream earlier. - fn write_substream( - &self, - cx: &mut Context<'_>, - substream: &mut Self::Substream, - buf: &[u8], - ) -> Poll> { - use quinn_proto::WriteError; - - let mut inner = self.inner.lock(); - - let id = substream; - - match inner.connection.connection.send_stream(*id).write(buf) { - Ok(bytes) => Poll::Ready(Ok(bytes)), - Err(WriteError::Blocked) => { - let mut substream = inner.substreams.get_mut(id).expect("known substream; qed"); - substream.write_waker = Some(cx.waker().clone()); - Poll::Pending - } - Err(WriteError::Stopped(err_code)) => Poll::Ready(Err(Error::Reset(err_code))), - Err(WriteError::UnknownStream) => { - tracing::error!( - "The application used a connection that is already being \ - closed. This is a bug in the application or in libp2p." - ); - Poll::Pending - } - } - } - - /// Reads data from a substream. The behaviour is the same as `futures::AsyncRead::poll_read`. - /// - /// If `Pending` is returned, then the current task will be notified once the substream - /// is ready to be read. However, for each individual substream, only the latest task that - /// was used to call this method may be notified. - /// - /// If `Async::Ready(0)` is returned, the substream has been closed by the remote and should - /// no longer be read afterwards. - /// - /// An error can be generated if the connection has been closed, or if a protocol misbehaviour - /// happened. - fn read_substream( - &self, - cx: &mut Context<'_>, - substream: &mut Self::Substream, - mut buf: &mut [u8], - ) -> Poll> { - use quinn_proto::{ReadError, ReadableError}; - use std::io::Write; - - let id = *substream; - - let mut inner = self.inner.lock(); - - let substream_state = inner - .substreams - .get_mut(substream) - .expect("invalid StreamMuxer::read_substream API usage"); - if substream_state.stopped { - return Poll::Ready(Ok(0)); - } - - let mut stream = inner.connection.connection.recv_stream(id); - let mut chunks = match stream.read(true) { - Ok(chunks) => chunks, - Err(ReadableError::UnknownStream) => { - return Poll::Ready(Ok(0)); // FIXME This is a hack, - // a rust client should close substream correctly - // return Poll::Ready(Err(Self::Error::ExpiredStream)) - } - 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::error!("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.poll_event_waker.take() { - waker.wake(); - } - } - let substream = inner.substreams.get_mut(&id).expect("known substream; qed"); - if pending && bytes == 0 { - substream.read_waker = Some(cx.waker().clone()); - Poll::Pending - } else { - Poll::Ready(Ok(bytes)) - } - } - - fn shutdown_substream( - &self, - cx: &mut Context<'_>, - substream: &mut Self::Substream, - ) -> Poll> { - let mut inner = self.inner.lock(); - let inner = &mut *inner; - - let mut substream_state = inner - .substreams - .get_mut(substream) - .expect("invalid StreamMuxer::shutdown_substream API usage"); - if substream_state.finished { - return Poll::Ready(Ok(())); - } - - match inner.connection.shutdown_substream(*substream) { - Ok(()) => { - substream_state.finished_waker = Some(cx.waker().clone()); - Poll::Pending - } - Err(quinn_proto::FinishError::Stopped(err)) => Poll::Ready(Err(Error::Reset(err))), - Err(quinn_proto::FinishError::UnknownStream) => { - // Illegal usage of the API. - debug_assert!(false); - Poll::Ready(Err(Error::ExpiredStream)) - } - } - } - - fn destroy_substream(&self, substream: Self::Substream) { - let mut inner = self.inner.lock(); - inner.substreams.remove(&substream); - } - - fn flush_substream( - &self, - _: &mut Context<'_>, - _: &mut Self::Substream, - ) -> Poll> { - // quinn doesn't support flushing, calling close will flush all substreams. - Poll::Ready(Ok(())) - } - fn poll_close(&self, cx: &mut Context<'_>) -> Poll> { let mut inner = self.inner.lock(); From 5157ea14a69513c17a3bb083d82ae05e5a84937d Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Mon, 27 Jun 2022 12:04:14 +0200 Subject: [PATCH 061/218] transports/quic: adapt to transport trait changes Adapt to the transport changes of libp2p#2652. Note: this is only a draft "to make it work", and not a proper implementation. It does not support listening on multiple addresses. The listening logic with multiple Endpoints will need to be supported for the upstream implementation. --- transports/quic/src/transport.rs | 89 +++++++++++++++++++------------- 1 file changed, 54 insertions(+), 35 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 6c06c38a6ff..54dcca5e342 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -31,7 +31,7 @@ use if_watch::IfEvent; use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, - transport::{ListenerEvent, TransportError}, + transport::{ListenerId, TransportError, TransportEvent}, PeerId, Transport, }; use std::task::{Context, Poll}; @@ -52,18 +52,16 @@ pub use quinn_proto::{ #[derive(Debug, Clone)] pub struct QuicTransport { endpoint: Arc, - /// The IP addresses of network interfaces on which the listening socket - /// is accepting connections. - /// - /// If the listen socket listens on all interfaces, these may change over - /// time as interfaces become available or unavailable. - in_addr: InAddr, + + listener: Option<(ListenerId, InAddr)>, } impl QuicTransport { pub fn new(endpoint: Arc) -> Self { - let in_addr = InAddr::new(endpoint.local_addr.ip()); - Self { endpoint, in_addr } + Self { + endpoint, + listener: None + } } } @@ -84,23 +82,27 @@ pub enum Error { impl Transport for QuicTransport { type Output = (PeerId, QuicMuxer); type Error = Error; - // type Listener = Pin< - // Box, Self::Error>> + Send>, - // >; - type Listener = Self; type ListenerUpgrade = Upgrade; type Dial = Pin> + Send>>; - fn listen_on( - &mut self, - addr: Multiaddr, - ) -> Result> { + fn listen_on(&mut self, addr: Multiaddr) -> Result> { multiaddr_to_socketaddr(&addr) .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; - Ok(self.clone()) + let listener = self.listener.get_or_insert((ListenerId::new(), InAddr::new(self.endpoint.local_addr.ip()))); + Ok(listener.0) } - fn address_translation(&self, server: &Multiaddr, observed: &Multiaddr) -> Option { + fn remove_listener(&mut self, id: ListenerId) -> bool { + if let Some((listener_id, _)) = self.listener { + if id == listener_id { + self.listener = None; + return true + } + } + false + } + + fn address_translation(&self, _server: &Multiaddr, observed: &Multiaddr) -> Option { Some(observed.clone()) } @@ -136,17 +138,21 @@ impl Transport for QuicTransport { // https://github.com/libp2p/specs/blob/master/relay/DCUtR.md#the-protocol self.dial(addr) } -} - -impl Stream for QuicTransport { - type Item = Result, Error>; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { + fn poll( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { let me = Pin::into_inner(self); + // Poll for a next IfEvent + let (listener_id, in_addr) = match me.listener.as_mut() { + Some((id, in_addr)) => (*id, in_addr), + None => return Poll::Pending + }; let endpoint = me.endpoint.as_ref(); // Poll for a next IfEvent - match me.in_addr.poll_next_unpin(cx) { + match in_addr.poll_next_unpin(cx) { Poll::Ready(mut item) => { if let Some(item) = item.take() { // Consume all events for up/down interface changes. @@ -157,7 +163,10 @@ impl Stream for QuicTransport { let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("New listen address: {}", ma); - return Poll::Ready(Some(Ok(ListenerEvent::NewAddress(ma)))); + return Poll::Ready(TransportEvent::NewAddress { + listener_id, + listen_addr: ma, + }); } } Ok(IfEvent::Down(inet)) => { @@ -166,7 +175,10 @@ impl Stream for QuicTransport { let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("Expired listen address: {}", ma); - return Poll::Ready(Some(Ok(ListenerEvent::AddressExpired(ma)))); + return Poll::Ready(TransportEvent::AddressExpired { + listener_id, + listen_addr: ma, + }); } } Err(err) => { @@ -174,9 +186,10 @@ impl Stream for QuicTransport { "Failure polling interfaces: {:?}.", err }; - return Poll::Ready(Some(Ok(ListenerEvent::Error(Error::IfWatcher( - err, - ))))); + return Poll::Ready(TransportEvent::ListenerError { + listener_id, + error: Error::IfWatcher(err), + }); } } } @@ -188,17 +201,23 @@ impl Stream for QuicTransport { let connection = match endpoint.poll_incoming(cx) { Poll::Ready(Some(connection)) => connection, - Poll::Ready(None) => return Poll::Ready(None), + Poll::Ready(None) => { + return Poll::Ready(TransportEvent::ListenerClosed { + listener_id, + reason: Ok(()), + }) + } Poll::Pending => return Poll::Pending, }; let local_addr = socketaddr_to_multiaddr(&connection.local_addr()); - let remote_addr = socketaddr_to_multiaddr(&connection.remote_addr()); - let event = ListenerEvent::Upgrade { + let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); + let event = TransportEvent::Incoming { upgrade: Upgrade::from_connection(connection), local_addr, - remote_addr, + send_back_addr, + listener_id, }; - Poll::Ready(Some(Ok(event))) + Poll::Ready(event) } } From 57743ef2284e07e406b093729044e7e14baf2af1 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Mon, 4 Jul 2022 13:01:10 +0200 Subject: [PATCH 062/218] transports/quic: support multiple listening endpoints --- transports/quic/src/connection.rs | 30 ++-- transports/quic/src/endpoint.rs | 64 +++----- transports/quic/src/lib.rs | 2 +- transports/quic/src/transport.rs | 239 ++++++++++++++++++++---------- transports/quic/tests/smoke.rs | 20 +-- 5 files changed, 204 insertions(+), 151 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index f91cd9af127..cc883a25b7b 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -44,7 +44,7 @@ use std::{ /// /// Contains everything needed to process a connection with a remote. /// Tied to a specific [`crate::Endpoint`]. -pub(crate) struct Connection { +pub struct Connection { /// Endpoint this connection belongs to. endpoint: Arc, /// Future whose job is to send a message to the endpoint. Only one at a time. @@ -54,7 +54,7 @@ pub(crate) struct Connection { from_endpoint: mpsc::Receiver, /// The QUIC state machine for this specific connection. - pub(crate) connection: quinn_proto::Connection, + pub connection: quinn_proto::Connection, /// Identifier for this connection according to the endpoint. Used when sending messages to /// the endpoint. connection_id: quinn_proto::ConnectionHandle, @@ -100,7 +100,7 @@ impl Connection { /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of /// its methods has ever been called. Failure to comply might lead to logic errors and panics. // TODO: maybe abstract `to_endpoint` more and make it generic? dunno - pub(crate) fn from_quinn_connection( + pub fn from_quinn_connection( endpoint: Arc, connection: quinn_proto::Connection, connection_id: quinn_proto::ConnectionHandle, @@ -124,9 +124,9 @@ impl Connection { /// The local address which was used when the peer established the connection. /// /// Works for server connections only. - pub(crate) fn local_addr(&self) -> SocketAddr { + pub fn local_addr(&self) -> SocketAddr { debug_assert_eq!(self.connection.side(), quinn_proto::Side::Server); - let endpoint_addr = self.endpoint.local_addr; + let endpoint_addr = self.endpoint.socket_addr(); self.connection .local_ip() .map(|ip| SocketAddr::new(ip, endpoint_addr.port())) @@ -134,25 +134,25 @@ impl Connection { // In a normal case scenario this should not happen, because // we get want to get a local addr for a server connection only. tracing::error!("trying to get quinn::local_ip for a client"); - endpoint_addr + endpoint_addr.clone() }) } /// Returns the address of the node we're connected to. // TODO: can change /!\ - pub(crate) fn remote_addr(&self) -> SocketAddr { + pub fn remote_addr(&self) -> SocketAddr { self.connection.remote_address() } /// Returns `true` if this connection is still pending. Returns `false` if we are connected to /// the remote or if the connection is closed. - pub(crate) fn is_handshaking(&self) -> bool { + pub fn is_handshaking(&self) -> bool { self.is_handshaking } /// Returns the address of the node we're connected to. /// Panics if the connection is still handshaking. - pub(crate) fn remote_peer_id(&self) -> PeerId { + pub fn remote_peer_id(&self) -> PeerId { debug_assert!(!self.is_handshaking()); let session = self.connection.crypto_session(); let identity = session @@ -171,7 +171,7 @@ impl Connection { /// Start closing the connection. A [`ConnectionEvent::ConnectionLost`] event will be /// produced in the future. - pub(crate) fn close(&mut self) { + pub fn close(&mut self) { // TODO: what if the user calls this multiple times? // We send a dummy `0` error code with no message, as the API of StreamMuxer doesn't // support this. @@ -187,7 +187,7 @@ impl Connection { /// /// If `None` is returned, then a [`ConnectionEvent::StreamAvailable`] event will later be /// produced when a substream is available. - pub(crate) fn pop_incoming_substream(&mut self) -> Option { + pub fn pop_incoming_substream(&mut self) -> Option { self.connection.streams().accept(quinn_proto::Dir::Bi) } @@ -198,7 +198,7 @@ impl Connection { /// /// If `None` is returned, then a [`ConnectionEvent::StreamOpened`] event will later be /// produced when a substream is available. - pub(crate) fn pop_outgoing_substream(&mut self) -> Option { + pub fn pop_outgoing_substream(&mut self) -> Option { self.connection.streams().open(quinn_proto::Dir::Bi) } @@ -210,7 +210,7 @@ impl Connection { /// On success, a [`quinn_proto::StreamEvent::Finished`] event will later be produced when the /// substream has been effectively closed. A [`ConnectionEvent::StreamStopped`] event can also /// be emitted. - pub(crate) fn shutdown_substream( + pub fn shutdown_substream( &mut self, id: quinn_proto::StreamId, ) -> Result<(), quinn_proto::FinishError> { @@ -220,7 +220,7 @@ impl Connection { } /// Polls the connection for an event that happend on it. - pub(crate) fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { + pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { // Nothing more can be done if the connection is closed. // Return `Pending` without registering the waker, essentially freezing the task forever. if self.closed.is_some() { @@ -399,7 +399,7 @@ impl Drop for Connection { /// Event generated by the [`Connection`]. #[derive(Debug)] -pub(crate) enum ConnectionEvent { +pub enum ConnectionEvent { /// Now connected to the remote and certificates are available. Connected, diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 6d0d3ceaa94..e474ea54dd8 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -36,16 +36,13 @@ use futures::{ channel::{mpsc, oneshot}, lock::Mutex, prelude::*, - stream::Stream, }; -use libp2p_core::multiaddr::Multiaddr; use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; use std::{ collections::{HashMap, VecDeque}, fmt, io, - pin::Pin, sync::{Arc, Weak}, - task::{Context, Poll}, + task::Poll, time::{Duration, Instant}, }; @@ -58,16 +55,11 @@ pub struct Config { server_config: Arc, /// The endpoint configuration to pass to `quinn_proto`. endpoint_config: Arc, - /// The [`Multiaddr`] to use to spawn the UDP socket. - multiaddr: Multiaddr, } impl Config { /// Creates a new configuration object with default values. - pub fn new( - keypair: &libp2p_core::identity::Keypair, - multiaddr: Multiaddr, - ) -> Result { + pub fn new(keypair: &libp2p_core::identity::Keypair) -> Result { let mut transport = quinn_proto::TransportConfig::default(); transport.max_concurrent_uni_streams(0u32.into()); // Can only panic if value is out of range. transport.datagram_receive_buffer_size(None); @@ -86,7 +78,6 @@ impl Config { client_config, server_config: Arc::new(server_config), endpoint_config: Default::default(), - multiaddr, }) } } @@ -100,32 +91,21 @@ pub struct Endpoint { /// See [`Endpoint::new_connections`] (just below) for a commentary about the mutex. to_endpoint: Mutex>, - /// Channel where new connections are being sent. - /// This is protected by a futures-friendly `Mutex`, meaning that receiving a connection is - /// done in two steps: locking this mutex, and grabbing the next element on the `Receiver`. - /// The only consequence of this `Mutex` is that multiple simultaneous calls to - /// [`Endpoint::poll_incoming`] are serialized. - new_connections: Mutex>, - /// Copy of [`Endpoint::to_endpoint`], except not behind a `Mutex`. Used if we want to be /// guaranteed a slot in the messages buffer. to_endpoint2: mpsc::Sender, - /// Socketaddr of the local UDP socket passed in the configuration at initialization after it - /// has potentially been modified to handle port number `0`. - pub(crate) local_addr: SocketAddr, + socket_addr: SocketAddr, } impl Endpoint { /// Builds a new `Endpoint`. - pub fn new(config: Config) -> Result, io::Error> { - let local_socket_addr = match crate::transport::multiaddr_to_socketaddr(&config.multiaddr) { - Some(a) => a, - None => panic!(), // TODO: Err(TransportError::MultiaddrNotSupported(multiaddr)), - }; - + pub fn new( + config: Config, + socket_addr: SocketAddr, + ) -> Result<(Arc, mpsc::Receiver), io::Error> { // NOT blocking, as per man:bind(2), as we pass an IP address. - let socket = std::net::UdpSocket::bind(&local_socket_addr)?; + let socket = std::net::UdpSocket::bind(&socket_addr)?; // TODO: /*let port_is_zero = local_socket_addr.port() == 0; let local_socket_addr = socket.local_addr()?; @@ -144,8 +124,7 @@ impl Endpoint { let endpoint = Arc::new(Endpoint { to_endpoint: Mutex::new(to_endpoint_tx), to_endpoint2, - new_connections: Mutex::new(new_connections_rx), - local_addr: socket.local_addr()?, + socket_addr, }); // TODO: just for testing, do proper task spawning @@ -158,17 +137,18 @@ impl Endpoint { )) .detach(); - Ok(endpoint) + Ok((endpoint, new_connections_rx)) + } + + pub fn socket_addr(&self) -> &SocketAddr { + &self.socket_addr } /// Asks the endpoint to start dialing the given address. /// /// Note that this method only *starts* the dialing. `Ok` is returned as soon as possible, even /// when the remote might end up being unreachable. - pub(crate) async fn dial( - &self, - addr: SocketAddr, - ) -> Result { + pub async fn dial(&self, addr: SocketAddr) -> Result { // The two `expect`s below can panic if the background task has stopped. The background // task can stop only if the `Endpoint` is destroyed or if the task itself panics. In other // words, we panic here iff a panic has already happened somewhere else, which is a @@ -183,19 +163,12 @@ impl Endpoint { rx.await.expect("background task has crashed") } - /// Tries to pop a new incoming connection from the queue. - pub(crate) fn poll_incoming(&self, cx: &mut Context) -> Poll> { - let mut connections_lock = self.new_connections.lock(); - let mut guard = futures::ready!(Pin::new(&mut connections_lock).poll(cx)); - Pin::new(&mut *guard).poll_next(cx) - } - /// Asks the endpoint to send a UDP packet. /// /// Note that this method only queues the packet and returns as soon as the packet is in queue. /// There is no guarantee that the packet will actually be sent, but considering that this is /// a UDP packet, you cannot rely on the packet being delivered anyway. - pub(crate) async fn send_udp_packet(&self, destination: SocketAddr, data: impl Into>) { + pub async fn send_udp_packet(&self, destination: SocketAddr, data: impl Into>) { let _ = self .to_endpoint .lock() @@ -213,7 +186,7 @@ impl Endpoint { /// /// If `event.is_drained()` is true, the event indicates that the connection no longer exists. /// This must therefore be the last event sent using this [`quinn_proto::ConnectionHandle`]. - pub(crate) async fn report_quinn_event( + pub async fn report_quinn_event( &self, connection_id: quinn_proto::ConnectionHandle, event: quinn_proto::EndpointEvent, @@ -234,7 +207,7 @@ impl Endpoint { /// /// This method bypasses back-pressure mechanisms and is meant to be called only from /// destructors, where waiting is not advisable. - pub(crate) fn report_quinn_event_non_block( + pub fn report_quinn_event_non_block( &self, connection_id: quinn_proto::ConnectionHandle, event: quinn_proto::EndpointEvent, @@ -251,7 +224,6 @@ impl Endpoint { assert!(result.is_ok()); } } - /// Message sent to the endpoint background task. #[derive(Debug)] enum ToEndpoint { diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 116d459bcd6..51d3e4df1fb 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -61,7 +61,7 @@ mod upgrade; pub mod transport; -pub use endpoint::{Config, Endpoint}; +pub use endpoint::Config; pub use error::Error; pub use muxer::QuicMuxer; pub use transport::QuicTransport; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 54dcca5e342..f60afdc5f59 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -22,10 +22,12 @@ //! //! Combines all the objects in the other modules to implement the trait. +use crate::connection::Connection; +use crate::Config; use crate::{endpoint::Endpoint, in_addr::InAddr, muxer::QuicMuxer, upgrade::Upgrade}; -use futures::prelude::*; use futures::stream::StreamExt; +use futures::{channel::mpsc, prelude::*, stream::SelectAll}; use if_watch::IfEvent; @@ -34,8 +36,12 @@ use libp2p_core::{ transport::{ListenerId, TransportError, TransportEvent}, PeerId, Transport, }; -use std::task::{Context, Poll}; -use std::{net::SocketAddr, pin::Pin, sync::Arc}; +use std::{ + net::SocketAddr, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; // We reexport the errors that are exposed in the API. // All of these types use one another. @@ -45,22 +51,17 @@ pub use quinn_proto::{ TransportError as QuinnTransportError, TransportErrorCode, }; -/// Wraps around an `Arc` and implements the [`Transport`] trait. -/// -/// > **Note**: This type is necessary because Rust unfortunately forbids implementing the -/// > `Transport` trait directly on `Arc`. -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct QuicTransport { - endpoint: Arc, - - listener: Option<(ListenerId, InAddr)>, + config: Config, + listeners: SelectAll, } impl QuicTransport { - pub fn new(endpoint: Arc) -> Self { + pub fn new(config: Config) -> Self { Self { - endpoint, - listener: None + listeners: SelectAll::new(), + config, } } } @@ -77,6 +78,12 @@ pub enum Error { /// Error while working with IfWatcher. #[error("{0}")] IfWatcher(std::io::Error), + + #[error("{0}")] + Socket(std::io::Error), + + #[error("Background task crashed.")] + TaskCrashed, } impl Transport for QuicTransport { @@ -86,20 +93,24 @@ impl Transport for QuicTransport { type Dial = Pin> + Send>>; fn listen_on(&mut self, addr: Multiaddr) -> Result> { - multiaddr_to_socketaddr(&addr) + let socket_addr = multiaddr_to_socketaddr(&addr) .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; - let listener = self.listener.get_or_insert((ListenerId::new(), InAddr::new(self.endpoint.local_addr.ip()))); - Ok(listener.0) + let in_addr = InAddr::new(socket_addr.ip()); + let (endpoint, new_connections_rx) = Endpoint::new(self.config.clone(), socket_addr) + .map_err(|e| TransportError::Other(Error::Socket(e)))?; + let listener_id = ListenerId::new(); + let listener = Listener::new(listener_id, endpoint, new_connections_rx, in_addr); + self.listeners.push(listener); + Ok(listener_id) } fn remove_listener(&mut self, id: ListenerId) -> bool { - if let Some((listener_id, _)) = self.listener { - if id == listener_id { - self.listener = None; - return true - } + if let Some(listener) = self.listeners.iter_mut().find(|l| l.listener_id == id) { + listener.close(Ok(())); + true + } else { + false } - false } fn address_translation(&self, _server: &Multiaddr, observed: &Multiaddr) -> Option { @@ -107,25 +118,26 @@ impl Transport for QuicTransport { } fn dial(&mut self, addr: Multiaddr) -> Result> { - let socket_addr = if let Some(socket_addr) = multiaddr_to_socketaddr(&addr) { - if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { - tracing::error!("multiaddr not supported"); - return Err(TransportError::MultiaddrNotSupported(addr)); - } - socket_addr - } else { - tracing::error!("multiaddr not supported"); - return Err(TransportError::MultiaddrNotSupported(addr)); - }; + todo!() + // let socket_addr = if let Some(socket_addr) = multiaddr_to_socketaddr(&addr) { + // if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { + // tracing::error!("multiaddr not supported"); + // return Err(TransportError::MultiaddrNotSupported(addr)); + // } + // socket_addr + // } else { + // tracing::error!("multiaddr not supported"); + // return Err(TransportError::MultiaddrNotSupported(addr)); + // }; - let endpoint = self.endpoint.clone(); + // let endpoint = self.endpoint.clone(); - Ok(async move { - let connection = endpoint.dial(socket_addr).await.map_err(Error::Reach)?; - let final_connec = Upgrade::from_connection(connection).await?; - Ok(final_connec) - } - .boxed()) + // Ok(async move { + // let connection = endpoint.dial(socket_addr).await.map_err(Error::Reach)?; + // let final_connec = Upgrade::from_connection(connection).await?; + // Ok(final_connec) + // } + // .boxed()) } fn dial_as_listener( @@ -140,45 +152,106 @@ impl Transport for QuicTransport { } fn poll( - self: Pin<&mut Self>, + mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - let me = Pin::into_inner(self); - // Poll for a next IfEvent - let (listener_id, in_addr) = match me.listener.as_mut() { - Some((id, in_addr)) => (*id, in_addr), - None => return Poll::Pending - }; - let endpoint = me.endpoint.as_ref(); + match self.listeners.poll_next_unpin(cx) { + Poll::Ready(Some(ev)) => Poll::Ready(ev), + _ => Poll::Pending, + } + } +} + +#[derive(Debug)] +struct Listener { + endpoint: Arc, + + listener_id: ListenerId, - // Poll for a next IfEvent - match in_addr.poll_next_unpin(cx) { + /// Channel where new connections are being sent. + new_connections: mpsc::Receiver, + + /// The IP addresses of network interfaces on which the listening socket + /// is accepting connections. + /// + /// If the listen socket listens on all interfaces, these may change over + /// time as interfaces become available or unavailable. + in_addr: InAddr, + + /// Set to `Some` if this [`Listener`] should close. + /// Optionally contains a [`TransportEvent::ListenerClosed`] that should be + /// reported before the listener's stream is terminated. + report_closed: Option::Item>>, +} + +impl Listener { + fn new( + listener_id: ListenerId, + endpoint: Arc, + new_connections: mpsc::Receiver, + in_addr: InAddr, + ) -> Self { + Listener { + endpoint, + listener_id, + new_connections, + in_addr, + report_closed: None, + } + } + + /// Report the listener as closed in a [`TransportEvent::ListenerClosed`] and + /// terminate the stream. + fn close(&mut self, reason: Result<(), Error>) { + match self.report_closed { + Some(_) => tracing::debug!("Listener was already closed."), + None => { + // Report the listener event as closed. + let _ = self + .report_closed + .insert(Some(TransportEvent::ListenerClosed { + listener_id: self.listener_id, + reason, + })); + } + } + } + + /// Poll for a next If Event. + fn poll_if_addr(&mut self, cx: &mut Context<'_>) -> Option<::Item> { + match self.in_addr.poll_next_unpin(cx) { Poll::Ready(mut item) => { if let Some(item) = item.take() { // Consume all events for up/down interface changes. match item { Ok(IfEvent::Up(inet)) => { let ip = inet.addr(); - if endpoint.local_addr.is_ipv4() == ip.is_ipv4() { - let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); + if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + let socket_addr = + SocketAddr::new(ip, self.endpoint.socket_addr().port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("New listen address: {}", ma); - return Poll::Ready(TransportEvent::NewAddress { - listener_id, + Some(TransportEvent::NewAddress { + listener_id: self.listener_id, listen_addr: ma, - }); + }) + } else { + self.poll_if_addr(cx) } } Ok(IfEvent::Down(inet)) => { let ip = inet.addr(); - if endpoint.local_addr.is_ipv4() == ip.is_ipv4() { - let socket_addr = SocketAddr::new(ip, endpoint.local_addr.port()); + if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + let socket_addr = + SocketAddr::new(ip, self.endpoint.socket_addr().port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("Expired listen address: {}", ma); - return Poll::Ready(TransportEvent::AddressExpired { - listener_id, + Some(TransportEvent::AddressExpired { + listener_id: self.listener_id, listen_addr: ma, - }); + }) + } else { + self.poll_if_addr(cx) } } Err(err) => { @@ -186,44 +259,56 @@ impl Transport for QuicTransport { "Failure polling interfaces: {:?}.", err }; - return Poll::Ready(TransportEvent::ListenerError { - listener_id, + Some(TransportEvent::ListenerError { + listener_id: self.listener_id, error: Error::IfWatcher(err), - }); + }) } } + } else { + self.poll_if_addr(cx) } } - Poll::Pending => { - // continue polling endpoint - } + Poll::Pending => None, } + } +} - let connection = match endpoint.poll_incoming(cx) { - Poll::Ready(Some(connection)) => connection, - Poll::Ready(None) => { - return Poll::Ready(TransportEvent::ListenerClosed { - listener_id, - reason: Ok(()), - }) +impl Stream for Listener { + type Item = TransportEvent<::ListenerUpgrade, Error>; + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + if let Some(closed) = self.report_closed.as_mut() { + // Listener was closed. + // Report the transport event if there is one. On the next iteration, return + // `Poll::Ready(None)` to terminate the stream. + return Poll::Ready(closed.take()); + } + if let Some(event) = self.poll_if_addr(cx) { + return Poll::Ready(Some(event)); + } + let connection = match futures::ready!(self.new_connections.poll_next_unpin(cx)) { + Some(c) => c, + None => { + self.close(Err(Error::TaskCrashed)); + return self.poll_next(cx); } - Poll::Pending => return Poll::Pending, }; + let local_addr = socketaddr_to_multiaddr(&connection.local_addr()); let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); let event = TransportEvent::Incoming { upgrade: Upgrade::from_connection(connection), local_addr, send_back_addr, - listener_id, + listener_id: self.listener_id, }; - Poll::Ready(event) + Poll::Ready(Some(event)) } } /// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns None if the format /// of the multiaddr is wrong. -pub(crate) fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { +pub fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { let mut iter = addr.iter(); let proto1 = iter.next()?; let proto2 = iter.next()?; diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 92ec9fccbe7..c35f8138674 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -3,21 +3,19 @@ use async_trait::async_trait; use futures::future::FutureExt; use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; use futures::stream::StreamExt; -use libp2p::core::upgrade; +use futures::task::Spawn; +use libp2p::core::muxing::StreamMuxerBox; +use libp2p::core::{upgrade, Transport}; use libp2p::request_response::{ ProtocolName, ProtocolSupport, RequestResponse, RequestResponseCodec, RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, }; -use libp2p::swarm::{Swarm, SwarmBuilder, SwarmEvent}; -use libp2p::{Multiaddr, Transport}; -use libp2p_core::muxing::StreamMuxerBox; -use libp2p_quic::{Config as QuicConfig, Endpoint as QuicEndpoint, QuicTransport}; +use libp2p::swarm::{Swarm, SwarmEvent}; +use libp2p_quic::{Config as QuicConfig, QuicTransport}; use rand::RngCore; -use std::{io, iter}; - -use futures::task::Spawn; use std::num::NonZeroU8; use std::time::Duration; +use std::{io, iter}; fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() @@ -27,10 +25,8 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { async fn create_swarm(keylog: bool) -> Result>> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); - let addr: Multiaddr = "/ip4/127.0.0.1/udp/0/quic".parse()?; - let config = QuicConfig::new(&keypair, addr).unwrap(); - let endpoint = QuicEndpoint::new(config).unwrap(); - let transport = QuicTransport::new(endpoint); + let config = QuicConfig::new(&keypair).unwrap(); + let transport = QuicTransport::new(config); // TODO: // transport From e5e5b34553da6bc700267665307aa96cb0d0b24d Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Mon, 4 Jul 2022 17:10:06 +0200 Subject: [PATCH 063/218] transports/quic: re-use endpoints for dialing --- transports/quic/Cargo.toml | 1 + transports/quic/src/connection.rs | 2 +- transports/quic/src/endpoint.rs | 88 +++++++++++++++++----------- transports/quic/src/lib.rs | 9 +-- transports/quic/src/transport.rs | 95 ++++++++++++++++++------------- transports/quic/tests/smoke.rs | 1 - 6 files changed, 119 insertions(+), 77 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index a8dd90a9c55..9425209e68b 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -15,6 +15,7 @@ if-watch = "1.0.0" libp2p-core = { version = "0.34.0", path = "../../core" } parking_lot = "0.12.0" quinn-proto = { version = "0.8.2", default-features = false, features = ["tls-rustls"] } +rand = "0.8.5" rcgen = "0.9.2" ring = "0.16.20" rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index cc883a25b7b..0c9217b190a 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -134,7 +134,7 @@ impl Connection { // In a normal case scenario this should not happen, because // we get want to get a local addr for a server connection only. tracing::error!("trying to get quinn::local_ip for a client"); - endpoint_addr.clone() + *endpoint_addr }) } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index e474ea54dd8..59804a1bd96 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,9 +28,7 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`background_task`] for a thorough description. -use crate::{connection::Connection, tls}; - -use std::net::{SocketAddr, UdpSocket}; +use crate::{connection::Connection, tls, transport}; use futures::{ channel::{mpsc, oneshot}, @@ -40,9 +38,10 @@ use futures::{ use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; use std::{ collections::{HashMap, VecDeque}, - fmt, io, + fmt, + net::{Ipv4Addr, SocketAddr, SocketAddrV4, UdpSocket}, sync::{Arc, Weak}, - task::Poll, + task::{Poll, Waker}, time::{Duration, Instant}, }; @@ -99,45 +98,52 @@ pub struct Endpoint { } impl Endpoint { - /// Builds a new `Endpoint`. - pub fn new( + /// Builds a new `Endpoint` that is listening on the [`SocketAddr`]. + pub fn new_bidirectional( config: Config, socket_addr: SocketAddr, - ) -> Result<(Arc, mpsc::Receiver), io::Error> { + ) -> Result<(Arc, mpsc::Receiver), transport::Error> { + let (new_connections_tx, new_connections_rx) = mpsc::channel(1); + let endpoint = Self::new(config, socket_addr, Some(new_connections_tx))?; + Ok((endpoint, new_connections_rx)) + } + + /// Builds a new `Endpoint` that only supports outbound connections. + pub fn new_dialer(config: Config) -> Result, transport::Error> { + let socket_addr = SocketAddrV4::new(Ipv4Addr::LOCALHOST, 0); + Self::new(config, socket_addr.into(), None) + } + + fn new( + config: Config, + socket_addr: SocketAddr, + new_connections: Option>, + ) -> Result, transport::Error> { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; - // TODO: - /*let port_is_zero = local_socket_addr.port() == 0; - let local_socket_addr = socket.local_addr()?; - if port_is_zero { - assert_ne!(local_socket_addr.port(), 0); - assert_eq!(multiaddr.pop(), Some(Protocol::Quic)); - assert_eq!(multiaddr.pop(), Some(Protocol::Udp(0))); - multiaddr.push(Protocol::Udp(local_socket_addr.port())); - multiaddr.push(Protocol::Quic); - }*/ - let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(32); let to_endpoint2 = to_endpoint_tx.clone(); - let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Arc::new(Endpoint { to_endpoint: Mutex::new(to_endpoint_tx), to_endpoint2, - socket_addr, + socket_addr: socket.local_addr()?, }); + let server_config = new_connections.map(|c| (c, config.server_config.clone())); + // TODO: just for testing, do proper task spawning async_global_executor::spawn(background_task( - config, + config.endpoint_config, + config.client_config, + server_config, Arc::downgrade(&endpoint), async_io::Async::::new(socket)?, - new_connections_tx, to_endpoint_rx.fuse(), )) .detach(); - Ok((endpoint, new_connections_rx)) + Ok(endpoint) } pub fn socket_addr(&self) -> &SocketAddr { @@ -335,17 +341,20 @@ enum ToEndpoint { /// for as long as any QUIC connection is open. /// async fn background_task( - config: Config, + endpoint_config: Arc, + client_config: quinn_proto::ClientConfig, + server_config: Option<(mpsc::Sender, Arc)>, endpoint_weak: Weak, udp_socket: async_io::Async, - mut new_connections: mpsc::Sender, mut receiver: stream::Fuse>, ) { + let (mut new_connections, server_config) = match server_config { + Some((a, b)) => (Some(a), Some(b)), + None => (None, None), + }; + // The actual QUIC state machine. - let mut endpoint = quinn_proto::Endpoint::new( - config.endpoint_config.clone(), - Some(config.server_config.clone()), - ); + let mut endpoint = quinn_proto::Endpoint::new(endpoint_config.clone(), server_config); // List of all active connections, with a sender to notify them of events. let mut alive_connections = HashMap::>::new(); @@ -365,6 +374,8 @@ async fn background_task( // code below. let mut next_packet_out: Option<(SocketAddr, Vec)> = None; + let mut new_connection_waker: Option = None; + // Main loop of the task. loop { // Start by flushing `next_packet_out`. @@ -409,7 +420,7 @@ async fn background_task( // name. While we don't use domain names, the underlying rustls library // is based upon the assumption that we do. let (connection_id, connection) = - match endpoint.connect(config.client_config.clone(), addr, "l") { + match endpoint.connect(client_config.clone(), addr, "l") { Ok(c) => c, Err(err) => { let _ = result.send(Err(err)); @@ -474,8 +485,17 @@ async fn background_task( readiness = { let active = !queued_new_connections.is_empty(); let new_connections = &mut new_connections; + let new_connection_waker = &mut new_connection_waker; future::poll_fn(move |cx| { - if active { new_connections.poll_ready(cx) } else { Poll::Pending } + match new_connections.as_mut() { + Some(ref mut c) if active => { + c.poll_ready(cx) + } + _ => { + let _ = new_connection_waker.insert(cx.waker().clone()); + Poll::Pending + } + } }) .fuse() } => { @@ -487,6 +507,7 @@ async fn background_task( let elem = queued_new_connections.pop_front() .expect("if queue is empty, the future above is always Pending; qed"); + let new_connections = new_connections.as_mut().expect("in case of None, the future above is always Pending; qed"); new_connections.start_send(elem) .expect("future is waken up only if poll_ready returned Ready; qed"); //endpoint.accept(); @@ -537,6 +558,9 @@ async fn background_task( // to the `new_connections` channel. We call `endpoint.accept()` only once // the element has successfully been sent on `new_connections`. queued_new_connections.push_back(connection); + if let Some(waker) = new_connection_waker.take() { + waker.wake(); + } }, } } diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 51d3e4df1fb..3dca1d3cbe3 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -25,13 +25,14 @@ //! Example: //! //! ``` -//! use libp2p_quic::{Config, Endpoint}; -//! use libp2p_core::Multiaddr; +//! use libp2p_quic::{Config, QuicTransport}; +//! use libp2p_core::{Multiaddr, Transport}; //! //! let keypair = libp2p_core::identity::Keypair::generate_ed25519(); +//! let quic_config = Config::new(&keypair).expect("could not make config"); +//! let mut quic_transport = QuicTransport::new(quic_config); //! let addr = "/ip4/127.0.0.1/udp/12345/quic".parse().expect("bad address?"); -//! let quic_config = Config::new(&keypair, addr).expect("could not make config"); -//! let quic_endpoint = Endpoint::new(quic_config).expect("I/O error"); +//! quic_transport.listen_on(addr).expect("listen error."); //! ``` //! //! The `Endpoint` struct implements the `Transport` trait of the `core` library. See the diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index f60afdc5f59..e309d53a8f5 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -55,6 +55,8 @@ pub use quinn_proto::{ pub struct QuicTransport { config: Config, listeners: SelectAll, + /// Endpoint to use if no listener exists. + dialer: Option>, } impl QuicTransport { @@ -62,6 +64,7 @@ impl QuicTransport { Self { listeners: SelectAll::new(), config, + dialer: None, } } } @@ -75,12 +78,9 @@ pub enum Error { /// Error after the remote has been reached. #[error("{0}")] Established(Libp2pQuicConnectionError), - /// Error while working with IfWatcher. - #[error("{0}")] - IfWatcher(std::io::Error), #[error("{0}")] - Socket(std::io::Error), + Io(#[from] std::io::Error), #[error("Background task crashed.")] TaskCrashed, @@ -94,13 +94,15 @@ impl Transport for QuicTransport { fn listen_on(&mut self, addr: Multiaddr) -> Result> { let socket_addr = multiaddr_to_socketaddr(&addr) - .ok_or_else(|| TransportError::MultiaddrNotSupported(addr))?; - let in_addr = InAddr::new(socket_addr.ip()); - let (endpoint, new_connections_rx) = Endpoint::new(self.config.clone(), socket_addr) - .map_err(|e| TransportError::Other(Error::Socket(e)))?; + .ok_or(TransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); - let listener = Listener::new(listener_id, endpoint, new_connections_rx, in_addr); + let listener = Listener::new(listener_id, socket_addr, self.config.clone()) + .map_err(TransportError::Other)?; self.listeners.push(listener); + // Drop reference to dialer endpoint so that the endpoint is dropped once the last + // connection that uses it is closed. + // New outbound connections will use a bidirectional (listener) endpoint. + let _ = self.dialer.take(); Ok(listener_id) } @@ -118,26 +120,40 @@ impl Transport for QuicTransport { } fn dial(&mut self, addr: Multiaddr) -> Result> { - todo!() - // let socket_addr = if let Some(socket_addr) = multiaddr_to_socketaddr(&addr) { - // if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { - // tracing::error!("multiaddr not supported"); - // return Err(TransportError::MultiaddrNotSupported(addr)); - // } - // socket_addr - // } else { - // tracing::error!("multiaddr not supported"); - // return Err(TransportError::MultiaddrNotSupported(addr)); - // }; - - // let endpoint = self.endpoint.clone(); - - // Ok(async move { - // let connection = endpoint.dial(socket_addr).await.map_err(Error::Reach)?; - // let final_connec = Upgrade::from_connection(connection).await?; - // Ok(final_connec) - // } - // .boxed()) + let socket_addr = multiaddr_to_socketaddr(&addr) + .ok_or_else(|| TransportError::MultiaddrNotSupported(addr.clone()))?; + if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { + tracing::error!("multiaddr not supported"); + return Err(TransportError::MultiaddrNotSupported(addr)); + } + let endpoint = if self.listeners.is_empty() { + match self.dialer.clone() { + Some(endpoint) => endpoint, + None => { + let endpoint = + Endpoint::new_dialer(self.config.clone()).map_err(TransportError::Other)?; + let _ = self.dialer.insert(endpoint.clone()); + endpoint + } + } + } else { + // Pick a random listener to use for dialing. + // TODO: Prefer listeners with same IP version. + let n = rand::random::() % self.listeners.len(); + let listener = self + .listeners + .iter_mut() + .nth(n) + .expect("Can not be out of bound."); + listener.endpoint.clone() + }; + + Ok(async move { + let connection = endpoint.dial(socket_addr).await.map_err(Error::Reach)?; + let final_connec = Upgrade::from_connection(connection).await?; + Ok(final_connec) + } + .boxed()) } fn dial_as_listener( @@ -169,7 +185,7 @@ struct Listener { listener_id: ListenerId, /// Channel where new connections are being sent. - new_connections: mpsc::Receiver, + new_connections_rx: mpsc::Receiver, /// The IP addresses of network interfaces on which the listening socket /// is accepting connections. @@ -187,17 +203,18 @@ struct Listener { impl Listener { fn new( listener_id: ListenerId, - endpoint: Arc, - new_connections: mpsc::Receiver, - in_addr: InAddr, - ) -> Self { - Listener { + socket_addr: SocketAddr, + config: Config, + ) -> Result { + let in_addr = InAddr::new(socket_addr.ip()); + let (endpoint, new_connections_rx) = Endpoint::new_bidirectional(config, socket_addr)?; + Ok(Listener { endpoint, listener_id, - new_connections, + new_connections_rx, in_addr, report_closed: None, - } + }) } /// Report the listener as closed in a [`TransportEvent::ListenerClosed`] and @@ -261,7 +278,7 @@ impl Listener { }; Some(TransportEvent::ListenerError { listener_id: self.listener_id, - error: Error::IfWatcher(err), + error: err.into(), }) } } @@ -286,7 +303,7 @@ impl Stream for Listener { if let Some(event) = self.poll_if_addr(cx) { return Poll::Ready(Some(event)); } - let connection = match futures::ready!(self.new_connections.poll_next_unpin(cx)) { + let connection = match futures::ready!(self.new_connections_rx.poll_next_unpin(cx)) { Some(c) => c, None => { self.close(Err(Error::TaskCrashed)); diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index c35f8138674..d00cb78cfbb 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -14,7 +14,6 @@ use libp2p::swarm::{Swarm, SwarmEvent}; use libp2p_quic::{Config as QuicConfig, QuicTransport}; use rand::RngCore; use std::num::NonZeroU8; -use std::time::Duration; use std::{io, iter}; fn generate_tls_keypair() -> libp2p::identity::Keypair { From 39d855c4e47b872a4e07e38a10b8c17165cb9c36 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Mon, 4 Jul 2022 18:20:33 +0200 Subject: [PATCH 064/218] transports/quic: test endpoint re-use --- transports/quic/src/transport.rs | 4 +- transports/quic/tests/smoke.rs | 109 ++++++++++++++++++++++++++++++- 2 files changed, 109 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index e309d53a8f5..e56d55eee5a 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -93,8 +93,8 @@ impl Transport for QuicTransport { type Dial = Pin> + Send>>; fn listen_on(&mut self, addr: Multiaddr) -> Result> { - let socket_addr = multiaddr_to_socketaddr(&addr) - .ok_or(TransportError::MultiaddrNotSupported(addr))?; + let socket_addr = + multiaddr_to_socketaddr(&addr).ok_or(TransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); let listener = Listener::new(listener_id, socket_addr, self.config.clone()) .map_err(TransportError::Other)?; diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index d00cb78cfbb..dbd66815406 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -2,15 +2,18 @@ use anyhow::Result; use async_trait::async_trait; use futures::future::FutureExt; use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; +use futures::select; use futures::stream::StreamExt; use futures::task::Spawn; +use libp2p::core::multiaddr::Protocol; use libp2p::core::muxing::StreamMuxerBox; -use libp2p::core::{upgrade, Transport}; +use libp2p::core::{upgrade, ConnectedPoint, Transport}; use libp2p::request_response::{ ProtocolName, ProtocolSupport, RequestResponse, RequestResponseCodec, RequestResponseConfig, RequestResponseEvent, RequestResponseMessage, }; -use libp2p::swarm::{Swarm, SwarmEvent}; +use libp2p::swarm::dial_opts::{DialOpts, PeerCondition}; +use libp2p::swarm::{DialError, Swarm, SwarmEvent}; use libp2p_quic::{Config as QuicConfig, QuicTransport}; use rand::RngCore; use std::num::NonZeroU8; @@ -437,3 +440,105 @@ fn concurrent_connections_and_streams() { // QuickCheck::new().quickcheck(prop as fn(_, _) -> _); } + +#[async_std::test] +async fn endpoint_reuse() -> Result<()> { + setup_global_subscriber(); + + let mut swarm_a = create_swarm(false).await?; + let mut swarm_b = create_swarm(false).await?; + let b_peer_id = *swarm_b.local_peer_id(); + + swarm_a.listen_on("/ip4/127.0.0.1/udp/0/quic".parse()?)?; + let a_addr = match swarm_a.next().await { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + }; + + swarm_b.dial(a_addr.clone()).unwrap(); + let b_send_back_addr = loop { + select! { + ev = swarm_a.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { endpoint, .. } => { + break endpoint.get_remote_address().clone() + } + SwarmEvent::IncomingConnection { local_addr, ..} => { + assert!(swarm_a.listeners().any(|a| a == &local_addr)); + } + e => panic!("{:?}", e), + }, + ev = swarm_b.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { .. } => {}, + e => panic!("{:?}", e), + } + } + }; + + let dial_opts = DialOpts::peer_id(b_peer_id) + .addresses(vec![b_send_back_addr.clone()]) + .extend_addresses_through_behaviour() + .condition(PeerCondition::Always) + .build(); + swarm_a.dial(dial_opts).unwrap(); + + // Expect the dial to fail since b is not listening on an address. + loop { + select! { + ev = swarm_a.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { ..} => panic!("Unexpected dial success."), + SwarmEvent::OutgoingConnectionError {error, .. } => { + assert!(matches!(error, DialError::Transport(_))); + break + } + _ => {} + }, + _ = swarm_b.select_next_some() => {}, + } + } + swarm_b.listen_on("/ip4/127.0.0.1/udp/0/quic".parse()?)?; + let b_addr = match swarm_b.next().await { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + }; + + let dial_opts = DialOpts::peer_id(b_peer_id) + .addresses(vec![b_addr.clone(), b_send_back_addr]) + .condition(PeerCondition::Always) + .build(); + swarm_a.dial(dial_opts).unwrap(); + let expected_b_addr = b_addr.with(Protocol::P2p(b_peer_id.into())); + + let mut a_reported = false; + let mut b_reported = false; + while !a_reported || !b_reported { + select! { + ev = swarm_a.select_next_some() => match ev{ + SwarmEvent::ConnectionEstablished { endpoint, ..} => { + assert!(endpoint.is_dialer()); + assert_eq!(endpoint.get_remote_address(), &expected_b_addr); + a_reported = true; + } + SwarmEvent::OutgoingConnectionError {error, .. } => { + panic!("Unexpected error {:}", error) + } + _ => {} + }, + ev = swarm_b.select_next_some() => match ev{ + SwarmEvent::ConnectionEstablished { endpoint, ..} => { + match endpoint { + ConnectedPoint::Dialer{..} => panic!("Unexpected outbound connection"), + ConnectedPoint::Listener {send_back_addr, local_addr} => { + // Expect that the local listening endpoint was used for dialing. + assert!(swarm_b.listeners().any(|a| a == &local_addr)); + assert_eq!(send_back_addr, a_addr); + b_reported = true; + } + } + } + _ => {} + }, + } + } + + Ok(()) +} From 0e797cf2cde34932e07c62bcad848d5815dff6ad Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 10 Jul 2022 12:26:36 +0200 Subject: [PATCH 065/218] transports/quic: use intra-doc links in docs Co-authored-by: Max Inden --- transports/quic/src/endpoint.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 59804a1bd96..36427a20966 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -98,7 +98,7 @@ pub struct Endpoint { } impl Endpoint { - /// Builds a new `Endpoint` that is listening on the [`SocketAddr`]. + /// Builds a new [`Endpoint`] that is listening on the [`SocketAddr`]. pub fn new_bidirectional( config: Config, socket_addr: SocketAddr, @@ -108,7 +108,7 @@ impl Endpoint { Ok((endpoint, new_connections_rx)) } - /// Builds a new `Endpoint` that only supports outbound connections. + /// Builds a new [`Endpoint`] that only supports outbound connections. pub fn new_dialer(config: Config) -> Result, transport::Error> { let socket_addr = SocketAddrV4::new(Ipv4Addr::LOCALHOST, 0); Self::new(config, socket_addr.into(), None) From ef7b82367642ae02038e598d04db8ac263e47096 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sun, 10 Jul 2022 13:56:44 +0200 Subject: [PATCH 066/218] transports/quic: remove mutex in `InAddr` --- transports/quic/src/in_addr.rs | 53 ++++++++++------------------------ 1 file changed, 16 insertions(+), 37 deletions(-) diff --git a/transports/quic/src/in_addr.rs b/transports/quic/src/in_addr.rs index 113b91f46d5..67b6abbf3f3 100644 --- a/transports/quic/src/in_addr.rs +++ b/transports/quic/src/in_addr.rs @@ -2,8 +2,7 @@ use if_watch::{IfEvent, IfWatcher}; use futures::{ future::{BoxFuture, FutureExt}, - lock::Mutex, - stream::{Stream, StreamExt}, + stream::Stream, }; use std::{ @@ -11,40 +10,34 @@ use std::{ net::IpAddr, ops::DerefMut, pin::Pin, - sync::Arc, task::{Context, Poll}, }; /// Watches for interface changes. -#[derive(Clone, Debug)] -pub(crate) struct InAddr(Arc>); +#[derive(Debug)] +pub enum InAddr { + /// The socket accepts connections on a single interface. + One { ip: Option }, + /// The socket accepts connections on all interfaces. + Any { if_watch: Box }, +} impl InAddr { /// If ip is specified then only one `IfEvent::Up` with IpNet(ip)/32 will be generated. /// If ip is unspecified then `IfEvent::Up/Down` events will be generated for all interfaces. - pub(crate) fn new(ip: IpAddr) -> Self { - let inner = if ip.is_unspecified() { + pub fn new(ip: IpAddr) -> Self { + if ip.is_unspecified() { let watcher = IfWatch::Pending(IfWatcher::new().boxed()); - InAddrInner::Any { + InAddr::Any { if_watch: Box::new(watcher), } } else { - InAddrInner::One { ip: Some(ip) } - }; - Self(Arc::new(Mutex::new(inner))) + InAddr::One { ip: Some(ip) } + } } } -/// The listening addresses of a `UdpSocket`. -#[derive(Debug)] -enum InAddrInner { - /// The socket accepts connections on a single interface. - One { ip: Option }, - /// The socket accepts connections on all interfaces. - Any { if_watch: Box }, -} - -enum IfWatch { +pub enum IfWatch { Pending(BoxFuture<'static, std::io::Result>), Ready(Box), } @@ -57,35 +50,21 @@ impl std::fmt::Debug for IfWatch { } } } - impl Stream for InAddr { type Item = Result; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { - let me = Pin::into_inner(self); - let mut lock = me.0.lock(); - let mut guard = futures::ready!(lock.poll_unpin(cx)); - let inner = &mut *guard; - - inner.poll_next_unpin(cx) - } -} - -impl Stream for InAddrInner { - type Item = Result; - fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { let me = Pin::into_inner(self); loop { match me { // If the listener is bound to a single interface, make sure the // address is reported once. - InAddrInner::One { ip } => { + InAddr::One { ip } => { if let Some(ip) = ip.take() { return Poll::Ready(Some(Ok(IfEvent::Up(ip.into())))); } } - InAddrInner::Any { if_watch } => { + InAddr::Any { if_watch } => { match if_watch.deref_mut() { // If we listen on all interfaces, wait for `if-watch` to be ready. IfWatch::Pending(f) => match futures::ready!(f.poll_unpin(cx)) { From a3755247c79615e6611c85e9660670b0026c495c Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sun, 10 Jul 2022 13:57:43 +0200 Subject: [PATCH 067/218] transports/quic: fix clippy --- transports/quic/src/muxer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 96898a01c2d..9fe6e7e4cd9 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -80,7 +80,7 @@ impl QuicMuxer { QuicMuxer { inner: Arc::new(Mutex::new(QuicMuxerInner { - connection: connection, + connection, substreams: Default::default(), pending_substreams: Default::default(), poll_close_waker: None, From 917bccc8e603f8b9407c9c1403292065c121a3d1 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sun, 10 Jul 2022 15:40:47 +0200 Subject: [PATCH 068/218] transport/quic: add test for closing a listener --- transports/quic/src/transport.rs | 178 +++++++++++++++++++++---------- 1 file changed, 119 insertions(+), 59 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index e56d55eee5a..f0a483eeba3 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -358,65 +358,125 @@ pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { } #[cfg(test)] -#[test] -fn multiaddr_to_udp_conversion() { +mod test { + + use futures::future::poll_fn; use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; - assert!( - multiaddr_to_socketaddr(&"/ip4/127.0.0.1/udp/1234".parse::().unwrap()).is_none() - ); - - assert_eq!( - multiaddr_to_socketaddr( - &"/ip4/127.0.0.1/udp/12345/quic" - .parse::() - .unwrap() - ), - Some(SocketAddr::new( - IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), - 12345, - )) - ); - assert_eq!( - multiaddr_to_socketaddr( - &"/ip4/255.255.255.255/udp/8080/quic" - .parse::() - .unwrap() - ), - Some(SocketAddr::new( - IpAddr::V4(Ipv4Addr::new(255, 255, 255, 255)), - 8080, - )) - ); - assert_eq!( - multiaddr_to_socketaddr( - &"/ip4/127.0.0.1/udp/55148/quic/p2p/12D3KooW9xk7Zp1gejwfwNpfm6L9zH5NL4Bx5rm94LRYJJHJuARZ" - .parse::() - .unwrap() - ), - Some(SocketAddr::new( - IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), - 55148, - )) - ); - assert_eq!( - multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), - Some(SocketAddr::new( - IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), - 12345, - )) - ); - assert_eq!( - multiaddr_to_socketaddr( - &"/ip6/ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/udp/8080/quic" - .parse::() - .unwrap() - ), - Some(SocketAddr::new( - IpAddr::V6(Ipv6Addr::new( - 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, - )), - 8080, - )) - ); + use super::*; + + #[test] + fn multiaddr_to_udp_conversion() { + assert!( + multiaddr_to_socketaddr(&"/ip4/127.0.0.1/udp/1234".parse::().unwrap()) + .is_none() + ); + + assert_eq!( + multiaddr_to_socketaddr( + &"/ip4/127.0.0.1/udp/12345/quic" + .parse::() + .unwrap() + ), + Some(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), + 12345, + )) + ); + assert_eq!( + multiaddr_to_socketaddr( + &"/ip4/255.255.255.255/udp/8080/quic" + .parse::() + .unwrap() + ), + Some(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(255, 255, 255, 255)), + 8080, + )) + ); + assert_eq!( + multiaddr_to_socketaddr( + &"/ip4/127.0.0.1/udp/55148/quic/p2p/12D3KooW9xk7Zp1gejwfwNpfm6L9zH5NL4Bx5rm94LRYJJHJuARZ" + .parse::() + .unwrap() + ), + Some(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), + 55148, + )) + ); + assert_eq!( + multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), + Some(SocketAddr::new( + IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), + 12345, + )) + ); + assert_eq!( + multiaddr_to_socketaddr( + &"/ip6/ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/udp/8080/quic" + .parse::() + .unwrap() + ), + Some(SocketAddr::new( + IpAddr::V6(Ipv6Addr::new( + 65535, 65535, 65535, 65535, 65535, 65535, 65535, 65535, + )), + 8080, + )) + ); + } + + #[async_std::test] + async fn close_listener() { + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); + let mut transport = QuicTransport::new(Config::new(&keypair).unwrap()); + + assert!(poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)) + .now_or_never() + .is_none()); + + // Run test twice to check that there is no unexpected behaviour if `QuicTransport.listener` + // is temporarily empty. + for _ in 0..2 { + let listener = transport + .listen_on("/ip4/0.0.0.0/udp/0/quic".parse().unwrap()) + .unwrap(); + match poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)).await { + TransportEvent::NewAddress { + listener_id, + listen_addr, + } => { + assert_eq!(listener_id, listener); + assert!( + matches!(listen_addr.iter().next(), Some(Protocol::Ip4(a)) if !a.is_unspecified()) + ); + assert!( + matches!(listen_addr.iter().nth(1), Some(Protocol::Udp(port)) if port != 0) + ); + assert!(matches!(listen_addr.iter().nth(2), Some(Protocol::Quic))); + } + e => panic!("Unexpected event: {:?}", e), + } + assert!( + transport.remove_listener(listener), + "Expect listener to exist." + ); + match poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)).await { + TransportEvent::ListenerClosed { + listener_id, + reason: Ok(()), + } => { + assert_eq!(listener_id, listener); + } + e => panic!("Unexpected event: {:?}", e), + } + // Poll once again so that the listener has the chance to return `Poll::Ready(None)` and + // be removed from the list of listeners. + assert!(poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)) + .now_or_never() + .is_none()); + assert!(transport.listeners.is_empty()); + } + } } From 5fe3e1cac44731d9c5aa43cb0ce9e41a1349a0a9 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 21 Jul 2022 17:07:45 +0300 Subject: [PATCH 069/218] Add an ignored test to dial ipv6 from ipv4 --- transports/quic/tests/smoke.rs | 35 ++++++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index dbd66815406..a2c3110b056 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -542,3 +542,38 @@ async fn endpoint_reuse() -> Result<()> { Ok(()) } + +#[async_std::test] +#[ignore] +async fn ipv4_dial_ipv6() -> Result<()> { + setup_global_subscriber(); + + let mut swarm_a = create_swarm(false).await?; + let mut swarm_b = create_swarm(false).await?; + + swarm_a.listen_on("/ip6/::1/udp/0/quic".parse()?)?; + let a_addr = match swarm_a.next().await { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + }; + + swarm_b.dial(a_addr.clone()).unwrap(); + + loop { + select! { + ev = swarm_a.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { .. } => { + return Ok(()) + } + SwarmEvent::IncomingConnection { local_addr, ..} => { + assert!(swarm_a.listeners().any(|a| a == &local_addr)); + } + e => panic!("{:?}", e), + }, + ev = swarm_b.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { .. } => {}, + e => panic!("{:?}", e), + } + } + }; +} From 574d534c4a8c4ea5ca2e6695780bdfa48f7dbb8f Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Thu, 21 Jul 2022 17:49:51 +0300 Subject: [PATCH 070/218] Get rid of recursion in poll_if_addr --- transports/quic/src/transport.rs | 82 +++++++++++++++----------------- 1 file changed, 39 insertions(+), 43 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index f0a483eeba3..70595f6a326 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -236,57 +236,53 @@ impl Listener { /// Poll for a next If Event. fn poll_if_addr(&mut self, cx: &mut Context<'_>) -> Option<::Item> { - match self.in_addr.poll_next_unpin(cx) { - Poll::Ready(mut item) => { - if let Some(item) = item.take() { - // Consume all events for up/down interface changes. - match item { - Ok(IfEvent::Up(inet)) => { - let ip = inet.addr(); - if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { - let socket_addr = - SocketAddr::new(ip, self.endpoint.socket_addr().port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("New listen address: {}", ma); - Some(TransportEvent::NewAddress { - listener_id: self.listener_id, - listen_addr: ma, - }) - } else { - self.poll_if_addr(cx) + loop { + match self.in_addr.poll_next_unpin(cx) { + Poll::Ready(mut item) => { + if let Some(item) = item.take() { + // Consume all events for up/down interface changes. + match item { + Ok(IfEvent::Up(inet)) => { + let ip = inet.addr(); + if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + let socket_addr = + SocketAddr::new(ip, self.endpoint.socket_addr().port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("New listen address: {}", ma); + return Some(TransportEvent::NewAddress { + listener_id: self.listener_id, + listen_addr: ma, + }) + } } - } - Ok(IfEvent::Down(inet)) => { - let ip = inet.addr(); - if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { - let socket_addr = - SocketAddr::new(ip, self.endpoint.socket_addr().port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("Expired listen address: {}", ma); - Some(TransportEvent::AddressExpired { + Ok(IfEvent::Down(inet)) => { + let ip = inet.addr(); + if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + let socket_addr = + SocketAddr::new(ip, self.endpoint.socket_addr().port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("Expired listen address: {}", ma); + return Some(TransportEvent::AddressExpired { + listener_id: self.listener_id, + listen_addr: ma, + }) + } + } + Err(err) => { + tracing::debug! { + "Failure polling interfaces: {:?}.", + err + }; + return Some(TransportEvent::ListenerError { listener_id: self.listener_id, - listen_addr: ma, + error: err.into(), }) - } else { - self.poll_if_addr(cx) } } - Err(err) => { - tracing::debug! { - "Failure polling interfaces: {:?}.", - err - }; - Some(TransportEvent::ListenerError { - listener_id: self.listener_id, - error: err.into(), - }) - } } - } else { - self.poll_if_addr(cx) } + Poll::Pending => return None, } - Poll::Pending => None, } } } From 9e1cc81967f0dff6d75f81ee6b4ae84a29d98e54 Mon Sep 17 00:00:00 2001 From: Roman Proskuryakov Date: Fri, 22 Jul 2022 10:53:35 +0300 Subject: [PATCH 071/218] cargo fmt --- transports/quic/src/transport.rs | 6 +++--- transports/quic/tests/smoke.rs | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 70595f6a326..7a80259ccd8 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -252,7 +252,7 @@ impl Listener { return Some(TransportEvent::NewAddress { listener_id: self.listener_id, listen_addr: ma, - }) + }); } } Ok(IfEvent::Down(inet)) => { @@ -265,7 +265,7 @@ impl Listener { return Some(TransportEvent::AddressExpired { listener_id: self.listener_id, listen_addr: ma, - }) + }); } } Err(err) => { @@ -276,7 +276,7 @@ impl Listener { return Some(TransportEvent::ListenerError { listener_id: self.listener_id, error: err.into(), - }) + }); } } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index a2c3110b056..a5539cb3ed2 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -575,5 +575,5 @@ async fn ipv4_dial_ipv6() -> Result<()> { e => panic!("{:?}", e), } } - }; + } } From 1a4b8acc80028ae2ac221224caf4ad9b3ceb95c1 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sat, 23 Jul 2022 23:11:07 +0200 Subject: [PATCH 072/218] quic: match socket family of endpoint for dialing --- transports/quic/src/endpoint.rs | 12 +++++--- transports/quic/src/transport.rs | 48 +++++++++++++++++++++----------- transports/quic/tests/smoke.rs | 1 - 3 files changed, 39 insertions(+), 22 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 36427a20966..82d7f8a6d7c 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -39,7 +39,7 @@ use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServer use std::{ collections::{HashMap, VecDeque}, fmt, - net::{Ipv4Addr, SocketAddr, SocketAddrV4, UdpSocket}, + net::{Ipv4Addr, Ipv6Addr, SocketAddr, UdpSocket}, sync::{Arc, Weak}, task::{Poll, Waker}, time::{Duration, Instant}, @@ -109,9 +109,13 @@ impl Endpoint { } /// Builds a new [`Endpoint`] that only supports outbound connections. - pub fn new_dialer(config: Config) -> Result, transport::Error> { - let socket_addr = SocketAddrV4::new(Ipv4Addr::LOCALHOST, 0); - Self::new(config, socket_addr.into(), None) + pub fn new_dialer(config: Config, is_ipv6: bool) -> Result, transport::Error> { + let socket_addr = if is_ipv6 { + SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0) + } else { + SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), 0) + }; + Self::new(config, socket_addr, None) } fn new( diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 7a80259ccd8..8c0f7402f54 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -55,8 +55,10 @@ pub use quinn_proto::{ pub struct QuicTransport { config: Config, listeners: SelectAll, - /// Endpoint to use if no listener exists. - dialer: Option>, + /// Endpoints to use for dialing Ipv4 addresses if no matching listener exists. + ipv4_dialer: Option>, + /// Endpoints to use for dialing Ipv6 addresses if no matching listener exists. + ipv6_dialer: Option>, } impl QuicTransport { @@ -64,7 +66,8 @@ impl QuicTransport { Self { listeners: SelectAll::new(), config, - dialer: None, + ipv4_dialer: None, + ipv6_dialer: None, } } } @@ -102,7 +105,10 @@ impl Transport for QuicTransport { // Drop reference to dialer endpoint so that the endpoint is dropped once the last // connection that uses it is closed. // New outbound connections will use a bidirectional (listener) endpoint. - let _ = self.dialer.take(); + match socket_addr { + SocketAddr::V4(_) => self.ipv4_dialer.take(), + SocketAddr::V6(_) => self.ipv6_dialer.take(), + }; Ok(listener_id) } @@ -126,25 +132,33 @@ impl Transport for QuicTransport { tracing::error!("multiaddr not supported"); return Err(TransportError::MultiaddrNotSupported(addr)); } - let endpoint = if self.listeners.is_empty() { - match self.dialer.clone() { - Some(endpoint) => endpoint, + let listeners = self + .listeners + .iter() + .filter(|l| { + let listen_addr = l.endpoint.socket_addr(); + listen_addr.is_ipv4() == socket_addr.is_ipv4() + && listen_addr.ip().is_loopback() == socket_addr.ip().is_loopback() + }) + .collect::>(); + let endpoint = if listeners.is_empty() { + let dialer = match socket_addr { + SocketAddr::V4(_) => &mut self.ipv4_dialer, + SocketAddr::V6(_) => &mut self.ipv6_dialer, + }; + match dialer { + Some(endpoint) => endpoint.clone(), None => { - let endpoint = - Endpoint::new_dialer(self.config.clone()).map_err(TransportError::Other)?; - let _ = self.dialer.insert(endpoint.clone()); + let endpoint = Endpoint::new_dialer(self.config.clone(), socket_addr.is_ipv6()) + .map_err(TransportError::Other)?; + let _ = dialer.insert(endpoint.clone()); endpoint } } } else { // Pick a random listener to use for dialing. - // TODO: Prefer listeners with same IP version. - let n = rand::random::() % self.listeners.len(); - let listener = self - .listeners - .iter_mut() - .nth(n) - .expect("Can not be out of bound."); + let n = rand::random::() % listeners.len(); + let listener = listeners.get(n).expect("Can not be out of bound."); listener.endpoint.clone() }; diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index a5539cb3ed2..bb4f1270950 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -544,7 +544,6 @@ async fn endpoint_reuse() -> Result<()> { } #[async_std::test] -#[ignore] async fn ipv4_dial_ipv6() -> Result<()> { setup_global_subscriber(); From bf06d9676e815f15f21a84624bd447a3ed49b199 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sat, 23 Jul 2022 23:24:00 +0200 Subject: [PATCH 073/218] quic: fix outdated docs & intra-doc links --- transports/quic/src/connection.rs | 2 +- transports/quic/src/endpoint.rs | 6 ------ 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 0c9217b190a..7027735c5e7 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -43,7 +43,7 @@ use std::{ /// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Upgrade`]. /// /// Contains everything needed to process a connection with a remote. -/// Tied to a specific [`crate::Endpoint`]. +/// Tied to a specific [`Endpoint`]. pub struct Connection { /// Endpoint this connection belongs to. endpoint: Arc, diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 82d7f8a6d7c..b3e9568a092 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -87,7 +87,6 @@ impl Config { // TODO: remove useless fields pub struct Endpoint { /// Channel to the background of the endpoint. - /// See [`Endpoint::new_connections`] (just below) for a commentary about the mutex. to_endpoint: Mutex>, /// Copy of [`Endpoint::to_endpoint`], except not behind a `Mutex`. Used if we want to be @@ -309,11 +308,6 @@ enum ToEndpoint { /// of elements being sent on `new_connections`, and the accept buffer of the /// [`quinn_proto::Endpoint`]. /// -/// Unfortunately, this design has the consequence that, on the network layer, we will accept a -/// certain number of incoming connections even if [`Endpoint::poll_incoming`] is never even -/// called. The `quinn-proto` library doesn't provide any way to not accept incoming connections -/// apart from filling the accept buffer. -/// /// ## Back-pressure on connections /// /// Because connections are processed by the user at a rate of their choice, we cannot properly From 57840a31cd4a72bfa29d62c0b12b6d6f18ab715f Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Thu, 4 Aug 2022 00:54:05 +0200 Subject: [PATCH 074/218] transports/quic: adapt QuicMuxer to libp2p#2724 Discussed in libp2p#2722. --- transports/quic/src/muxer.rs | 303 ++++++++++++++++------------------- 1 file changed, 142 insertions(+), 161 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 9fe6e7e4cd9..aaa0a7edcbe 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -22,7 +22,7 @@ use crate::connection::{Connection, ConnectionEvent}; use crate::error::Error; use futures::{AsyncRead, AsyncWrite}; -use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; +use libp2p_core::muxing::StreamMuxer; use parking_lot::Mutex; use std::{ collections::{HashMap, VecDeque}, @@ -54,6 +54,60 @@ struct QuicMuxerInner { poll_event_waker: Option, } +impl QuicMuxerInner { + fn poll_connection(&mut self, cx: &mut Context<'_>) { + while let Poll::Ready(event) = self.connection.poll_event(cx) { + match event { + ConnectionEvent::Connected => { + tracing::error!("Unexpected Connected event on established QUIC connection"); + } + ConnectionEvent::ConnectionLost(_) => { + if let Some(waker) = self.poll_close_waker.take() { + waker.wake(); + } + self.connection.close(); + } + + ConnectionEvent::StreamOpened => { + if let Some(waker) = self.pending_substreams.pop_front() { + waker.wake(); + } + } + ConnectionEvent::StreamReadable(substream) => { + if let Some(substream) = self.substreams.get_mut(&substream) { + if let Some(waker) = substream.read_waker.take() { + waker.wake(); + } + } + } + ConnectionEvent::StreamWritable(substream) => { + if let Some(substream) = self.substreams.get_mut(&substream) { + if let Some(waker) = substream.write_waker.take() { + waker.wake(); + } + } + } + ConnectionEvent::StreamFinished(substream) => { + if let Some(substream) = self.substreams.get_mut(&substream) { + substream.finished = true; + if let Some(waker) = substream.finished_waker.take() { + waker.wake(); + } + } + } + ConnectionEvent::StreamStopped(substream) => { + if let Some(substream) = self.substreams.get_mut(&substream) { + substream.stopped = true; + } + } + ConnectionEvent::StreamAvailable => { + // Handled below. + } + } + } + } +} + /// State of a single substream. #[derive(Default, Clone)] struct SubstreamState { @@ -89,6 +143,93 @@ impl QuicMuxer { } } } +impl StreamMuxer for QuicMuxer { + type Substream = Substream; + type Error = Error; + + fn poll_address_change( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.inner.lock().poll_connection(cx); + // TODO + Poll::Pending + } + + fn poll_inbound( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let mut inner = self.inner.lock(); + inner.poll_connection(cx); + if let Some(substream_id) = inner.connection.pop_incoming_substream() { + inner.substreams.insert(substream_id, Default::default()); + let substream = Substream::new(substream_id, self.inner.clone()); + Poll::Ready(Ok(substream)) + } else { + inner.poll_event_waker = Some(cx.waker().clone()); + Poll::Pending + } + } + + fn poll_outbound( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let mut inner = self.inner.lock(); + inner.poll_connection(cx); + if let Some(substream_id) = inner.connection.pop_outgoing_substream() { + inner.substreams.insert(substream_id, Default::default()); + let substream = Substream::new(substream_id, self.inner.clone()); + Poll::Ready(Ok(substream)) + } else { + inner.pending_substreams.push_back(cx.waker().clone()); + Poll::Pending + } + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let mut inner = self.inner.lock(); + inner.poll_connection(cx); + + if inner.connection.connection.is_drained() { + return Poll::Ready(Ok(())); + } + + if inner.substreams.is_empty() { + let connection = &mut inner.connection; + if !connection.connection.is_closed() { + connection.close(); + if let Some(waker) = inner.poll_event_waker.take() { + waker.wake(); + } + } else { + } + while let Poll::Ready(event) = inner.connection.poll_event(cx) { + if let ConnectionEvent::ConnectionLost(_) = event { + return Poll::Ready(Ok(())); + } + } + } else { + for substream in inner.substreams.clone().keys() { + if let Err(e) = inner.connection.shutdown_substream(*substream) { + tracing::error!("substream finish error on muxer close: {}", e); + } + } + } + + // Register `cx.waker()` as being woken up if the connection closes. + inner.poll_close_waker = Some(cx.waker().clone()); + + Poll::Pending + } +} + +impl fmt::Debug for QuicMuxer { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_tuple("QuicMuxer").finish() + } +} pub struct Substream { id: quinn_proto::StreamId, @@ -270,163 +411,3 @@ impl AsyncWrite for Substream { } } } - -impl StreamMuxer for QuicMuxer { - type OutboundSubstream = (); - type Substream = Substream; - type Error = Error; - - /// Polls for a connection-wide event. - /// - /// This function behaves the same as a `Stream`. - /// - /// If `Pending` is returned, then the current task will be notified once the muxer - /// is ready to be polled, similar to the API of `Stream::poll()`. - /// Only the latest task that was used to call this method may be notified. - /// - /// It is permissible and common to use this method to perform background - /// work, such as processing incoming packets and polling timers. - /// - /// An error can be generated if the connection has been closed. - fn poll_event( - &self, - cx: &mut Context<'_>, - ) -> Poll, Self::Error>> { - // We use `poll_event` to perform the background processing of the entire connection. - let mut inner = self.inner.lock(); - - while let Poll::Ready(event) = inner.connection.poll_event(cx) { - match event { - ConnectionEvent::Connected => { - tracing::error!("Unexpected Connected event on established QUIC connection"); - } - ConnectionEvent::ConnectionLost(_) => { - if let Some(waker) = inner.poll_close_waker.take() { - waker.wake(); - } - inner.connection.close(); - } - - ConnectionEvent::StreamOpened => { - if let Some(waker) = inner.pending_substreams.pop_front() { - waker.wake(); - } - } - ConnectionEvent::StreamReadable(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { - if let Some(waker) = substream.read_waker.take() { - waker.wake(); - } - } - } - ConnectionEvent::StreamWritable(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { - if let Some(waker) = substream.write_waker.take() { - waker.wake(); - } - } - } - ConnectionEvent::StreamFinished(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { - substream.finished = true; - if let Some(waker) = substream.finished_waker.take() { - waker.wake(); - } - } - } - ConnectionEvent::StreamStopped(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { - substream.stopped = true; - } - } - ConnectionEvent::StreamAvailable => { - // Handled below. - } - } - } - - if let Some(substream_id) = inner.connection.pop_incoming_substream() { - inner.substreams.insert(substream_id, Default::default()); - let substream = Substream::new(substream_id, self.inner.clone()); - Poll::Ready(Ok(StreamMuxerEvent::InboundSubstream(substream))) - } else { - inner.poll_event_waker = Some(cx.waker().clone()); - Poll::Pending - } - } - - /// Opens a new outgoing substream, and produces the equivalent to a future that will be - /// resolved when it becomes available. - /// - /// We provide the same handler to poll it by multiple tasks, which is done as a FIFO - /// queue via `poll_outbound`. - fn open_outbound(&self) -> Self::OutboundSubstream {} - - /// Polls the outbound substream. - /// - /// If `Pending` is returned, then the current task will be notified once the substream - /// is ready to be polled, similar to the API of `Future::poll()`. - fn poll_outbound( - &self, - cx: &mut Context<'_>, - _: &mut Self::OutboundSubstream, - ) -> Poll> { - let mut inner = self.inner.lock(); - if let Some(substream_id) = inner.connection.pop_outgoing_substream() { - inner.substreams.insert(substream_id, Default::default()); - let substream = Substream::new(substream_id, self.inner.clone()); - Poll::Ready(Ok(substream)) - } else { - inner.pending_substreams.push_back(cx.waker().clone()); - Poll::Pending - } - } - - /// Destroys an outbound substream future. Use this after the outbound substream has finished, - /// or if you want to interrupt it. - fn destroy_outbound(&self, _: Self::OutboundSubstream) { - // Do nothing because we don't know which waker should be destroyed. - // TODO `Self::OutboundSubstream` -> autoincrement id. - } - - fn poll_close(&self, cx: &mut Context<'_>) -> Poll> { - let mut inner = self.inner.lock(); - - if inner.connection.connection.is_drained() { - return Poll::Ready(Ok(())); - } - - if inner.substreams.is_empty() { - let connection = &mut inner.connection; - if !connection.connection.is_closed() { - connection.close(); - if let Some(waker) = inner.poll_event_waker.take() { - waker.wake(); - } - } else { - } - while let Poll::Ready(event) = inner.connection.poll_event(cx) { - if let ConnectionEvent::ConnectionLost(_) = event { - return Poll::Ready(Ok(())); - } - } - } else { - for substream in inner.substreams.clone().keys() { - if let Err(e) = inner.connection.shutdown_substream(*substream) { - tracing::error!("substream finish error on muxer close: {}", e); - } - } - } - - // Register `cx.waker()` as being woken up if the connection closes. - inner.poll_close_waker = Some(cx.waker().clone()); - - Poll::Pending - } -} - -impl fmt::Debug for QuicMuxer { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_tuple("QuicMuxer").finish() - } -} From 3ce0ef9de7f012af4416e7f1556db613f8c974b1 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sat, 13 Aug 2022 16:33:50 +0200 Subject: [PATCH 075/218] transports/quic: apply suggestions from review --- transports/quic/src/muxer.rs | 48 +++++++++++++++++------------------- 1 file changed, 23 insertions(+), 25 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index aaa0a7edcbe..ba4a771c328 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -25,14 +25,15 @@ use futures::{AsyncRead, AsyncWrite}; use libp2p_core::muxing::StreamMuxer; use parking_lot::Mutex; use std::{ - collections::{HashMap, VecDeque}, - fmt, io, + collections::HashMap, + io, pin::Pin, sync::{Arc, Weak}, task::{Context, Poll, Waker}, }; /// State for a single opened QUIC connection. +#[derive(Debug)] pub struct QuicMuxer { // Note: This could theoretically be an asynchronous future, in order to yield the current // task if a task running in parallel is already holding the lock. However, using asynchronous @@ -41,17 +42,18 @@ pub struct QuicMuxer { } /// Mutex-protected fields of [`QuicMuxer`]. +#[derive(Debug)] struct QuicMuxerInner { /// Inner connection object that yields events. connection: Connection, // /// State of all the substreams that the muxer reports as open. substreams: HashMap, - /// A FIFO of wakers to wake if a new outgoing substream is opened. - pending_substreams: VecDeque, + /// Waker to wake if a new outbound substream is opened. + poll_outbound_waker: Option, + /// Waker to wake if a new inbound substream was happened. + poll_inbound_waker: Option, /// Waker to wake if the connection is closed. poll_close_waker: Option, - /// Waker to wake if any event is happened. - poll_event_waker: Option, } impl QuicMuxerInner { @@ -59,7 +61,7 @@ impl QuicMuxerInner { while let Poll::Ready(event) = self.connection.poll_event(cx) { match event { ConnectionEvent::Connected => { - tracing::error!("Unexpected Connected event on established QUIC connection"); + tracing::warn!("Unexpected Connected event on established QUIC connection"); } ConnectionEvent::ConnectionLost(_) => { if let Some(waker) = self.poll_close_waker.take() { @@ -69,7 +71,7 @@ impl QuicMuxerInner { } ConnectionEvent::StreamOpened => { - if let Some(waker) = self.pending_substreams.pop_front() { + if let Some(waker) = self.poll_outbound_waker.take() { waker.wake(); } } @@ -101,7 +103,9 @@ impl QuicMuxerInner { } } ConnectionEvent::StreamAvailable => { - // Handled below. + if let Some(waker) = self.poll_inbound_waker.take() { + waker.wake(); + } } } } @@ -109,7 +113,7 @@ impl QuicMuxerInner { } /// State of a single substream. -#[derive(Default, Clone)] +#[derive(Debug, Default, Clone)] struct SubstreamState { /// Waker to wake if the substream becomes readable or stopped. read_waker: Option, @@ -136,9 +140,9 @@ impl QuicMuxer { inner: Arc::new(Mutex::new(QuicMuxerInner { connection, substreams: Default::default(), - pending_substreams: Default::default(), + poll_outbound_waker: None, poll_close_waker: None, - poll_event_waker: None, + poll_inbound_waker: None, })), } } @@ -167,7 +171,7 @@ impl StreamMuxer for QuicMuxer { let substream = Substream::new(substream_id, self.inner.clone()); Poll::Ready(Ok(substream)) } else { - inner.poll_event_waker = Some(cx.waker().clone()); + inner.poll_inbound_waker = Some(cx.waker().clone()); Poll::Pending } } @@ -183,7 +187,7 @@ impl StreamMuxer for QuicMuxer { let substream = Substream::new(substream_id, self.inner.clone()); Poll::Ready(Ok(substream)) } else { - inner.pending_substreams.push_back(cx.waker().clone()); + inner.poll_outbound_waker = Some(cx.waker().clone()); Poll::Pending } } @@ -200,7 +204,7 @@ impl StreamMuxer for QuicMuxer { let connection = &mut inner.connection; if !connection.connection.is_closed() { connection.close(); - if let Some(waker) = inner.poll_event_waker.take() { + if let Some(waker) = inner.poll_inbound_waker.take() { waker.wake(); } } else { @@ -211,9 +215,9 @@ impl StreamMuxer for QuicMuxer { } } } else { - for substream in inner.substreams.clone().keys() { - if let Err(e) = inner.connection.shutdown_substream(*substream) { - tracing::error!("substream finish error on muxer close: {}", e); + for substream in inner.substreams.keys().cloned().collect::>() { + if let Err(e) = inner.connection.shutdown_substream(substream) { + tracing::warn!("substream finish error on muxer close: {}", e); } } } @@ -225,12 +229,6 @@ impl StreamMuxer for QuicMuxer { } } -impl fmt::Debug for QuicMuxer { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_tuple("QuicMuxer").finish() - } -} - pub struct Substream { id: quinn_proto::StreamId, muxer: Weak>, @@ -319,7 +317,7 @@ impl AsyncRead for Substream { } } if chunks.finalize().should_transmit() { - if let Some(waker) = muxer.poll_event_waker.take() { + if let Some(waker) = muxer.poll_inbound_waker.take() { waker.wake(); } } From 3060d122a1f5751e252f7a7479f2ddb48b02847d Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sat, 13 Aug 2022 16:38:58 +0200 Subject: [PATCH 076/218] transports/quic: rename QuicMuxerInner -> Inner --- transports/quic/src/muxer.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index ba4a771c328..06d8c783376 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -38,12 +38,12 @@ pub struct QuicMuxer { // Note: This could theoretically be an asynchronous future, in order to yield the current // task if a task running in parallel is already holding the lock. However, using asynchronous // mutexes without async/await is extremely tedious and maybe not worth the effort. - inner: Arc>, + inner: Arc>, } /// Mutex-protected fields of [`QuicMuxer`]. #[derive(Debug)] -struct QuicMuxerInner { +struct Inner { /// Inner connection object that yields events. connection: Connection, // /// State of all the substreams that the muxer reports as open. @@ -56,7 +56,7 @@ struct QuicMuxerInner { poll_close_waker: Option, } -impl QuicMuxerInner { +impl Inner { fn poll_connection(&mut self, cx: &mut Context<'_>) { while let Poll::Ready(event) = self.connection.poll_event(cx) { match event { @@ -137,7 +137,7 @@ impl QuicMuxer { assert!(!connection.is_handshaking()); QuicMuxer { - inner: Arc::new(Mutex::new(QuicMuxerInner { + inner: Arc::new(Mutex::new(Inner { connection, substreams: Default::default(), poll_outbound_waker: None, @@ -231,11 +231,11 @@ impl StreamMuxer for QuicMuxer { pub struct Substream { id: quinn_proto::StreamId, - muxer: Weak>, + muxer: Weak>, } impl Substream { - fn new(id: quinn_proto::StreamId, muxer: Arc>) -> Self { + fn new(id: quinn_proto::StreamId, muxer: Arc>) -> Self { Self { id, muxer: Arc::downgrade(&muxer), From 63c6edc0697c6a0e388d32e97bc73cc3488ca5e0 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sat, 13 Aug 2022 17:07:52 +0200 Subject: [PATCH 077/218] transports/quic: improve poll_{inbound, outbound} --- transports/quic/src/muxer.rs | 36 ++++++++++++++++++++---------------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 06d8c783376..96c6cddf4b6 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -166,14 +166,16 @@ impl StreamMuxer for QuicMuxer { ) -> Poll> { let mut inner = self.inner.lock(); inner.poll_connection(cx); - if let Some(substream_id) = inner.connection.pop_incoming_substream() { - inner.substreams.insert(substream_id, Default::default()); - let substream = Substream::new(substream_id, self.inner.clone()); - Poll::Ready(Ok(substream)) - } else { - inner.poll_inbound_waker = Some(cx.waker().clone()); - Poll::Pending - } + let substream_id = match inner.connection.pop_incoming_substream() { + Some(id) => id, + None => { + inner.poll_inbound_waker = Some(cx.waker().clone()); + return Poll::Pending; + } + }; + inner.substreams.insert(substream_id, Default::default()); + let substream = Substream::new(substream_id, self.inner.clone()); + Poll::Ready(Ok(substream)) } fn poll_outbound( @@ -182,14 +184,16 @@ impl StreamMuxer for QuicMuxer { ) -> Poll> { let mut inner = self.inner.lock(); inner.poll_connection(cx); - if let Some(substream_id) = inner.connection.pop_outgoing_substream() { - inner.substreams.insert(substream_id, Default::default()); - let substream = Substream::new(substream_id, self.inner.clone()); - Poll::Ready(Ok(substream)) - } else { - inner.poll_outbound_waker = Some(cx.waker().clone()); - Poll::Pending - } + let substream_id = match inner.connection.pop_outgoing_substream() { + Some(id) => id, + None => { + inner.poll_outbound_waker = Some(cx.waker().clone()); + return Poll::Pending; + } + }; + inner.substreams.insert(substream_id, Default::default()); + let substream = Substream::new(substream_id, self.inner.clone()); + Poll::Ready(Ok(substream)) } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { From 95fc6dadb3774058538dba526a3170867ba9b041 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Fri, 19 Aug 2022 18:14:17 +0200 Subject: [PATCH 078/218] transports/quic: drive connection in `QuicMuxer::poll` --- transports/quic/src/muxer.rs | 139 ++++++++++++++++------------------- 1 file changed, 65 insertions(+), 74 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 96c6cddf4b6..637bf9a0a41 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -22,7 +22,7 @@ use crate::connection::{Connection, ConnectionEvent}; use crate::error::Error; use futures::{AsyncRead, AsyncWrite}; -use libp2p_core::muxing::StreamMuxer; +use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ collections::HashMap, @@ -56,41 +56,83 @@ struct Inner { poll_close_waker: Option, } -impl Inner { - fn poll_connection(&mut self, cx: &mut Context<'_>) { - while let Poll::Ready(event) = self.connection.poll_event(cx) { +/// State of a single substream. +#[derive(Debug, Default, Clone)] +struct SubstreamState { + /// Waker to wake if the substream becomes readable or stopped. + read_waker: Option, + /// Waker to wake if the substream becomes writable or stopped. + write_waker: Option, + /// True if the substream has been finished. + finished: bool, + /// True if the substream has been stopped. + stopped: bool, + /// Waker to wake if the substream becomes closed or stopped. + finished_waker: Option, +} + +impl QuicMuxer { + /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. + /// + /// # Panic + /// + /// Panics if `connection.is_handshaking()` returns `true`. + pub(crate) fn from_connection(connection: Connection) -> Self { + assert!(!connection.is_handshaking()); + + QuicMuxer { + inner: Arc::new(Mutex::new(Inner { + connection, + substreams: Default::default(), + poll_outbound_waker: None, + poll_close_waker: None, + poll_inbound_waker: None, + })), + } + } +} +impl StreamMuxer for QuicMuxer { + type Substream = Substream; + type Error = Error; + + fn poll( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let mut inner = self.inner.lock(); + while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { ConnectionEvent::Connected => { tracing::warn!("Unexpected Connected event on established QUIC connection"); } ConnectionEvent::ConnectionLost(_) => { - if let Some(waker) = self.poll_close_waker.take() { + if let Some(waker) = inner.poll_close_waker.take() { waker.wake(); } - self.connection.close(); + inner.connection.close(); } ConnectionEvent::StreamOpened => { - if let Some(waker) = self.poll_outbound_waker.take() { + if let Some(waker) = inner.poll_outbound_waker.take() { waker.wake(); } } ConnectionEvent::StreamReadable(substream) => { - if let Some(substream) = self.substreams.get_mut(&substream) { + if let Some(substream) = inner.substreams.get_mut(&substream) { if let Some(waker) = substream.read_waker.take() { waker.wake(); } } } ConnectionEvent::StreamWritable(substream) => { - if let Some(substream) = self.substreams.get_mut(&substream) { + if let Some(substream) = inner.substreams.get_mut(&substream) { if let Some(waker) = substream.write_waker.take() { waker.wake(); } } } ConnectionEvent::StreamFinished(substream) => { - if let Some(substream) = self.substreams.get_mut(&substream) { + if let Some(substream) = inner.substreams.get_mut(&substream) { substream.finished = true; if let Some(waker) = substream.finished_waker.take() { waker.wake(); @@ -98,65 +140,18 @@ impl Inner { } } ConnectionEvent::StreamStopped(substream) => { - if let Some(substream) = self.substreams.get_mut(&substream) { + if let Some(substream) = inner.substreams.get_mut(&substream) { substream.stopped = true; } } ConnectionEvent::StreamAvailable => { - if let Some(waker) = self.poll_inbound_waker.take() { + if let Some(waker) = inner.poll_inbound_waker.take() { waker.wake(); } } } } - } -} - -/// State of a single substream. -#[derive(Debug, Default, Clone)] -struct SubstreamState { - /// Waker to wake if the substream becomes readable or stopped. - read_waker: Option, - /// Waker to wake if the substream becomes writable or stopped. - write_waker: Option, - /// True if the substream has been finished. - finished: bool, - /// True if the substream has been stopped. - stopped: bool, - /// Waker to wake if the substream becomes closed or stopped. - finished_waker: Option, -} - -impl QuicMuxer { - /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. - /// - /// # Panic - /// - /// Panics if `connection.is_handshaking()` returns `true`. - pub(crate) fn from_connection(connection: Connection) -> Self { - assert!(!connection.is_handshaking()); - - QuicMuxer { - inner: Arc::new(Mutex::new(Inner { - connection, - substreams: Default::default(), - poll_outbound_waker: None, - poll_close_waker: None, - poll_inbound_waker: None, - })), - } - } -} -impl StreamMuxer for QuicMuxer { - type Substream = Substream; - type Error = Error; - - fn poll_address_change( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - self.inner.lock().poll_connection(cx); - // TODO + // TODO: poll address change Poll::Pending } @@ -165,13 +160,12 @@ impl StreamMuxer for QuicMuxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); - inner.poll_connection(cx); let substream_id = match inner.connection.pop_incoming_substream() { - Some(id) => id, - None => { - inner.poll_inbound_waker = Some(cx.waker().clone()); - return Poll::Pending; - } + Some(id) => id, + None => { + inner.poll_inbound_waker = Some(cx.waker().clone()); + return Poll::Pending; + } }; inner.substreams.insert(substream_id, Default::default()); let substream = Substream::new(substream_id, self.inner.clone()); @@ -183,13 +177,12 @@ impl StreamMuxer for QuicMuxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); - inner.poll_connection(cx); let substream_id = match inner.connection.pop_outgoing_substream() { - Some(id) => id, - None => { - inner.poll_outbound_waker = Some(cx.waker().clone()); - return Poll::Pending; - } + Some(id) => id, + None => { + inner.poll_outbound_waker = Some(cx.waker().clone()); + return Poll::Pending; + } }; inner.substreams.insert(substream_id, Default::default()); let substream = Substream::new(substream_id, self.inner.clone()); @@ -198,8 +191,6 @@ impl StreamMuxer for QuicMuxer { fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut inner = self.inner.lock(); - inner.poll_connection(cx); - if inner.connection.connection.is_drained() { return Poll::Ready(Ok(())); } From 0d7c8a5667626b66f370c333aa7c318b57fb035a Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sun, 21 Aug 2022 17:12:21 +0200 Subject: [PATCH 079/218] transports/quic: refactor `Connection::poll_event` --- transports/quic/src/connection.rs | 197 ++++++++++++++---------------- transports/quic/src/endpoint.rs | 43 +------ transports/quic/src/muxer.rs | 7 +- transports/quic/src/upgrade.rs | 44 +++---- 4 files changed, 118 insertions(+), 173 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 7027735c5e7..afb2b02ce84 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -26,7 +26,7 @@ //! All interactions with a QUIC connection should be done through this struct. // TODO: docs -use crate::endpoint::Endpoint; +use crate::endpoint::{Endpoint, ToEndpoint}; use async_io::Timer; use futures::{channel::mpsc, prelude::*}; @@ -34,7 +34,6 @@ use libp2p_core::PeerId; use std::{ fmt, net::SocketAddr, - pin::Pin, sync::Arc, task::{Context, Poll}, time::Instant, @@ -48,7 +47,7 @@ pub struct Connection { /// Endpoint this connection belongs to. endpoint: Arc, /// Future whose job is to send a message to the endpoint. Only one at a time. - pending_to_endpoint: Option + Send + Sync>>>, + pending_to_endpoint: Option, /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. /// Passed at initialization. from_endpoint: mpsc::Receiver, @@ -71,6 +70,8 @@ pub struct Connection { /// Contains `None` if it is still open. /// Contains `Some` if and only if a `ConnectionLost` event has been emitted. closed: Option, + + to_endpoint: mpsc::Sender, } /// Error on the connection as a whole. @@ -110,6 +111,7 @@ impl Connection { let is_handshaking = connection.is_handshaking(); Connection { + to_endpoint: endpoint.to_endpoint2.clone(), endpoint, pending_to_endpoint: None, connection, @@ -227,158 +229,100 @@ impl Connection { return Poll::Pending; } - // Process events that the endpoint has sent to us. loop { - match Pin::new(&mut self.from_endpoint).poll_next(cx) { - Poll::Ready(Some(event)) => self.connection.handle_event(event), + match self.from_endpoint.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => { + self.connection.handle_event(event); + continue; + } Poll::Ready(None) => { debug_assert!(self.closed.is_none()); let err = Error::ClosedChannel; self.closed = Some(err.clone()); return Poll::Ready(ConnectionEvent::ConnectionLost(err)); } - Poll::Pending => break, + Poll::Pending => {} } - } - 'send_pending: loop { // Sending the pending event to the endpoint. If the endpoint is too busy, we just // stop the processing here. - // There is a bit of a question in play here: should we continue to accept events - // through `from_endpoint` if `to_endpoint` is busy? // We need to be careful to avoid a potential deadlock if both `from_endpoint` and // `to_endpoint` are full. As such, we continue to transfer data from `from_endpoint` // to the `quinn_proto::Connection` (see above). // However we don't deliver substream-related events to the user as long as // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. - if let Some(pending_to_endpoint) = &mut self.pending_to_endpoint { - match Future::poll(Pin::new(pending_to_endpoint), cx) { + if self.pending_to_endpoint.is_some() { + match self.to_endpoint.poll_ready_unpin(cx) { + Poll::Ready(Ok(())) => { + self.to_endpoint + .start_send(self.pending_to_endpoint.take().expect("is_some")) + .expect("To be ready"); + } + Poll::Ready(Err(_)) => todo!(), Poll::Pending => return Poll::Pending, - Poll::Ready(()) => self.pending_to_endpoint = None, } } - let now = Instant::now(); - // Poll the connection for packets to send on the UDP socket and try to send them on // `to_endpoint`. // FIXME max_datagrams - if let Some(transmit) = self.connection.poll_transmit(now, 1) { - let endpoint = self.endpoint.clone(); - debug_assert!(self.pending_to_endpoint.is_none()); - self.pending_to_endpoint = Some(Box::pin(async move { - // TODO: ECN bits not handled - endpoint - .send_udp_packet(transmit.destination, transmit.contents) - .await; - })); - continue 'send_pending; + if let Some(transmit) = self.connection.poll_transmit(Instant::now(), 1) { + // TODO: ECN bits not handled + self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket { + destination: transmit.destination, + data: transmit.contents, + }); + continue; } // Timeout system. - // We break out of the following loop until if `poll_timeout()` returns `None` or if - // polling `self.next_timeout` returns `Poll::Pending`. - loop { - if let Some(next_timeout) = &mut self.next_timeout { - match Future::poll(Pin::new(next_timeout), cx) { - Poll::Ready(when) => { - self.connection.handle_timeout(when); - self.next_timeout = None; - } - Poll::Pending => break, + if let Some(when) = self.connection.poll_timeout() { + let mut timer = Timer::at(when); + match timer.poll_unpin(cx) { + Poll::Ready(when) => { + self.connection.handle_timeout(when); + continue; } + Poll::Pending => self.next_timeout = Some(timer), } - if let Some(when) = self.connection.poll_timeout() { - self.next_timeout = Some(Timer::at(when)); - continue; - } - break; } // The connection also needs to be able to send control messages to the endpoint. This is // handled here, and we try to send them on `to_endpoint` as well. - if let Some(endpoint_event) = self.connection.poll_endpoint_events() { - let endpoint = self.endpoint.clone(); + if let Some(event) = self.connection.poll_endpoint_events() { let connection_id = self.connection_id; - debug_assert!(self.pending_to_endpoint.is_none()); - self.pending_to_endpoint = Some(Box::pin(async move { - endpoint - .report_quinn_event(connection_id, endpoint_event) - .await; - })); - continue 'send_pending; + self.pending_to_endpoint = Some(ToEndpoint::ProcessConnectionEvent { + connection_id, + event, + }); + continue; } // The final step consists in handling the events related to the various substreams. - while let Some(event) = self.connection.poll() { - match event { - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { - dir: quinn_proto::Dir::Uni, - }) - | quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { - dir: quinn_proto::Dir::Uni, - }) - | quinn_proto::Event::DatagramReceived => { - // We don't use datagrams or unidirectional streams. If these events - // happen, it is by some code not compatible with libp2p-quic. - self.connection - .close(Instant::now(), From::from(0u32), Default::default()); - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { - return Poll::Ready(ConnectionEvent::StreamReadable(id)); - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Writable { id }) => { - return Poll::Ready(ConnectionEvent::StreamWritable(id)); - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { - id, .. - }) => { - // The `Stop` QUIC event is more or less similar to a `Reset`, except that - // it applies only on the writing side of the pipe. - return Poll::Ready(ConnectionEvent::StreamStopped(id)); - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { - dir: quinn_proto::Dir::Bi, - }) => { - return Poll::Ready(ConnectionEvent::StreamAvailable); - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { - dir: quinn_proto::Dir::Bi, - }) => { - return Poll::Ready(ConnectionEvent::StreamOpened); - } - quinn_proto::Event::ConnectionLost { reason } => { - debug_assert!(self.closed.is_none()); + match self.connection.poll() { + Some(ev) => match ConnectionEvent::try_from(ev) { + Ok(ConnectionEvent::ConnectionLost(err)) => { self.is_handshaking = false; - let err = Error::Quinn(reason); self.closed = Some(err.clone()); - // self.close(); - // self.connection - // .close(Instant::now(), From::from(0u32), Default::default()); return Poll::Ready(ConnectionEvent::ConnectionLost(err)); } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { - return Poll::Ready(ConnectionEvent::StreamFinished(id)); - } - quinn_proto::Event::Connected => { + Ok(ConnectionEvent::Connected) => { debug_assert!(self.is_handshaking); debug_assert!(!self.connection.is_handshaking()); self.is_handshaking = false; return Poll::Ready(ConnectionEvent::Connected); } - quinn_proto::Event::HandshakeDataReady => { - if !self.is_handshaking { - tracing::error!("Got HandshakeDataReady while not handshaking"); - } + Ok(event) => return Poll::Ready(event), + Err(_proto_ev) => { + // unreachable: We don't use datagrams or unidirectional streams. + continue; } - } + }, + None => {} } - - break; + return Poll::Pending; } - - Poll::Pending } } @@ -423,4 +367,45 @@ pub enum ConnectionEvent { /// A substream has been stopped. This concept is similar to the concept of a substream being /// "reset", as in a TCP socket being reset for example. StreamStopped(quinn_proto::StreamId), + + HandshakeDataReady, +} + +impl TryFrom for ConnectionEvent { + type Error = quinn_proto::Event; + + fn try_from(event: quinn_proto::Event) -> Result { + match event { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { + Ok(ConnectionEvent::StreamReadable(id)) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Writable { id }) => { + Ok(ConnectionEvent::StreamWritable(id)) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { id, .. }) => { + Ok(ConnectionEvent::StreamStopped(id)) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Bi, + }) => Ok(ConnectionEvent::StreamAvailable), + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Bi, + }) => Ok(ConnectionEvent::StreamOpened), + quinn_proto::Event::ConnectionLost { reason } => { + Ok(ConnectionEvent::ConnectionLost(Error::Quinn(reason))) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { + Ok(ConnectionEvent::StreamFinished(id)) + } + quinn_proto::Event::Connected => Ok(ConnectionEvent::Connected), + quinn_proto::Event::HandshakeDataReady => Ok(ConnectionEvent::HandshakeDataReady), + ev @ quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Uni, + }) + | ev @ quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Uni, + }) + | ev @ quinn_proto::Event::DatagramReceived => Err(ev), + } + } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b3e9568a092..b7ca5f2b511 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -91,7 +91,7 @@ pub struct Endpoint { /// Copy of [`Endpoint::to_endpoint`], except not behind a `Mutex`. Used if we want to be /// guaranteed a slot in the messages buffer. - to_endpoint2: mpsc::Sender, + pub to_endpoint2: mpsc::Sender, socket_addr: SocketAddr, } @@ -172,45 +172,6 @@ impl Endpoint { rx.await.expect("background task has crashed") } - /// Asks the endpoint to send a UDP packet. - /// - /// Note that this method only queues the packet and returns as soon as the packet is in queue. - /// There is no guarantee that the packet will actually be sent, but considering that this is - /// a UDP packet, you cannot rely on the packet being delivered anyway. - pub async fn send_udp_packet(&self, destination: SocketAddr, data: impl Into>) { - let _ = self - .to_endpoint - .lock() - .await - .send(ToEndpoint::SendUdpPacket { - destination, - data: data.into(), - }) - .await; - } - - /// Report to the endpoint an event on a [`quinn_proto::Connection`]. - /// - /// This is typically called by a [`Connection`]. - /// - /// If `event.is_drained()` is true, the event indicates that the connection no longer exists. - /// This must therefore be the last event sent using this [`quinn_proto::ConnectionHandle`]. - pub async fn report_quinn_event( - &self, - connection_id: quinn_proto::ConnectionHandle, - event: quinn_proto::EndpointEvent, - ) { - self.to_endpoint - .lock() - .await - .send(ToEndpoint::ProcessConnectionEvent { - connection_id, - event, - }) - .await - .expect("background task has crashed"); - } - /// Similar to [`Endpoint::report_quinn_event`], except that the message sending is guaranteed /// to be instantaneous and to succeed. /// @@ -235,7 +196,7 @@ impl Endpoint { } /// Message sent to the endpoint background task. #[derive(Debug)] -enum ToEndpoint { +pub enum ToEndpoint { /// Instruct the endpoint to start connecting to the given address. Dial { /// UDP address to connect to. diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 637bf9a0a41..d2c666bf8a3 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -102,8 +102,11 @@ impl StreamMuxer for QuicMuxer { let mut inner = self.inner.lock(); while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { - ConnectionEvent::Connected => { - tracing::warn!("Unexpected Connected event on established QUIC connection"); + ConnectionEvent::Connected | ConnectionEvent::HandshakeDataReady => { + tracing::warn!( + "Unexpected event {:?} on established QUIC connection", + event + ); } ConnectionEvent::ConnectionLost(_) => { if let Some(waker) = inner.poll_close_waker.take() { diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 4114c8c2df2..c1cc50110d5 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -26,7 +26,7 @@ use crate::{ transport, }; -use futures::prelude::*; +use futures::{prelude::*, ready}; use libp2p_core::PeerId; use std::{ fmt, @@ -57,29 +57,25 @@ impl Future for Upgrade { .as_mut() .expect("Future polled after it has completed"); - let event = Connection::poll_event(connection, cx); - match event { - Poll::Pending => Poll::Pending, - Poll::Ready(ConnectionEvent::Connected) => { - let peer_id = connection.remote_peer_id(); - let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); - Poll::Ready(Ok((peer_id, muxer))) - } - Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { - Poll::Ready(Err(transport::Error::Established(err))) - } - // Other items are: - // - StreamAvailable - // - StreamOpened - // - StreamReadable - // - StreamWritable - // - StreamFinished - // - StreamStopped - Poll::Ready(_) => { - // They can happen only after we finished handshake and connected to the peer. - // But for `Upgrade` we get `Connected` event, wrap connection into a muxer - // and pass it to the result Stream of muxers. - unreachable!() + loop { + match ready!(connection.poll_event(cx)) { + ConnectionEvent::Connected => { + let peer_id = connection.remote_peer_id(); + let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); + return Poll::Ready(Ok((peer_id, muxer))); + } + ConnectionEvent::ConnectionLost(err) => { + return Poll::Ready(Err(transport::Error::Established(err))) + } + // Other items are: + // - HandshakeDataReady + // - StreamAvailable + // - StreamOpened + // - StreamReadable + // - StreamWritable + // - StreamFinished + // - StreamStopped + _ => {} } } } From 67b52aaa836c34e4d9c72f29d00c05f907141663 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sun, 21 Aug 2022 17:15:19 +0200 Subject: [PATCH 080/218] transports/quic: rm `Connection::is_handshaking` --- transports/quic/src/connection.rs | 24 ------------------------ transports/quic/src/muxer.rs | 6 ------ 2 files changed, 30 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index afb2b02ce84..17f3b7a0d72 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -59,13 +59,6 @@ pub struct Connection { connection_id: quinn_proto::ConnectionHandle, /// `Future` that triggers at the `Instant` that `self.connection.poll_timeout()` indicates. next_timeout: Option, - - /// In other to avoid race conditions where a "connected" event happens if we were not - /// handshaking, we cache whether the connection is handshaking and only set this to true - /// after a "connected" event has been received. - /// - /// In other words, this flag indicates whether a "connected" hasn't been received yet. - is_handshaking: bool, /// Contains a `Some` if the connection is closed, with the reason of the closure. /// Contains `None` if it is still open. /// Contains `Some` if and only if a `ConnectionLost` event has been emitted. @@ -108,8 +101,6 @@ impl Connection { from_endpoint: mpsc::Receiver, ) -> Self { assert!(!connection.is_closed()); - let is_handshaking = connection.is_handshaking(); - Connection { to_endpoint: endpoint.to_endpoint2.clone(), endpoint, @@ -118,7 +109,6 @@ impl Connection { next_timeout: None, from_endpoint, connection_id, - is_handshaking, closed: None, } } @@ -146,16 +136,9 @@ impl Connection { self.connection.remote_address() } - /// Returns `true` if this connection is still pending. Returns `false` if we are connected to - /// the remote or if the connection is closed. - pub fn is_handshaking(&self) -> bool { - self.is_handshaking - } - /// Returns the address of the node we're connected to. /// Panics if the connection is still handshaking. pub fn remote_peer_id(&self) -> PeerId { - debug_assert!(!self.is_handshaking()); let session = self.connection.crypto_session(); let identity = session .peer_identity() @@ -303,16 +286,9 @@ impl Connection { match self.connection.poll() { Some(ev) => match ConnectionEvent::try_from(ev) { Ok(ConnectionEvent::ConnectionLost(err)) => { - self.is_handshaking = false; self.closed = Some(err.clone()); return Poll::Ready(ConnectionEvent::ConnectionLost(err)); } - Ok(ConnectionEvent::Connected) => { - debug_assert!(self.is_handshaking); - debug_assert!(!self.connection.is_handshaking()); - self.is_handshaking = false; - return Poll::Ready(ConnectionEvent::Connected); - } Ok(event) => return Poll::Ready(event), Err(_proto_ev) => { // unreachable: We don't use datagrams or unidirectional streams. diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index d2c666bf8a3..b5e34524ba5 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -73,13 +73,7 @@ struct SubstreamState { impl QuicMuxer { /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. - /// - /// # Panic - /// - /// Panics if `connection.is_handshaking()` returns `true`. pub(crate) fn from_connection(connection: Connection) -> Self { - assert!(!connection.is_handshaking()); - QuicMuxer { inner: Arc::new(Mutex::new(Inner { connection, From 66974fc414aaffb372845d0b59a0eaa8551df187 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Mon, 22 Aug 2022 04:52:27 +0200 Subject: [PATCH 081/218] transports/quic: refactor connection closing --- transports/quic/src/connection.rs | 52 +++------ transports/quic/src/endpoint.rs | 23 +--- transports/quic/src/muxer.rs | 183 +++++++++++------------------- transports/quic/tests/smoke.rs | 17 ++- 4 files changed, 96 insertions(+), 179 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 17f3b7a0d72..a00b9e093d4 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -59,10 +59,6 @@ pub struct Connection { connection_id: quinn_proto::ConnectionHandle, /// `Future` that triggers at the `Instant` that `self.connection.poll_timeout()` indicates. next_timeout: Option, - /// Contains a `Some` if the connection is closed, with the reason of the closure. - /// Contains `None` if it is still open. - /// Contains `Some` if and only if a `ConnectionLost` event has been emitted. - closed: Option, to_endpoint: mpsc::Sender, } @@ -109,7 +105,6 @@ impl Connection { next_timeout: None, from_endpoint, connection_id, - closed: None, } } @@ -137,7 +132,6 @@ impl Connection { } /// Returns the address of the node we're connected to. - /// Panics if the connection is still handshaking. pub fn remote_peer_id(&self) -> PeerId { let session = self.connection.crypto_session(); let identity = session @@ -157,15 +151,10 @@ impl Connection { /// Start closing the connection. A [`ConnectionEvent::ConnectionLost`] event will be /// produced in the future. pub fn close(&mut self) { - // TODO: what if the user calls this multiple times? // We send a dummy `0` error code with no message, as the API of StreamMuxer doesn't // support this. self.connection .close(Instant::now(), From::from(0u32), Default::default()); - self.endpoint.report_quinn_event_non_block( - self.connection_id, - quinn_proto::EndpointEvent::drained(), - ); } /// Pops a new substream opened by the remote. @@ -206,12 +195,7 @@ impl Connection { /// Polls the connection for an event that happend on it. pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { - // Nothing more can be done if the connection is closed. - // Return `Pending` without registering the waker, essentially freezing the task forever. - if self.closed.is_some() { - return Poll::Pending; - } - + let mut closed = None; loop { match self.from_endpoint.poll_next_unpin(cx) { Poll::Ready(Some(event)) => { @@ -219,10 +203,9 @@ impl Connection { continue; } Poll::Ready(None) => { - debug_assert!(self.closed.is_none()); - let err = Error::ClosedChannel; - self.closed = Some(err.clone()); - return Poll::Ready(ConnectionEvent::ConnectionLost(err)); + if closed.is_none() { + return Poll::Ready(ConnectionEvent::ConnectionLost(Error::ClosedChannel)); + } } Poll::Pending => {} } @@ -241,6 +224,7 @@ impl Connection { self.to_endpoint .start_send(self.pending_to_endpoint.take().expect("is_some")) .expect("To be ready"); + continue; } Poll::Ready(Err(_)) => todo!(), Poll::Pending => return Poll::Pending, @@ -282,18 +266,21 @@ impl Connection { continue; } + if let Some(closed) = closed { + return Poll::Ready(ConnectionEvent::ConnectionLost(closed)); + } + // The final step consists in handling the events related to the various substreams. match self.connection.poll() { Some(ev) => match ConnectionEvent::try_from(ev) { - Ok(ConnectionEvent::ConnectionLost(err)) => { - self.closed = Some(err.clone()); - return Poll::Ready(ConnectionEvent::ConnectionLost(err)); - } - Ok(event) => return Poll::Ready(event), - Err(_proto_ev) => { - // unreachable: We don't use datagrams or unidirectional streams. + Ok(ConnectionEvent::ConnectionLost(reason)) => { + // Continue in the loop once more so that we can send a + // `EndpointEvent::drained` to the endpoint before returning. + closed = Some(reason); continue; } + Ok(event) => return Poll::Ready(event), + Err(_) => unreachable!("We don't use datagrams or unidirectional streams."), }, None => {} } @@ -308,15 +295,6 @@ impl fmt::Debug for Connection { } } -impl Drop for Connection { - fn drop(&mut self) { - let is_drained = self.connection.is_drained(); - if !is_drained { - self.close(); - } - } -} - /// Event generated by the [`Connection`]. #[derive(Debug)] pub enum ConnectionEvent { diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b7ca5f2b511..83555bb9cd0 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -171,29 +171,8 @@ impl Endpoint { .expect("background task has crashed"); rx.await.expect("background task has crashed") } - - /// Similar to [`Endpoint::report_quinn_event`], except that the message sending is guaranteed - /// to be instantaneous and to succeed. - /// - /// This method bypasses back-pressure mechanisms and is meant to be called only from - /// destructors, where waiting is not advisable. - pub fn report_quinn_event_non_block( - &self, - connection_id: quinn_proto::ConnectionHandle, - event: quinn_proto::EndpointEvent, - ) { - // We implement this by cloning the `mpsc::Sender`. Since each sender is guaranteed a slot - // in the buffer, cloning the sender reserves the slot and sending thus always succeeds. - let result = self - .to_endpoint2 - .clone() - .try_send(ToEndpoint::ProcessConnectionEvent { - connection_id, - event, - }); - assert!(result.is_ok()); - } } + /// Message sent to the endpoint background task. #[derive(Debug)] pub enum ToEndpoint { diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index b5e34524ba5..f3daaf0c657 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -26,9 +26,9 @@ use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ collections::HashMap, - io, + io::{self, Write}, pin::Pin, - sync::{Arc, Weak}, + sync::Arc, task::{Context, Poll, Waker}, }; @@ -52,8 +52,8 @@ struct Inner { poll_outbound_waker: Option, /// Waker to wake if a new inbound substream was happened. poll_inbound_waker: Option, - /// Waker to wake if the connection is closed. - poll_close_waker: Option, + /// Waker to wake if the connection should be polled again. + poll_connection_waker: Option, } /// State of a single substream. @@ -63,10 +63,6 @@ struct SubstreamState { read_waker: Option, /// Waker to wake if the substream becomes writable or stopped. write_waker: Option, - /// True if the substream has been finished. - finished: bool, - /// True if the substream has been stopped. - stopped: bool, /// Waker to wake if the substream becomes closed or stopped. finished_waker: Option, } @@ -79,8 +75,8 @@ impl QuicMuxer { connection, substreams: Default::default(), poll_outbound_waker: None, - poll_close_waker: None, poll_inbound_waker: None, + poll_connection_waker: None, })), } } @@ -102,13 +98,9 @@ impl StreamMuxer for QuicMuxer { event ); } - ConnectionEvent::ConnectionLost(_) => { - if let Some(waker) = inner.poll_close_waker.take() { - waker.wake(); - } - inner.connection.close(); + ConnectionEvent::ConnectionLost(err) => { + return Poll::Ready(Err(Error::ConnectionLost(err))) } - ConnectionEvent::StreamOpened => { if let Some(waker) = inner.poll_outbound_waker.take() { waker.wake(); @@ -128,19 +120,17 @@ impl StreamMuxer for QuicMuxer { } } } - ConnectionEvent::StreamFinished(substream) => { + ConnectionEvent::StreamFinished(substream) + | ConnectionEvent::StreamStopped(substream) => { if let Some(substream) = inner.substreams.get_mut(&substream) { - substream.finished = true; + if let Some(waker) = substream.write_waker.take() { + waker.wake(); + } if let Some(waker) = substream.finished_waker.take() { waker.wake(); } } } - ConnectionEvent::StreamStopped(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { - substream.stopped = true; - } - } ConnectionEvent::StreamAvailable => { if let Some(waker) = inner.poll_inbound_waker.take() { waker.wake(); @@ -148,6 +138,7 @@ impl StreamMuxer for QuicMuxer { } } } + inner.poll_connection_waker = Some(cx.waker().clone()); // TODO: poll address change Poll::Pending } @@ -158,7 +149,10 @@ impl StreamMuxer for QuicMuxer { ) -> Poll> { let mut inner = self.inner.lock(); let substream_id = match inner.connection.pop_incoming_substream() { - Some(id) => id, + Some(id) => { + inner.poll_outbound_waker = None; + id + } None => { inner.poll_inbound_waker = Some(cx.waker().clone()); return Poll::Pending; @@ -175,7 +169,10 @@ impl StreamMuxer for QuicMuxer { ) -> Poll> { let mut inner = self.inner.lock(); let substream_id = match inner.connection.pop_outgoing_substream() { - Some(id) => id, + Some(id) => { + inner.poll_outbound_waker = None; + id + } None => { inner.poll_outbound_waker = Some(cx.waker().clone()); return Poll::Pending; @@ -192,55 +189,37 @@ impl StreamMuxer for QuicMuxer { return Poll::Ready(Ok(())); } - if inner.substreams.is_empty() { - let connection = &mut inner.connection; - if !connection.connection.is_closed() { - connection.close(); - if let Some(waker) = inner.poll_inbound_waker.take() { - waker.wake(); - } - } else { - } - while let Poll::Ready(event) = inner.connection.poll_event(cx) { - if let ConnectionEvent::ConnectionLost(_) = event { - return Poll::Ready(Ok(())); - } - } - } else { + if inner.connection.connection.streams().send_streams() != 0 { for substream in inner.substreams.keys().cloned().collect::>() { if let Err(e) = inner.connection.shutdown_substream(substream) { tracing::warn!("substream finish error on muxer close: {}", e); } } } - - // Register `cx.waker()` as being woken up if the connection closes. - inner.poll_close_waker = Some(cx.waker().clone()); - + loop { + if inner.connection.connection.streams().send_streams() == 0 + && !inner.connection.connection.is_closed() + { + inner.connection.close() + } + match inner.connection.poll_event(cx) { + Poll::Ready(ConnectionEvent::ConnectionLost(_)) => return Poll::Ready(Ok(())), + Poll::Ready(_) => {} + Poll::Pending => break, + } + } Poll::Pending } } pub struct Substream { id: quinn_proto::StreamId, - muxer: Weak>, + muxer: Arc>, } impl Substream { fn new(id: quinn_proto::StreamId, muxer: Arc>) -> Self { - Self { - id, - muxer: Arc::downgrade(&muxer), - } - } -} - -impl Drop for Substream { - fn drop(&mut self) { - if let Some(muxer) = self.muxer.upgrade() { - let mut muxer = muxer.lock(); - muxer.substreams.remove(&self.id); - } + Self { id, muxer } } } @@ -249,34 +228,20 @@ impl AsyncRead for Substream { self: Pin<&mut Self>, cx: &mut Context<'_>, mut buf: &mut [u8], - ) -> Poll> { + ) -> Poll> { use quinn_proto::{ReadError, ReadableError}; - use std::io::Write; - - let muxer = self - .muxer - .upgrade() - .expect("StreamMuxer::read_substream: muxer is dead"); - let mut muxer = muxer.lock(); - - let substream_state = muxer - .substreams - .get(&self.id) - .expect("invalid Substream::poll_read API usage"); - if substream_state.stopped { - return Poll::Ready(Ok(0)); - } + let mut muxer = self.muxer.lock(); let mut stream = muxer.connection.connection.recv_stream(self.id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(ReadableError::UnknownStream) => { - return Poll::Ready(Ok(0)); // FIXME This is a hack, - // a rust client should close substream correctly - // return Poll::Ready(Err(Self::Error::ExpiredStream)) + return Poll::Ready(Ok(0)); } Err(ReadableError::IllegalOrderedRead) => { - panic!("Illegal ordered read can only happen if `stream.read(false)` is used."); + unreachable!( + "Illegal ordered read can only happen if `stream.read(false)` is used." + ); } }; let mut bytes = 0; @@ -290,9 +255,7 @@ impl AsyncRead for Substream { buf.write_all(&chunk.bytes).expect("enough buffer space"); bytes += chunk.bytes.len(); } - Ok(None) => { - break; - } + Ok(None) => break, Err(ReadError::Reset(error_code)) => { tracing::error!( "substream {} was reset with error code {}", @@ -309,12 +272,12 @@ impl AsyncRead for Substream { } } if chunks.finalize().should_transmit() { - if let Some(waker) = muxer.poll_inbound_waker.take() { + if let Some(waker) = muxer.poll_connection_waker.take() { waker.wake(); } } if pending && bytes == 0 { - let mut substream_state = muxer + let substream_state = muxer .substreams .get_mut(&self.id) .expect("known substream; qed"); @@ -332,33 +295,23 @@ impl AsyncWrite for Substream { cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - use quinn_proto::WriteError; - - let muxer = self - .muxer - .upgrade() - .expect("Substream::poll_write: muxer is dead"); - let mut muxer = muxer.lock(); + let mut muxer = self.muxer.lock(); match muxer.connection.connection.send_stream(self.id).write(buf) { Ok(bytes) => Poll::Ready(Ok(bytes)), - Err(WriteError::Blocked) => { - let mut substream = muxer + Err(quinn_proto::WriteError::Blocked) => { + let substream = muxer .substreams .get_mut(&self.id) .expect("known substream; qed"); substream.write_waker = Some(cx.waker().clone()); Poll::Pending } - Err(err @ WriteError::Stopped(_)) => { + Err(err @ quinn_proto::WriteError::Stopped(_)) => { Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } - Err(WriteError::UnknownStream) => { - tracing::error!( - "The application used a connection that is already being \ - closed. This is a bug in the application or in libp2p." - ); - Poll::Pending + Err(quinn_proto::WriteError::UnknownStream) => { + Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())) } } } @@ -369,35 +322,27 @@ impl AsyncWrite for Substream { } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let muxer = self - .muxer - .upgrade() - .expect("Substream::poll_close: muxer is dead"); - let mut muxer = muxer.lock(); - let muxer = &mut *muxer; - - let mut substream_state = muxer - .substreams - .get_mut(&self.id) - .expect("invalid Substream::poll_close API usage"); - if substream_state.finished { - return Poll::Ready(Ok(())); - } - + let mut muxer = self.muxer.lock(); match muxer.connection.shutdown_substream(self.id) { Ok(()) => { + let substream_state = muxer + .substreams + .get_mut(&self.id) + .expect("Substream is not finished."); substream_state.finished_waker = Some(cx.waker().clone()); Poll::Pending } Err(err @ quinn_proto::FinishError::Stopped(_)) => { Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } - Err(quinn_proto::FinishError::UnknownStream) => { - // Illegal usage of the API. - debug_assert!(false); - Poll::Ready(Ok(())) - // Poll::Ready(Err(Error::ExpiredStream)) FIXME - } + Err(quinn_proto::FinishError::UnknownStream) => Poll::Ready(Ok(())), } } } + +impl Drop for Substream { + fn drop(&mut self) { + let mut muxer = self.muxer.lock(); + muxer.substreams.remove(&self.id); + } +} diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index bb4f1270950..bd8d0682dd9 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -13,7 +13,7 @@ use libp2p::request_response::{ RequestResponseEvent, RequestResponseMessage, }; use libp2p::swarm::dial_opts::{DialOpts, PeerCondition}; -use libp2p::swarm::{DialError, Swarm, SwarmEvent}; +use libp2p::swarm::{ConnectionError, DialError, Swarm, SwarmEvent}; use libp2p_quic::{Config as QuicConfig, QuicTransport}; use rand::RngCore; use std::num::NonZeroU8; @@ -179,6 +179,21 @@ async fn smoke() -> Result<()> { e => panic!("{:?}", e), } + a.disconnect_peer_id(*b.local_peer_id()).unwrap(); + + match a.next().await { + Some(SwarmEvent::ConnectionClosed { cause: None, .. }) => {} + e => panic!("{:?}", e), + } + + match b.next().await { + Some(SwarmEvent::ConnectionClosed { + cause: Some(ConnectionError::IO(_)), + .. + }) => {} + e => panic!("{:?}", e), + } + Ok(()) } From c88efe84c75d7f52964d22ed82ebe4dff2418f16 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Mon, 22 Aug 2022 06:32:07 +0200 Subject: [PATCH 082/218] transports/quic: rm mutex around to_endpoint tx When sending a message to the background task, don't use the `SinkExt::send` future since this requires static ownership of the channel sender. Instead use `Sink::poll_ready` and `Sink::start_send`, which allows us to inline the sending of messages into our poll functions. With this we don't need the Mutex around `to_endpoint` anymore. The background task, the transport, and each connections owns exactly one `Sender`, thus we don't have unbounded growth here. --- transports/quic/src/connection.rs | 25 ++-- transports/quic/src/endpoint.rs | 78 ++++--------- transports/quic/src/transport.rs | 182 ++++++++++++++++++++++-------- 3 files changed, 163 insertions(+), 122 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index a00b9e093d4..b0290fe4ac5 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -34,7 +34,6 @@ use libp2p_core::PeerId; use std::{ fmt, net::SocketAddr, - sync::Arc, task::{Context, Poll}, time::Instant, }; @@ -45,7 +44,7 @@ use std::{ /// Tied to a specific [`Endpoint`]. pub struct Connection { /// Endpoint this connection belongs to. - endpoint: Arc, + endpoint: Endpoint, /// Future whose job is to send a message to the endpoint. Only one at a time. pending_to_endpoint: Option, /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. @@ -59,8 +58,6 @@ pub struct Connection { connection_id: quinn_proto::ConnectionHandle, /// `Future` that triggers at the `Instant` that `self.connection.poll_timeout()` indicates. next_timeout: Option, - - to_endpoint: mpsc::Sender, } /// Error on the connection as a whole. @@ -91,14 +88,13 @@ impl Connection { /// its methods has ever been called. Failure to comply might lead to logic errors and panics. // TODO: maybe abstract `to_endpoint` more and make it generic? dunno pub fn from_quinn_connection( - endpoint: Arc, + endpoint: Endpoint, connection: quinn_proto::Connection, connection_id: quinn_proto::ConnectionHandle, from_endpoint: mpsc::Receiver, ) -> Self { assert!(!connection.is_closed()); Connection { - to_endpoint: endpoint.to_endpoint2.clone(), endpoint, pending_to_endpoint: None, connection, @@ -113,7 +109,7 @@ impl Connection { /// Works for server connections only. pub fn local_addr(&self) -> SocketAddr { debug_assert_eq!(self.connection.side(), quinn_proto::Side::Server); - let endpoint_addr = self.endpoint.socket_addr(); + let endpoint_addr = self.endpoint.socket_addr; self.connection .local_ip() .map(|ip| SocketAddr::new(ip, endpoint_addr.port())) @@ -121,7 +117,7 @@ impl Connection { // In a normal case scenario this should not happen, because // we get want to get a local addr for a server connection only. tracing::error!("trying to get quinn::local_ip for a client"); - *endpoint_addr + endpoint_addr }) } @@ -219,14 +215,15 @@ impl Connection { // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. if self.pending_to_endpoint.is_some() { - match self.to_endpoint.poll_ready_unpin(cx) { + match self.endpoint.to_endpoint.poll_ready_unpin(cx) { Poll::Ready(Ok(())) => { - self.to_endpoint - .start_send(self.pending_to_endpoint.take().expect("is_some")) - .expect("To be ready"); - continue; + let to_endpoint = self.pending_to_endpoint.take().expect("is some"); + self.endpoint + .to_endpoint + .start_send(to_endpoint) + .expect("Channel is ready."); } - Poll::Ready(Err(_)) => todo!(), + Poll::Ready(Err(_)) => panic!("Background task crashed"), Poll::Pending => return Poll::Pending, } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 83555bb9cd0..a65da3bae60 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -32,7 +32,6 @@ use crate::{connection::Connection, tls, transport}; use futures::{ channel::{mpsc, oneshot}, - lock::Mutex, prelude::*, }; use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; @@ -40,7 +39,7 @@ use std::{ collections::{HashMap, VecDeque}, fmt, net::{Ipv4Addr, Ipv6Addr, SocketAddr, UdpSocket}, - sync::{Arc, Weak}, + sync::Arc, task::{Poll, Waker}, time::{Duration, Instant}, }; @@ -85,15 +84,12 @@ impl Config { // TODO: expand docs // TODO: Debug trait // TODO: remove useless fields +#[derive(Clone)] pub struct Endpoint { /// Channel to the background of the endpoint. - to_endpoint: Mutex>, + pub to_endpoint: mpsc::Sender, - /// Copy of [`Endpoint::to_endpoint`], except not behind a `Mutex`. Used if we want to be - /// guaranteed a slot in the messages buffer. - pub to_endpoint2: mpsc::Sender, - - socket_addr: SocketAddr, + pub socket_addr: SocketAddr, } impl Endpoint { @@ -101,14 +97,14 @@ impl Endpoint { pub fn new_bidirectional( config: Config, socket_addr: SocketAddr, - ) -> Result<(Arc, mpsc::Receiver), transport::Error> { + ) -> Result<(Endpoint, mpsc::Receiver), transport::Error> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Self::new(config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) } /// Builds a new [`Endpoint`] that only supports outbound connections. - pub fn new_dialer(config: Config, is_ipv6: bool) -> Result, transport::Error> { + pub fn new_dialer(config: Config, is_ipv6: bool) -> Result { let socket_addr = if is_ipv6 { SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0) } else { @@ -121,17 +117,15 @@ impl Endpoint { config: Config, socket_addr: SocketAddr, new_connections: Option>, - ) -> Result, transport::Error> { + ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(32); - let to_endpoint2 = to_endpoint_tx.clone(); - let endpoint = Arc::new(Endpoint { - to_endpoint: Mutex::new(to_endpoint_tx), - to_endpoint2, + let endpoint = Endpoint { + to_endpoint: to_endpoint_tx, socket_addr: socket.local_addr()?, - }); + }; let server_config = new_connections.map(|c| (c, config.server_config.clone())); @@ -140,7 +134,7 @@ impl Endpoint { config.endpoint_config, config.client_config, server_config, - Arc::downgrade(&endpoint), + endpoint.clone(), async_io::Async::::new(socket)?, to_endpoint_rx.fuse(), )) @@ -148,29 +142,6 @@ impl Endpoint { Ok(endpoint) } - - pub fn socket_addr(&self) -> &SocketAddr { - &self.socket_addr - } - - /// Asks the endpoint to start dialing the given address. - /// - /// Note that this method only *starts* the dialing. `Ok` is returned as soon as possible, even - /// when the remote might end up being unreachable. - pub async fn dial(&self, addr: SocketAddr) -> Result { - // The two `expect`s below can panic if the background task has stopped. The background - // task can stop only if the `Endpoint` is destroyed or if the task itself panics. In other - // words, we panic here iff a panic has already happened somewhere else, which is a - // reasonable thing to do. - let (tx, rx) = oneshot::channel(); - self.to_endpoint - .lock() - .await - .send(ToEndpoint::Dial { addr, result: tx }) - .await - .expect("background task has crashed"); - rx.await.expect("background task has crashed") - } } /// Message sent to the endpoint background task. @@ -274,7 +245,7 @@ pub enum ToEndpoint { /// The background task shuts down if `endpoint_weak`, `receiver` or `new_connections` become /// disconnected/invalid. This corresponds to the lifetime of the associated [`Endpoint`]. /// -/// Keep in mind that we pass an `Arc` whenever we create a new connection, which +/// Keep in mind that we pass an `Endpoint` whenever we create a new connection, which /// guarantees that the [`Endpoint`], and therefore the background task, is properly kept alive /// for as long as any QUIC connection is open. /// @@ -282,7 +253,7 @@ async fn background_task( endpoint_config: Arc, client_config: quinn_proto::ClientConfig, server_config: Option<(mpsc::Sender, Arc)>, - endpoint_weak: Weak, + endpoint: Endpoint, udp_socket: async_io::Async, mut receiver: stream::Fuse>, ) { @@ -292,7 +263,7 @@ async fn background_task( }; // The actual QUIC state machine. - let mut endpoint = quinn_proto::Endpoint::new(endpoint_config.clone(), server_config); + let mut proto_endpoint = quinn_proto::Endpoint::new(endpoint_config.clone(), server_config); // List of all active connections, with a sender to notify them of events. let mut alive_connections = HashMap::>::new(); @@ -339,7 +310,7 @@ async fn background_task( // The endpoint might request packets to be sent out. This is handled in priority to avoid // buffering up packets. - if let Some(packet) = endpoint.poll_transmit() { + if let Some(packet) = proto_endpoint.poll_transmit() { debug_assert!(next_packet_out.is_none()); next_packet_out = Some((packet.destination, packet.contents)); continue; @@ -358,7 +329,7 @@ async fn background_task( // name. While we don't use domain names, the underlying rustls library // is based upon the assumption that we do. let (connection_id, connection) = - match endpoint.connect(client_config.clone(), addr, "l") { + match proto_endpoint.connect(client_config.clone(), addr, "l") { Ok(c) => c, Err(err) => { let _ = result.send(Err(err)); @@ -366,14 +337,9 @@ async fn background_task( } }; - let endpoint_arc = match endpoint_weak.upgrade() { - Some(ep) => ep, - None => return, // Shut down the task if the endpoint is dead. - }; - debug_assert_eq!(connection.side(), quinn_proto::Side::Client); let (tx, rx) = mpsc::channel(16); - let connection = Connection::from_quinn_connection(endpoint_arc, connection, connection_id, rx); + let connection = Connection::from_quinn_connection(endpoint.clone(), connection, connection_id, rx); alive_connections.insert(connection_id, tx); let _ = result.send(Ok(connection)); } @@ -391,7 +357,7 @@ async fn background_task( alive_connections.remove(&connection_id); } - let event_back = endpoint.handle_event(connection_id, event); + let event_back = proto_endpoint.handle_event(connection_id, event); if let Some(event_back) = event_back { debug_assert!(!is_drained_event); @@ -467,7 +433,7 @@ async fn background_task( let packet = From::from(&socket_recv_buffer[..packet_len]); let local_ip = udp_socket.get_ref().local_addr().ok().map(|a| a.ip()); // TODO: ECN bits aren't handled - let event = endpoint.handle(Instant::now(), packet_src, local_ip, None, packet); + let event = proto_endpoint.handle(Instant::now(), packet_src, local_ip, None, packet); match event { None => {}, @@ -485,11 +451,7 @@ async fn background_task( debug_assert_eq!(connec.side(), quinn_proto::Side::Server); let (tx, rx) = mpsc::channel(16); alive_connections.insert(connec_id, tx); - let endpoint_arc = match endpoint_weak.upgrade() { - Some(ep) => ep, - None => return, // Shut down the task if the endpoint is dead. - }; - let connection = Connection::from_quinn_connection(endpoint_arc, connec, connec_id, rx); + let connection = Connection::from_quinn_connection(endpoint.clone(), connec, connec_id, rx); // As explained in the documentation, we put this new connection in an // intermediary buffer. At the next loop iteration we will try to move it diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 8c0f7402f54..26dcac29109 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -23,9 +23,11 @@ //! Combines all the objects in the other modules to implement the trait. use crate::connection::Connection; +use crate::endpoint::ToEndpoint; use crate::Config; use crate::{endpoint::Endpoint, in_addr::InAddr, muxer::QuicMuxer, upgrade::Upgrade}; +use futures::channel::oneshot; use futures::stream::StreamExt; use futures::{channel::mpsc, prelude::*, stream::SelectAll}; @@ -36,10 +38,11 @@ use libp2p_core::{ transport::{ListenerId, TransportError, TransportEvent}, PeerId, Transport, }; +use std::collections::VecDeque; +use std::task::Waker; use std::{ net::SocketAddr, pin::Pin, - sync::Arc, task::{Context, Poll}, }; @@ -55,10 +58,10 @@ pub use quinn_proto::{ pub struct QuicTransport { config: Config, listeners: SelectAll, - /// Endpoints to use for dialing Ipv4 addresses if no matching listener exists. - ipv4_dialer: Option>, - /// Endpoints to use for dialing Ipv6 addresses if no matching listener exists. - ipv6_dialer: Option>, + /// Dialer for Ipv4 addresses if no matching listener exists. + ipv4_dialer: Option, + /// Dialer for Ipv6 addresses if no matching listener exists. + ipv6_dialer: Option, } impl QuicTransport { @@ -132,38 +135,49 @@ impl Transport for QuicTransport { tracing::error!("multiaddr not supported"); return Err(TransportError::MultiaddrNotSupported(addr)); } - let listeners = self + let mut listeners = self .listeners - .iter() + .iter_mut() .filter(|l| { - let listen_addr = l.endpoint.socket_addr(); + let listen_addr = l.endpoint.socket_addr; listen_addr.is_ipv4() == socket_addr.is_ipv4() && listen_addr.ip().is_loopback() == socket_addr.ip().is_loopback() }) .collect::>(); - let endpoint = if listeners.is_empty() { + + let (tx, rx) = oneshot::channel(); + let to_endpoint = ToEndpoint::Dial { + addr: socket_addr, + result: tx, + }; + if listeners.is_empty() { let dialer = match socket_addr { SocketAddr::V4(_) => &mut self.ipv4_dialer, SocketAddr::V6(_) => &mut self.ipv6_dialer, }; - match dialer { - Some(endpoint) => endpoint.clone(), - None => { - let endpoint = Endpoint::new_dialer(self.config.clone(), socket_addr.is_ipv6()) - .map_err(TransportError::Other)?; - let _ = dialer.insert(endpoint.clone()); - endpoint - } + if dialer.is_none() { + let _ = dialer.insert(Dialer::new(self.config.clone(), socket_addr.is_ipv6())?); } + dialer + .as_mut() + .unwrap() + .pending_dials + .push_back(to_endpoint); } else { // Pick a random listener to use for dialing. let n = rand::random::() % listeners.len(); - let listener = listeners.get(n).expect("Can not be out of bound."); - listener.endpoint.clone() + let listener = listeners.get_mut(n).expect("Can not be out of bound."); + listener.pending_dials.push_back(to_endpoint); + if let Some(waker) = listener.waker.take() { + waker.wake() + } }; Ok(async move { - let connection = endpoint.dial(socket_addr).await.map_err(Error::Reach)?; + let connection = rx + .await + .expect("background task has crashed") + .map_err(Error::Reach)?; let final_connec = Upgrade::from_connection(connection).await?; Ok(final_connec) } @@ -185,6 +199,12 @@ impl Transport for QuicTransport { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { + if let Some(dialer) = self.ipv4_dialer.as_mut() { + dialer.drive_dials(cx) + } + if let Some(dialer) = self.ipv6_dialer.as_mut() { + dialer.drive_dials(cx) + } match self.listeners.poll_next_unpin(cx) { Poll::Ready(Some(ev)) => Poll::Ready(ev), _ => Poll::Pending, @@ -192,9 +212,41 @@ impl Transport for QuicTransport { } } +#[derive(Debug)] +struct Dialer { + endpoint: Endpoint, + pending_dials: VecDeque, +} + +impl Dialer { + fn new(config: Config, is_ipv6: bool) -> Result> { + let endpoint = Endpoint::new_dialer(config, is_ipv6).map_err(TransportError::Other)?; + Ok(Dialer { + endpoint, + pending_dials: VecDeque::new(), + }) + } + + fn drive_dials(&mut self, cx: &mut Context<'_>) { + if !self.pending_dials.is_empty() { + match self.endpoint.to_endpoint.poll_ready_unpin(cx) { + Poll::Ready(Ok(())) => { + let to_endpoint = self.pending_dials.pop_front().expect("!is_empty"); + self.endpoint + .to_endpoint + .start_send(to_endpoint) + .expect("Channel is ready."); + } + Poll::Ready(Err(_)) => panic!("Background task crashed."), + Poll::Pending => {} + } + } + } +} + #[derive(Debug)] struct Listener { - endpoint: Arc, + endpoint: Endpoint, listener_id: ListenerId, @@ -212,6 +264,10 @@ struct Listener { /// Optionally contains a [`TransportEvent::ListenerClosed`] that should be /// reported before the listener's stream is terminated. report_closed: Option::Item>>, + + pending_dials: VecDeque, + + waker: Option, } impl Listener { @@ -228,6 +284,8 @@ impl Listener { new_connections_rx, in_addr, report_closed: None, + pending_dials: VecDeque::new(), + waker: None, }) } @@ -258,9 +316,9 @@ impl Listener { match item { Ok(IfEvent::Up(inet)) => { let ip = inet.addr(); - if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { let socket_addr = - SocketAddr::new(ip, self.endpoint.socket_addr().port()); + SocketAddr::new(ip, self.endpoint.socket_addr.port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("New listen address: {}", ma); return Some(TransportEvent::NewAddress { @@ -271,9 +329,9 @@ impl Listener { } Ok(IfEvent::Down(inet)) => { let ip = inet.addr(); - if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { let socket_addr = - SocketAddr::new(ip, self.endpoint.socket_addr().port()); + SocketAddr::new(ip, self.endpoint.socket_addr.port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("Expired listen address: {}", ma); return Some(TransportEvent::AddressExpired { @@ -304,32 +362,56 @@ impl Listener { impl Stream for Listener { type Item = TransportEvent<::ListenerUpgrade, Error>; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - if let Some(closed) = self.report_closed.as_mut() { - // Listener was closed. - // Report the transport event if there is one. On the next iteration, return - // `Poll::Ready(None)` to terminate the stream. - return Poll::Ready(closed.take()); - } - if let Some(event) = self.poll_if_addr(cx) { - return Poll::Ready(Some(event)); - } - let connection = match futures::ready!(self.new_connections_rx.poll_next_unpin(cx)) { - Some(c) => c, - None => { - self.close(Err(Error::TaskCrashed)); - return self.poll_next(cx); + loop { + if let Some(closed) = self.report_closed.as_mut() { + // Listener was closed. + // Report the transport event if there is one. On the next iteration, return + // `Poll::Ready(None)` to terminate the stream. + return Poll::Ready(closed.take()); } - }; - - let local_addr = socketaddr_to_multiaddr(&connection.local_addr()); - let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); - let event = TransportEvent::Incoming { - upgrade: Upgrade::from_connection(connection), - local_addr, - send_back_addr, - listener_id: self.listener_id, - }; - Poll::Ready(Some(event)) + if let Some(event) = self.poll_if_addr(cx) { + return Poll::Ready(Some(event)); + } + if !self.pending_dials.is_empty() { + match self.endpoint.to_endpoint.poll_ready_unpin(cx) { + Poll::Ready(Ok(_)) => { + let to_endpoint = self + .pending_dials + .pop_front() + .expect("Pending dials is not empty."); + self.endpoint + .to_endpoint + .start_send(to_endpoint) + .expect("Channel is ready"); + } + Poll::Ready(Err(_)) => { + self.close(Err(Error::TaskCrashed)); + continue; + } + Poll::Pending => {} + } + } + match self.new_connections_rx.poll_next_unpin(cx) { + Poll::Ready(Some(connection)) => { + let local_addr = socketaddr_to_multiaddr(&connection.local_addr()); + let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); + let event = TransportEvent::Incoming { + upgrade: Upgrade::from_connection(connection), + local_addr, + send_back_addr, + listener_id: self.listener_id, + }; + return Poll::Ready(Some(event)); + } + Poll::Ready(None) => { + self.close(Err(Error::TaskCrashed)); + continue; + } + Poll::Pending => {} + }; + self.waker = Some(cx.waker().clone()); + return Poll::Pending; + } } } From 0a82be48c2350ccda53f61a2a76707f303287fd8 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Mon, 22 Aug 2022 06:39:59 +0200 Subject: [PATCH 083/218] transports/quic/tests: drive peers concurrently --- transports/quic/tests/smoke.rs | 201 +++++++++++++++++---------------- 1 file changed, 103 insertions(+), 98 deletions(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index bd8d0682dd9..0291a16f9f0 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -1,6 +1,6 @@ use anyhow::Result; use async_trait::async_trait; -use futures::future::FutureExt; +use futures::future::{join, FutureExt}; use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; use futures::select; use futures::stream::StreamExt; @@ -82,118 +82,123 @@ async fn smoke() -> Result<()> { b.behaviour_mut() .send_request(&Swarm::local_peer_id(&a), Ping(data.clone())); - match b.next().await { - Some(SwarmEvent::Dialing(_)) => {} - e => panic!("{:?}", e), - } + let b_id = *b.local_peer_id(); + + let fut_a = async move { + match a.next().await { + Some(SwarmEvent::IncomingConnection { .. }) => {} + e => panic!("{:?}", e), + }; + + match a.next().await { + Some(SwarmEvent::ConnectionEstablished { .. }) => {} + e => panic!("{:?}", e), + }; + + 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::IncomingConnection { .. }) => {} - e => panic!("{:?}", e), - }; + match a.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::ResponseSent { .. })) => {} + e => panic!("{:?}", e), + } - match b.next().await { - Some(SwarmEvent::ConnectionEstablished { .. }) => {} - e => panic!("{:?}", e), - }; + a.behaviour_mut() + .send_request(&b_id, Ping(b"another substream".to_vec())); - match a.next().await { - Some(SwarmEvent::ConnectionEstablished { .. }) => {} - e => panic!("{:?}", e), - }; + assert!(a.next().now_or_never().is_none()); - 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(); + 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), } - 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()), - ); + a.disconnect_peer_id(b_id).unwrap(); - assert!(a.next().now_or_never().is_none()); + match a.next().await { + Some(SwarmEvent::ConnectionClosed { cause: None, .. }) => {} + e => panic!("{:?}", e), + } + }; - 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(); + let fut_b = async { + match b.next().await { + Some(SwarmEvent::Dialing(_)) => {} + e => panic!("{:?}", e), } - e => panic!("{:?}", e), - } - match b.next().await { - Some(SwarmEvent::Behaviour(RequestResponseEvent::ResponseSent { .. })) => {} - e => panic!("{:?}", e), - } + match b.next().await { + Some(SwarmEvent::ConnectionEstablished { .. }) => {} + 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), - } + assert!(b.next().now_or_never().is_none()); - a.disconnect_peer_id(*b.local_peer_id()).unwrap(); + match b.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { + message: + RequestResponseMessage::Response { + response: Pong(pong), + .. + }, + .. + })) => assert_eq!(data, pong), + e => panic!("{:?}", e), + } - match a.next().await { - Some(SwarmEvent::ConnectionClosed { cause: None, .. }) => {} - e => panic!("{:?}", e), - } + 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::ConnectionClosed { - cause: Some(ConnectionError::IO(_)), - .. - }) => {} - e => panic!("{:?}", e), - } + match b.next().await { + Some(SwarmEvent::Behaviour(RequestResponseEvent::ResponseSent { .. })) => {} + e => panic!("{:?}", e), + } + + match b.next().await { + Some(SwarmEvent::ConnectionClosed { + cause: Some(ConnectionError::IO(_)), + .. + }) => {} + e => panic!("{:?}", e), + } + }; + join(fut_a, fut_b).await; Ok(()) } From fe3e09b710caeaaf26a6ea83e618a19d3c2285a1 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Fri, 9 Sep 2022 23:43:29 +0200 Subject: [PATCH 084/218] transports/quic: upgrade to if-watch v2.0.0 See corresponding change in tcp transport: libp2p#2813. --- transports/quic/Cargo.toml | 2 +- transports/quic/src/in_addr.rs | 100 -------------------- transports/quic/src/lib.rs | 1 - transports/quic/src/transport.rs | 153 +++++++++++++++---------------- 4 files changed, 77 insertions(+), 179 deletions(-) delete mode 100644 transports/quic/src/in_addr.rs diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 0480c0629ed..621ce8d8e05 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -11,7 +11,7 @@ license = "MIT" async-global-executor = "2.0.2" async-io = "1.6.0" futures = "0.3.15" -if-watch = "1.0.0" +if-watch = "2.0.0" libp2p-core = { version = "0.36.0", path = "../../core" } parking_lot = "0.12.0" quinn-proto = { version = "0.8.2", default-features = false, features = ["tls-rustls"] } diff --git a/transports/quic/src/in_addr.rs b/transports/quic/src/in_addr.rs deleted file mode 100644 index 67b6abbf3f3..00000000000 --- a/transports/quic/src/in_addr.rs +++ /dev/null @@ -1,100 +0,0 @@ -use if_watch::{IfEvent, IfWatcher}; - -use futures::{ - future::{BoxFuture, FutureExt}, - stream::Stream, -}; - -use std::{ - io::Result, - net::IpAddr, - ops::DerefMut, - pin::Pin, - task::{Context, Poll}, -}; - -/// Watches for interface changes. -#[derive(Debug)] -pub enum InAddr { - /// The socket accepts connections on a single interface. - One { ip: Option }, - /// The socket accepts connections on all interfaces. - Any { if_watch: Box }, -} - -impl InAddr { - /// If ip is specified then only one `IfEvent::Up` with IpNet(ip)/32 will be generated. - /// If ip is unspecified then `IfEvent::Up/Down` events will be generated for all interfaces. - pub fn new(ip: IpAddr) -> Self { - if ip.is_unspecified() { - let watcher = IfWatch::Pending(IfWatcher::new().boxed()); - InAddr::Any { - if_watch: Box::new(watcher), - } - } else { - InAddr::One { ip: Some(ip) } - } - } -} - -pub enum IfWatch { - Pending(BoxFuture<'static, std::io::Result>), - Ready(Box), -} - -impl std::fmt::Debug for IfWatch { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - match *self { - IfWatch::Pending(_) => write!(f, "Pending"), - IfWatch::Ready(_) => write!(f, "Ready"), - } - } -} -impl Stream for InAddr { - type Item = Result; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context) -> Poll> { - let me = Pin::into_inner(self); - loop { - match me { - // If the listener is bound to a single interface, make sure the - // address is reported once. - InAddr::One { ip } => { - if let Some(ip) = ip.take() { - return Poll::Ready(Some(Ok(IfEvent::Up(ip.into())))); - } - } - InAddr::Any { if_watch } => { - match if_watch.deref_mut() { - // If we listen on all interfaces, wait for `if-watch` to be ready. - IfWatch::Pending(f) => match futures::ready!(f.poll_unpin(cx)) { - Ok(watcher) => { - *if_watch = Box::new(IfWatch::Ready(Box::new(watcher))); - continue; - } - Err(err) => { - *if_watch = Box::new(IfWatch::Pending(IfWatcher::new().boxed())); - return Poll::Ready(Some(Err(err))); - } - }, - // Consume all events for up/down interface changes. - IfWatch::Ready(watcher) => { - if let Poll::Ready(ev) = watcher.poll_unpin(cx) { - match ev { - Ok(event) => { - return Poll::Ready(Some(Ok(event))); - } - Err(err) => { - return Poll::Ready(Some(Err(err))); - } - } - } - } - } - } - } - break; - } - Poll::Pending - } -} diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 3dca1d3cbe3..2d9a4491bab 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -55,7 +55,6 @@ mod connection; mod endpoint; mod error; -mod in_addr; mod muxer; mod tls; mod upgrade; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 26dcac29109..0aa3295ac0a 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -25,13 +25,14 @@ use crate::connection::Connection; use crate::endpoint::ToEndpoint; use crate::Config; -use crate::{endpoint::Endpoint, in_addr::InAddr, muxer::QuicMuxer, upgrade::Upgrade}; +use crate::{endpoint::Endpoint, muxer::QuicMuxer, upgrade::Upgrade}; use futures::channel::oneshot; +use futures::ready; use futures::stream::StreamExt; use futures::{channel::mpsc, prelude::*, stream::SelectAll}; -use if_watch::IfEvent; +use if_watch::{IfEvent, IfWatcher}; use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, @@ -253,17 +254,13 @@ struct Listener { /// Channel where new connections are being sent. new_connections_rx: mpsc::Receiver, - /// The IP addresses of network interfaces on which the listening socket - /// is accepting connections. - /// - /// If the listen socket listens on all interfaces, these may change over - /// time as interfaces become available or unavailable. - in_addr: InAddr, + if_watcher: Option, - /// Set to `Some` if this [`Listener`] should close. - /// Optionally contains a [`TransportEvent::ListenerClosed`] that should be - /// reported before the listener's stream is terminated. - report_closed: Option::Item>>, + /// Whether the listener was closed and the stream should terminate. + is_closed: bool, + + /// Pending event to reported. + pending_event: Option<::Item>, pending_dials: VecDeque, @@ -276,14 +273,29 @@ impl Listener { socket_addr: SocketAddr, config: Config, ) -> Result { - let in_addr = InAddr::new(socket_addr.ip()); let (endpoint, new_connections_rx) = Endpoint::new_bidirectional(config, socket_addr)?; + + let if_watcher; + let pending_event; + if socket_addr.ip().is_unspecified() { + if_watcher = Some(IfWatcher::new()?); + pending_event = None; + } else { + if_watcher = None; + let ma = socketaddr_to_multiaddr(&endpoint.socket_addr); + pending_event = Some(TransportEvent::NewAddress { + listener_id, + listen_addr: ma, + }) + } + Ok(Listener { endpoint, listener_id, new_connections_rx, - in_addr, - report_closed: None, + if_watcher, + is_closed: false, + pending_event, pending_dials: VecDeque::new(), waker: None, }) @@ -292,68 +304,54 @@ impl Listener { /// Report the listener as closed in a [`TransportEvent::ListenerClosed`] and /// terminate the stream. fn close(&mut self, reason: Result<(), Error>) { - match self.report_closed { - Some(_) => tracing::debug!("Listener was already closed."), - None => { - // Report the listener event as closed. - let _ = self - .report_closed - .insert(Some(TransportEvent::ListenerClosed { - listener_id: self.listener_id, - reason, - })); - } + if self.is_closed { + return; } + self.pending_event = Some(TransportEvent::ListenerClosed { + listener_id: self.listener_id, + reason, + }); + self.is_closed = true; } /// Poll for a next If Event. - fn poll_if_addr(&mut self, cx: &mut Context<'_>) -> Option<::Item> { + fn poll_if_addr(&mut self, cx: &mut Context<'_>) -> Poll<::Item> { + let if_watcher = match self.if_watcher.as_mut() { + Some(iw) => iw, + None => return Poll::Pending, + }; loop { - match self.in_addr.poll_next_unpin(cx) { - Poll::Ready(mut item) => { - if let Some(item) = item.take() { - // Consume all events for up/down interface changes. - match item { - Ok(IfEvent::Up(inet)) => { - let ip = inet.addr(); - if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { - let socket_addr = - SocketAddr::new(ip, self.endpoint.socket_addr.port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("New listen address: {}", ma); - return Some(TransportEvent::NewAddress { - listener_id: self.listener_id, - listen_addr: ma, - }); - } - } - Ok(IfEvent::Down(inet)) => { - let ip = inet.addr(); - if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { - let socket_addr = - SocketAddr::new(ip, self.endpoint.socket_addr.port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("Expired listen address: {}", ma); - return Some(TransportEvent::AddressExpired { - listener_id: self.listener_id, - listen_addr: ma, - }); - } - } - Err(err) => { - tracing::debug! { - "Failure polling interfaces: {:?}.", - err - }; - return Some(TransportEvent::ListenerError { - listener_id: self.listener_id, - error: err.into(), - }); - } - } + match ready!(if_watcher.poll_if_event(cx)) { + Ok(IfEvent::Up(inet)) => { + let ip = inet.addr(); + if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { + let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr.port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("New listen address: {}", ma); + return Poll::Ready(TransportEvent::NewAddress { + listener_id: self.listener_id, + listen_addr: ma, + }); } } - Poll::Pending => return None, + Ok(IfEvent::Down(inet)) => { + let ip = inet.addr(); + if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { + let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr.port()); + let ma = socketaddr_to_multiaddr(&socket_addr); + tracing::debug!("Expired listen address: {}", ma); + return Poll::Ready(TransportEvent::AddressExpired { + listener_id: self.listener_id, + listen_addr: ma, + }); + } + } + Err(err) => { + return Poll::Ready(TransportEvent::ListenerError { + listener_id: self.listener_id, + error: err.into(), + }) + } } } } @@ -363,15 +361,16 @@ impl Stream for Listener { type Item = TransportEvent<::ListenerUpgrade, Error>; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { - if let Some(closed) = self.report_closed.as_mut() { - // Listener was closed. - // Report the transport event if there is one. On the next iteration, return - // `Poll::Ready(None)` to terminate the stream. - return Poll::Ready(closed.take()); - } - if let Some(event) = self.poll_if_addr(cx) { + if let Some(event) = self.pending_event.take() { return Poll::Ready(Some(event)); } + if self.is_closed { + return Poll::Ready(None); + } + match self.poll_if_addr(cx) { + Poll::Ready(event) => return Poll::Ready(Some(event)), + Poll::Pending => {} + } if !self.pending_dials.is_empty() { match self.endpoint.to_endpoint.poll_ready_unpin(cx) { Poll::Ready(Ok(_)) => { From b6924dbb757d17fbca77e93c5a0e055feb639ad9 Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Fri, 9 Sep 2022 23:49:15 +0200 Subject: [PATCH 085/218] transports/quic: fix clippy --- transports/quic/tests/smoke.rs | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 0291a16f9f0..6a41e4886c3 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -77,10 +77,9 @@ async fn smoke() -> Result<()> { let mut data = vec![0; 4096 * 10]; rng.fill_bytes(&mut data); + b.behaviour_mut().add_address(a.local_peer_id(), addr); b.behaviour_mut() - .add_address(&Swarm::local_peer_id(&a), addr); - b.behaviour_mut() - .send_request(&Swarm::local_peer_id(&a), Ping(data.clone())); + .send_request(a.local_peer_id(), Ping(data.clone())); let b_id = *b.local_peer_id(); @@ -405,9 +404,9 @@ fn concurrent_connections_and_streams() { for (listener_peer_id, listener_addr) in &listeners { dialer .behaviour_mut() - .add_address(&listener_peer_id, listener_addr.clone()); + .add_address(listener_peer_id, listener_addr.clone()); - dialer.dial(listener_peer_id.clone()).unwrap(); + dialer.dial(*listener_peer_id).unwrap(); } // Wait for responses to each request. @@ -543,8 +542,8 @@ async fn endpoint_reuse() -> Result<()> { } _ => {} }, - ev = swarm_b.select_next_some() => match ev{ - SwarmEvent::ConnectionEstablished { endpoint, ..} => { + ev = swarm_b.select_next_some() => { + if let SwarmEvent::ConnectionEstablished { endpoint, ..} = ev { match endpoint { ConnectedPoint::Dialer{..} => panic!("Unexpected outbound connection"), ConnectedPoint::Listener {send_back_addr, local_addr} => { @@ -555,7 +554,6 @@ async fn endpoint_reuse() -> Result<()> { } } } - _ => {} }, } } From 689460f03f4dc016d1a1011630fbce5fdee9371d Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sat, 10 Sep 2022 01:00:01 +0200 Subject: [PATCH 086/218] transports/quic: fix smoke test --- transports/quic/tests/smoke.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 6a41e4886c3..ec174014574 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -1,5 +1,6 @@ use anyhow::Result; use async_trait::async_trait; +use futures::channel::oneshot; use futures::future::{join, FutureExt}; use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; use futures::select; @@ -83,6 +84,8 @@ async fn smoke() -> Result<()> { let b_id = *b.local_peer_id(); + let (sync_tx, sync_rx) = oneshot::channel(); + let fut_a = async move { match a.next().await { Some(SwarmEvent::IncomingConnection { .. }) => {} @@ -133,6 +136,8 @@ async fn smoke() -> Result<()> { e => panic!("{:?}", e), } + sync_rx.await.unwrap(); + a.disconnect_peer_id(b_id).unwrap(); match a.next().await { @@ -188,6 +193,8 @@ async fn smoke() -> Result<()> { e => panic!("{:?}", e), } + sync_tx.send(()).unwrap(); + match b.next().await { Some(SwarmEvent::ConnectionClosed { cause: Some(ConnectionError::IO(_)), From 41d39fbf3f5fde7035aaf82413fc7897bf93970d Mon Sep 17 00:00:00 2001 From: elenaf9 Date: Sat, 10 Sep 2022 18:34:05 +0200 Subject: [PATCH 087/218] transports/quic: add `Endpoint::try_send` --- transports/quic/src/connection.rs | 27 +++++++------ transports/quic/src/endpoint.rs | 28 +++++++++++-- transports/quic/src/transport.rs | 66 +++++++++++++++---------------- 3 files changed, 71 insertions(+), 50 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index b0290fe4ac5..29ae0596a42 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -66,6 +66,9 @@ pub enum Error { /// Endpoint has force-killed this connection because it was too busy. #[error("Endpoint has force-killed our connection")] ClosedChannel, + /// The background task driving the endpoint has crashed. + #[error("Background task crashed.")] + TaskCrashed, /// Error in the inner state machine. #[error("{0}")] Quinn(#[from] quinn_proto::ConnectionError), @@ -109,7 +112,7 @@ impl Connection { /// Works for server connections only. pub fn local_addr(&self) -> SocketAddr { debug_assert_eq!(self.connection.side(), quinn_proto::Side::Server); - let endpoint_addr = self.endpoint.socket_addr; + let endpoint_addr = self.endpoint.socket_addr(); self.connection .local_ip() .map(|ip| SocketAddr::new(ip, endpoint_addr.port())) @@ -117,7 +120,7 @@ impl Connection { // In a normal case scenario this should not happen, because // we get want to get a local addr for a server connection only. tracing::error!("trying to get quinn::local_ip for a client"); - endpoint_addr + *endpoint_addr }) } @@ -214,17 +217,17 @@ impl Connection { // However we don't deliver substream-related events to the user as long as // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. - if self.pending_to_endpoint.is_some() { - match self.endpoint.to_endpoint.poll_ready_unpin(cx) { - Poll::Ready(Ok(())) => { - let to_endpoint = self.pending_to_endpoint.take().expect("is some"); - self.endpoint - .to_endpoint - .start_send(to_endpoint) - .expect("Channel is ready."); + if let Some(to_endpoint) = self.pending_to_endpoint.take() { + match self.endpoint.try_send(to_endpoint, cx) { + Ok(Ok(())) => {} + Ok(Err(to_endpoint)) => { + self.pending_to_endpoint = Some(to_endpoint); + return Poll::Pending; + } + Err(_) => { + tracing::error!("Background task crashed."); + return Poll::Ready(ConnectionEvent::ConnectionLost(Error::TaskCrashed)); } - Poll::Ready(Err(_)) => panic!("Background task crashed"), - Poll::Pending => return Poll::Pending, } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index a65da3bae60..ed1fb3052d8 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -31,7 +31,10 @@ use crate::{connection::Connection, tls, transport}; use futures::{ - channel::{mpsc, oneshot}, + channel::{ + mpsc::{self, SendError}, + oneshot, + }, prelude::*, }; use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; @@ -40,7 +43,7 @@ use std::{ fmt, net::{Ipv4Addr, Ipv6Addr, SocketAddr, UdpSocket}, sync::Arc, - task::{Poll, Waker}, + task::{Context, Poll, Waker}, time::{Duration, Instant}, }; @@ -87,9 +90,9 @@ impl Config { #[derive(Clone)] pub struct Endpoint { /// Channel to the background of the endpoint. - pub to_endpoint: mpsc::Sender, + to_endpoint: mpsc::Sender, - pub socket_addr: SocketAddr, + socket_addr: SocketAddr, } impl Endpoint { @@ -142,6 +145,23 @@ impl Endpoint { Ok(endpoint) } + + pub fn socket_addr(&self) -> &SocketAddr { + &self.socket_addr + } + + pub fn try_send( + &mut self, + to_endpoint: ToEndpoint, + cx: &mut Context<'_>, + ) -> Result, SendError> { + match self.to_endpoint.poll_ready_unpin(cx) { + Poll::Ready(Ok(())) => {} + Poll::Ready(Err(err)) => return Err(err), + Poll::Pending => return Ok(Err(to_endpoint)), + }; + self.to_endpoint.start_send(to_endpoint).map(Ok) + } } /// Message sent to the endpoint background task. diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 0aa3295ac0a..a64dfa053dc 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -27,6 +27,7 @@ use crate::endpoint::ToEndpoint; use crate::Config; use crate::{endpoint::Endpoint, muxer::QuicMuxer, upgrade::Upgrade}; +use futures::channel::mpsc::SendError; use futures::channel::oneshot; use futures::ready; use futures::stream::StreamExt; @@ -140,7 +141,7 @@ impl Transport for QuicTransport { .listeners .iter_mut() .filter(|l| { - let listen_addr = l.endpoint.socket_addr; + let listen_addr = l.endpoint.socket_addr(); listen_addr.is_ipv4() == socket_addr.is_ipv4() && listen_addr.ip().is_loopback() == socket_addr.ip().is_loopback() }) @@ -177,7 +178,7 @@ impl Transport for QuicTransport { Ok(async move { let connection = rx .await - .expect("background task has crashed") + .map_err(|_| Error::TaskCrashed)? .map_err(Error::Reach)?; let final_connec = Upgrade::from_connection(connection).await?; Ok(final_connec) @@ -201,10 +202,18 @@ impl Transport for QuicTransport { cx: &mut Context<'_>, ) -> Poll> { if let Some(dialer) = self.ipv4_dialer.as_mut() { - dialer.drive_dials(cx) + if dialer.drive_dials(cx).is_err() { + // Background task of dialer crashed. + // Drop dialer and all pending dials so that the connection receiver is notified. + self.ipv4_dialer = None; + } } if let Some(dialer) = self.ipv6_dialer.as_mut() { - dialer.drive_dials(cx) + if dialer.drive_dials(cx).is_err() { + // Background task of dialer crashed. + // Drop dialer and all pending dials so that the connection receiver is notified. + self.ipv4_dialer = None; + } } match self.listeners.poll_next_unpin(cx) { Poll::Ready(Some(ev)) => Poll::Ready(ev), @@ -228,20 +237,18 @@ impl Dialer { }) } - fn drive_dials(&mut self, cx: &mut Context<'_>) { - if !self.pending_dials.is_empty() { - match self.endpoint.to_endpoint.poll_ready_unpin(cx) { - Poll::Ready(Ok(())) => { - let to_endpoint = self.pending_dials.pop_front().expect("!is_empty"); - self.endpoint - .to_endpoint - .start_send(to_endpoint) - .expect("Channel is ready."); + fn drive_dials(&mut self, cx: &mut Context<'_>) -> Result<(), SendError> { + if let Some(to_endpoint) = self.pending_dials.pop_front() { + match self.endpoint.try_send(to_endpoint, cx) { + Ok(Ok(())) => {} + Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), + Err(err) => { + tracing::error!("Background task of dialing endpoint crashed."); + return Err(err); } - Poll::Ready(Err(_)) => panic!("Background task crashed."), - Poll::Pending => {} } } + Ok(()) } } @@ -282,7 +289,7 @@ impl Listener { pending_event = None; } else { if_watcher = None; - let ma = socketaddr_to_multiaddr(&endpoint.socket_addr); + let ma = socketaddr_to_multiaddr(endpoint.socket_addr()); pending_event = Some(TransportEvent::NewAddress { listener_id, listen_addr: ma, @@ -324,8 +331,8 @@ impl Listener { match ready!(if_watcher.poll_if_event(cx)) { Ok(IfEvent::Up(inet)) => { let ip = inet.addr(); - if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { - let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr.port()); + if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr().port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("New listen address: {}", ma); return Poll::Ready(TransportEvent::NewAddress { @@ -336,8 +343,8 @@ impl Listener { } Ok(IfEvent::Down(inet)) => { let ip = inet.addr(); - if self.endpoint.socket_addr.is_ipv4() == ip.is_ipv4() { - let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr.port()); + if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { + let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr().port()); let ma = socketaddr_to_multiaddr(&socket_addr); tracing::debug!("Expired listen address: {}", ma); return Poll::Ready(TransportEvent::AddressExpired { @@ -371,23 +378,14 @@ impl Stream for Listener { Poll::Ready(event) => return Poll::Ready(Some(event)), Poll::Pending => {} } - if !self.pending_dials.is_empty() { - match self.endpoint.to_endpoint.poll_ready_unpin(cx) { - Poll::Ready(Ok(_)) => { - let to_endpoint = self - .pending_dials - .pop_front() - .expect("Pending dials is not empty."); - self.endpoint - .to_endpoint - .start_send(to_endpoint) - .expect("Channel is ready"); - } - Poll::Ready(Err(_)) => { + if let Some(to_endpoint) = self.pending_dials.pop_front() { + match self.endpoint.try_send(to_endpoint, cx) { + Ok(Ok(())) => {} + Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), + Err(_) => { self.close(Err(Error::TaskCrashed)); continue; } - Poll::Pending => {} } } match self.new_connections_rx.poll_next_unpin(cx) { From 4e027b1739d6f3dc6754a5dc15720dfb3b3fcf88 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 19 Sep 2022 03:15:58 +0200 Subject: [PATCH 088/218] transports/quic: handle substream being dropped --- transports/quic/src/muxer.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index f3daaf0c657..4014a72a860 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -24,6 +24,7 @@ use crate::error::Error; use futures::{AsyncRead, AsyncWrite}; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; +use quinn_proto::FinishError; use std::{ collections::HashMap, io::{self, Write}, @@ -123,6 +124,9 @@ impl StreamMuxer for QuicMuxer { ConnectionEvent::StreamFinished(substream) | ConnectionEvent::StreamStopped(substream) => { if let Some(substream) = inner.substreams.get_mut(&substream) { + if let Some(waker) = substream.read_waker.take() { + waker.wake(); + } if let Some(waker) = substream.write_waker.take() { waker.wake(); } @@ -344,5 +348,19 @@ impl Drop for Substream { fn drop(&mut self) { let mut muxer = self.muxer.lock(); muxer.substreams.remove(&self.id); + let _ = muxer + .connection + .connection + .recv_stream(self.id) + .stop(0u32.into()); + let mut send_stream = muxer.connection.connection.send_stream(self.id); + match send_stream.finish() { + Ok(()) => {} + // Already finished or reset, which is fine. + Err(FinishError::UnknownStream) => {} + Err(FinishError::Stopped(reason)) => { + let _ = send_stream.reset(reason); + } + } } } From bdba780ee5eaf20d6035a76343ac941ecfd4f0b3 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 19 Sep 2022 03:44:04 +0200 Subject: [PATCH 089/218] transports/quic: return err on read after reset --- transports/quic/src/muxer.rs | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 4014a72a860..4cfdc48afa7 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -260,14 +260,8 @@ impl AsyncRead for Substream { bytes += chunk.bytes.len(); } Ok(None) => break, - Err(ReadError::Reset(error_code)) => { - tracing::error!( - "substream {} was reset with error code {}", - self.id, - error_code - ); - bytes = 0; - break; + Err(err @ ReadError::Reset(_)) => { + return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } Err(ReadError::Blocked) => { pending = true; From 40cb4f33ffbb6d22ada1c00c620aa780e25130ee Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 19 Sep 2022 04:11:37 +0200 Subject: [PATCH 090/218] transports/quic: apply comments from code review --- transports/quic/src/connection.rs | 3 +-- transports/quic/src/transport.rs | 9 +++------ 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 29ae0596a42..c1060d69870 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -67,7 +67,7 @@ pub enum Error { #[error("Endpoint has force-killed our connection")] ClosedChannel, /// The background task driving the endpoint has crashed. - #[error("Background task crashed.")] + #[error("Background task crashed")] TaskCrashed, /// Error in the inner state machine. #[error("{0}")] @@ -225,7 +225,6 @@ impl Connection { return Poll::Pending; } Err(_) => { - tracing::error!("Background task crashed."); return Poll::Ready(ConnectionEvent::ConnectionLost(Error::TaskCrashed)); } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index a64dfa053dc..24f9d21d6c0 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -27,11 +27,10 @@ use crate::endpoint::ToEndpoint; use crate::Config; use crate::{endpoint::Endpoint, muxer::QuicMuxer, upgrade::Upgrade}; -use futures::channel::mpsc::SendError; -use futures::channel::oneshot; +use futures::channel::{mpsc, oneshot}; use futures::ready; use futures::stream::StreamExt; -use futures::{channel::mpsc, prelude::*, stream::SelectAll}; +use futures::{prelude::*, stream::SelectAll}; use if_watch::{IfEvent, IfWatcher}; @@ -134,7 +133,6 @@ impl Transport for QuicTransport { let socket_addr = multiaddr_to_socketaddr(&addr) .ok_or_else(|| TransportError::MultiaddrNotSupported(addr.clone()))?; if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { - tracing::error!("multiaddr not supported"); return Err(TransportError::MultiaddrNotSupported(addr)); } let mut listeners = self @@ -237,13 +235,12 @@ impl Dialer { }) } - fn drive_dials(&mut self, cx: &mut Context<'_>) -> Result<(), SendError> { + fn drive_dials(&mut self, cx: &mut Context<'_>) -> Result<(), mpsc::SendError> { if let Some(to_endpoint) = self.pending_dials.pop_front() { match self.endpoint.try_send(to_endpoint, cx) { Ok(Ok(())) => {} Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), Err(err) => { - tracing::error!("Background task of dialing endpoint crashed."); return Err(err); } } From f8d143030b8399bf47d3b84343a4149870249f30 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 14:10:17 +0200 Subject: [PATCH 091/218] transports/quic: better naming, fix docs --- transports/quic/src/connection.rs | 49 ++++++++++++++++--------------- transports/quic/src/muxer.rs | 13 ++++---- transports/quic/src/transport.rs | 5 +++- 3 files changed, 36 insertions(+), 31 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index c1060d69870..53ecf3b41cc 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -45,7 +45,7 @@ use std::{ pub struct Connection { /// Endpoint this connection belongs to. endpoint: Endpoint, - /// Future whose job is to send a message to the endpoint. Only one at a time. + /// Pending message to be sent to the background task that is driving the endpoint. pending_to_endpoint: Option, /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. /// Passed at initialization. @@ -96,7 +96,7 @@ impl Connection { connection_id: quinn_proto::ConnectionHandle, from_endpoint: mpsc::Receiver, ) -> Self { - assert!(!connection.is_closed()); + debug_assert!(!connection.is_closed()); Connection { endpoint, pending_to_endpoint: None, @@ -107,21 +107,24 @@ impl Connection { } } - /// The local address which was used when the peer established the connection. + /// The local address which was used when the remote established the connection to us. /// - /// Works for server connections only. - pub fn local_addr(&self) -> SocketAddr { - debug_assert_eq!(self.connection.side(), quinn_proto::Side::Server); + /// `None` for client connections. + pub fn local_addr(&self) -> Option { + if self.connection.side().is_client() { + return None; + } let endpoint_addr = self.endpoint.socket_addr(); - self.connection - .local_ip() - .map(|ip| SocketAddr::new(ip, endpoint_addr.port())) - .unwrap_or_else(|| { - // In a normal case scenario this should not happen, because - // we get want to get a local addr for a server connection only. - tracing::error!("trying to get quinn::local_ip for a client"); - *endpoint_addr - }) + + // Local address may differ from the socket address if the socket is + // bound to a wildcard address. + let addr = match self.connection.local_ip() { + Some(ip) => SocketAddr::new(ip, endpoint_addr.port()), + // TODO: `quinn_proto::Connection::local_ip` is only supported for linux, + // so for other platforms we currently still return the endpoint address. + None => *endpoint_addr, + }; + Some(addr) } /// Returns the address of the node we're connected to. @@ -160,7 +163,7 @@ impl Connection { /// /// If `None` is returned, then a [`ConnectionEvent::StreamAvailable`] event will later be /// produced when a substream is available. - pub fn pop_incoming_substream(&mut self) -> Option { + pub fn accept_substream(&mut self) -> Option { self.connection.streams().accept(quinn_proto::Dir::Bi) } @@ -171,7 +174,7 @@ impl Connection { /// /// If `None` is returned, then a [`ConnectionEvent::StreamOpened`] event will later be /// produced when a substream is available. - pub fn pop_outgoing_substream(&mut self) -> Option { + pub fn open_substream(&mut self) -> Option { self.connection.streams().open(quinn_proto::Dir::Bi) } @@ -183,16 +186,14 @@ impl Connection { /// On success, a [`quinn_proto::StreamEvent::Finished`] event will later be produced when the /// substream has been effectively closed. A [`ConnectionEvent::StreamStopped`] event can also /// be emitted. - pub fn shutdown_substream( + pub fn finish_substream( &mut self, id: quinn_proto::StreamId, ) -> Result<(), quinn_proto::FinishError> { - // 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. self.connection.send_stream(id).finish() } - /// Polls the connection for an event that happend on it. + /// Polls the connection for an event that happened on it. pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { let mut closed = None; loop { @@ -303,10 +304,10 @@ pub enum ConnectionEvent { /// Connection has been closed and can no longer be used. ConnectionLost(Error), - /// Generated after [`Connection::pop_incoming_substream`] has been called and has returned + /// Generated after [`Connection::accept_substream`] has been called and has returned /// `None`. After this event has been generated, this method is guaranteed to return `Some`. StreamAvailable, - /// Generated after [`Connection::pop_outgoing_substream`] has been called and has returned + /// Generated after [`Connection::open_substream`] has been called and has returned /// `None`. After this event has been generated, this method is guaranteed to return `Some`. StreamOpened, @@ -315,7 +316,7 @@ pub enum ConnectionEvent { /// Generated after `write_substream` has returned a `Blocked` error. StreamWritable(quinn_proto::StreamId), - /// Generated after [`Connection::shutdown_substream`] has been called. + /// Generated after [`Connection::finish_substream`] has been called. StreamFinished(quinn_proto::StreamId), /// A substream has been stopped. This concept is similar to the concept of a substream being /// "reset", as in a TCP socket being reset for example. diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 4cfdc48afa7..5ccf8753c2b 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -82,6 +82,7 @@ impl QuicMuxer { } } } + impl StreamMuxer for QuicMuxer { type Substream = Substream; type Error = Error; @@ -94,7 +95,8 @@ impl StreamMuxer for QuicMuxer { while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { ConnectionEvent::Connected | ConnectionEvent::HandshakeDataReady => { - tracing::warn!( + debug_assert!( + false, "Unexpected event {:?} on established QUIC connection", event ); @@ -143,7 +145,6 @@ impl StreamMuxer for QuicMuxer { } } inner.poll_connection_waker = Some(cx.waker().clone()); - // TODO: poll address change Poll::Pending } @@ -152,7 +153,7 @@ impl StreamMuxer for QuicMuxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); - let substream_id = match inner.connection.pop_incoming_substream() { + let substream_id = match inner.connection.accept_substream() { Some(id) => { inner.poll_outbound_waker = None; id @@ -172,7 +173,7 @@ impl StreamMuxer for QuicMuxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); - let substream_id = match inner.connection.pop_outgoing_substream() { + let substream_id = match inner.connection.open_substream() { Some(id) => { inner.poll_outbound_waker = None; id @@ -195,7 +196,7 @@ impl StreamMuxer for QuicMuxer { if inner.connection.connection.streams().send_streams() != 0 { for substream in inner.substreams.keys().cloned().collect::>() { - if let Err(e) = inner.connection.shutdown_substream(substream) { + if let Err(e) = inner.connection.finish_substream(substream) { tracing::warn!("substream finish error on muxer close: {}", e); } } @@ -321,7 +322,7 @@ impl AsyncWrite for Substream { fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut muxer = self.muxer.lock(); - match muxer.connection.shutdown_substream(self.id) { + match muxer.connection.finish_substream(self.id) { Ok(()) => { let substream_state = muxer .substreams diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 24f9d21d6c0..1f36a804027 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -387,7 +387,10 @@ impl Stream for Listener { } match self.new_connections_rx.poll_next_unpin(cx) { Poll::Ready(Some(connection)) => { - let local_addr = socketaddr_to_multiaddr(&connection.local_addr()); + let local_addr = connection + .local_addr() + .expect("exists for server connections."); + let local_addr = socketaddr_to_multiaddr(&local_addr); let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); let event = TransportEvent::Incoming { upgrade: Upgrade::from_connection(connection), From 4c3229b812f26ac22ccc848d6e4a3fbca6a5b284 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 14:17:45 +0200 Subject: [PATCH 092/218] transports/quic: add doc for `Endpoint:try_send` --- transports/quic/src/endpoint.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index ed1fb3052d8..ede1000a6a9 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -150,6 +150,13 @@ impl Endpoint { &self.socket_addr } + /// Try to send a message to the background task without blocking. + /// + /// This first polls the channel for capacity. + /// If the channel is full, the message is returned in `Ok(Err(_))` + /// and the context's waker is registered for wake-up. + /// + /// If the background task crashed `Err` is returned. pub fn try_send( &mut self, to_endpoint: ToEndpoint, From e393fe5fbcdbe04f53f0233c8b5efdbecc0dabf4 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 14:54:32 +0200 Subject: [PATCH 093/218] transports/quic: add `ip_to_listenaddr` --- transports/quic/src/transport.rs | 33 ++++++++++++++++++++------------ 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 1f36a804027..0029b31a157 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -40,6 +40,7 @@ use libp2p_core::{ PeerId, Transport, }; use std::collections::VecDeque; +use std::net::IpAddr; use std::task::Waker; use std::{ net::SocketAddr, @@ -327,26 +328,20 @@ impl Listener { loop { match ready!(if_watcher.poll_if_event(cx)) { Ok(IfEvent::Up(inet)) => { - let ip = inet.addr(); - if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { - let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr().port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("New listen address: {}", ma); + if let Some(addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { + tracing::debug!("New listen address: {}", addr); return Poll::Ready(TransportEvent::NewAddress { listener_id: self.listener_id, - listen_addr: ma, + listen_addr: addr, }); } } Ok(IfEvent::Down(inet)) => { - let ip = inet.addr(); - if self.endpoint.socket_addr().is_ipv4() == ip.is_ipv4() { - let socket_addr = SocketAddr::new(ip, self.endpoint.socket_addr().port()); - let ma = socketaddr_to_multiaddr(&socket_addr); - tracing::debug!("Expired listen address: {}", ma); + if let Some(addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { + tracing::debug!("Expired listen address: {}", addr); return Poll::Ready(TransportEvent::AddressExpired { listener_id: self.listener_id, - listen_addr: ma, + listen_addr: addr, }); } } @@ -412,6 +407,20 @@ impl Stream for Listener { } } +/// Turn an [`IpAddr`] into a listen-address for the endpoint. +/// +/// Returns `None` if the address is not the same socket family as the +/// address that the endpoint is bound to. +pub fn ip_to_listenaddr(endpoint: &Endpoint, ip: IpAddr) -> Option { + // True if either both addresses are Ipv4 or both Ipv6. + let is_same_ip_family = endpoint.socket_addr().is_ipv4() == ip.is_ipv4(); + if !is_same_ip_family { + return None; + } + let socket_addr = SocketAddr::new(ip, endpoint.socket_addr().port()); + Some(socketaddr_to_multiaddr(&socket_addr)) +} + /// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns None if the format /// of the multiaddr is wrong. pub fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { From d28db187c8157ecb7f121ab7897866d96cbbdddf Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 15:23:53 +0200 Subject: [PATCH 094/218] transports/quic: disable connection migration --- transports/quic/src/endpoint.rs | 4 ++++ transports/quic/src/muxer.rs | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index ede1000a6a9..3a4c0daa17f 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -72,6 +72,10 @@ impl Config { let mut server_config = QuinnServerConfig::with_crypto(Arc::new(server_tls_config)); server_config.transport = Arc::clone(&transport); + // Disables connection migration. + // Long-term this should be enabled, however we then need to handle address change + // on connections in the `QuicMuxer`. + server_config.migration(false); let mut client_config = QuinnClientConfig::new(Arc::new(client_tls_config)); client_config.transport = transport; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 5ccf8753c2b..f2b6d3366bb 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -92,6 +92,8 @@ impl StreamMuxer for QuicMuxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); + // Poll the inner [`quinn_proto::Connection`] for events and wake + // the wakers of related poll-based methods. while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { ConnectionEvent::Connected | ConnectionEvent::HandshakeDataReady => { @@ -145,6 +147,10 @@ impl StreamMuxer for QuicMuxer { } } inner.poll_connection_waker = Some(cx.waker().clone()); + + // TODO: If connection migration is enabled (currently disabled) address + // change on the connection needs to be handled. + Poll::Pending } From 42db0ede6601b6cc40e18fc20d54489bfc2d18a1 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 15:36:54 +0200 Subject: [PATCH 095/218] transports/quic: minor fix --- transports/quic/src/transport.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 0029b31a157..e915a29d532 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -328,20 +328,20 @@ impl Listener { loop { match ready!(if_watcher.poll_if_event(cx)) { Ok(IfEvent::Up(inet)) => { - if let Some(addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { - tracing::debug!("New listen address: {}", addr); + if let Some(listen_addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { + tracing::debug!("New listen address: {}", listen_addr); return Poll::Ready(TransportEvent::NewAddress { listener_id: self.listener_id, - listen_addr: addr, + listen_addr, }); } } Ok(IfEvent::Down(inet)) => { - if let Some(addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { - tracing::debug!("Expired listen address: {}", addr); + if let Some(listen_addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { + tracing::debug!("Expired listen address: {}", listen_addr); return Poll::Ready(TransportEvent::AddressExpired { listener_id: self.listener_id, - listen_addr: addr, + listen_addr, }); } } From d46b72e9af0a85038d5f79cc6b877eed61fe8109 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 16:53:42 +0200 Subject: [PATCH 096/218] transports/quic: minor fixes --- transports/quic/src/connection.rs | 2 -- transports/quic/src/endpoint.rs | 23 +++++++++-------------- transports/quic/tests/smoke.rs | 8 -------- 3 files changed, 9 insertions(+), 24 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 53ecf3b41cc..8b3f371ffe0 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -89,7 +89,6 @@ impl Connection { /// /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of /// its methods has ever been called. Failure to comply might lead to logic errors and panics. - // TODO: maybe abstract `to_endpoint` more and make it generic? dunno pub fn from_quinn_connection( endpoint: Endpoint, connection: quinn_proto::Connection, @@ -128,7 +127,6 @@ impl Connection { } /// Returns the address of the node we're connected to. - // TODO: can change /!\ pub fn remote_addr(&self) -> SocketAddr { self.connection.remote_address() } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 3a4c0daa17f..53e36299ecd 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -88,14 +88,12 @@ impl Config { } /// Object containing all the QUIC resources shared between all connections. -// TODO: expand docs -// TODO: Debug trait -// TODO: remove useless fields #[derive(Clone)] pub struct Endpoint { /// Channel to the background of the endpoint. to_endpoint: mpsc::Sender, - + /// Address that the socket is bound to. + /// Note: this may be a wildcard ip address. socket_addr: SocketAddr, } @@ -201,7 +199,7 @@ pub enum ToEndpoint { }, } -/// Task that runs in the background for as long as the endpont is alive. Responsible for +/// Task that runs in the background for as long as the endpoint is alive. Responsible for /// processing messages and the UDP socket. /// /// The `receiver` parameter must be the receiving side of the `Endpoint::to_endpoint` sender. @@ -223,13 +221,11 @@ pub enum ToEndpoint { /// in play: /// /// - One channel, represented by `Endpoint::to_endpoint` and `receiver`, that communicates -/// messages from [`Endpoint`] to the background task and from the [`Connection`] to the -/// background task. +/// messages from [`Endpoint`] to the background task. /// - One channel per each existing connection that communicates messages from the background /// task to that [`Connection`]. /// - One channel for the background task to send newly-opened connections to. The receiving -/// side is normally processed by a "listener" as defined by the [`libp2p_core::Transport`] -/// trait. +/// side is processed by the [`crate::transport::Listener`]. /// /// In order to avoid an unbounded buffering of events, we prioritize sending data on the UDP /// socket over everything else. If the network interface is too busy to process our packets, @@ -342,7 +338,6 @@ async fn background_task( // The endpoint might request packets to be sent out. This is handled in priority to avoid // buffering up packets. if let Some(packet) = proto_endpoint.poll_transmit() { - debug_assert!(next_packet_out.is_none()); next_packet_out = Some((packet.destination, packet.contents)); continue; } @@ -352,9 +347,7 @@ async fn background_task( // Received a message from a different part of the code requesting us to // do something. match message { - // Shut down if the endpoint has shut down. - None => return, - + None => unreachable!("Sender side is never dropped or closed."), Some(ToEndpoint::Dial { addr, result }) => { // This `"l"` seems necessary because an empty string is an invalid domain // name. While we don't use domain names, the underlying rustls library @@ -501,6 +494,8 @@ async fn background_task( impl fmt::Debug for Endpoint { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_tuple("Endpoint").finish() + f.debug_struct("Endpoint") + .field("socket_addr", &self.socket_addr) + .finish() } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index ec174014574..d6507e73c45 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -31,14 +31,6 @@ async fn create_swarm(keylog: bool) -> Result>> let config = QuicConfig::new(&keypair).unwrap(); let transport = QuicTransport::new(config); - // TODO: - // transport - // .transport - // .max_idle_timeout(Some(quinn_proto::VarInt::from_u32(1_000u32).into())); - // if keylog { - // transport.enable_keylogger(); - // } - let transport = Transport::map(transport, |(peer, muxer), _| { (peer, StreamMuxerBox::new(muxer)) }) From ec3c74a812fadc7b0fc95e582a5f8d1d13571b27 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 20:37:17 +0200 Subject: [PATCH 097/218] transports/quic: rework forwarding of new connections The existing implementation was based on an old API of the quinn_proto Endpoint which by now has changed. In particular we can not explicitly `accept` new connections on the endpoint anymore. Instead if there is a new connections and our channel for new connections is full because the endpoint is too busy, we now simply drop the connection to backpressure the remote. --- swarm-derive/src/lib.rs | 2 +- transports/quic/src/endpoint.rs | 99 ++++++++++++--------------------- 2 files changed, 38 insertions(+), 63 deletions(-) diff --git a/swarm-derive/src/lib.rs b/swarm-derive/src/lib.rs index 6899ba7d79d..5f901ed6ff8 100644 --- a/swarm-derive/src/lib.rs +++ b/swarm-derive/src/lib.rs @@ -125,7 +125,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { ) .unwrap(); let ty = &field.ty; - quote! {#variant(<#ty as NetworkBehaviour>::OutEvent)} + quote! {#variant(<#ty as ::libp2p::swarm::NetworkBehaviour>::OutEvent)} }) .collect::>(); let visibility = &ast.vis; diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 53e36299ecd..d81c5045b4d 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -39,11 +39,11 @@ use futures::{ }; use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; use std::{ - collections::{HashMap, VecDeque}, + collections::HashMap, fmt, net::{Ipv4Addr, Ipv6Addr, SocketAddr, UdpSocket}, sync::Arc, - task::{Context, Poll, Waker}, + task::{Context, Poll}, time::{Duration, Instant}, }; @@ -298,19 +298,10 @@ async fn background_task( // Buffer where we write packets received from the UDP socket. let mut socket_recv_buffer = vec![0; 65536]; - // The quinn_proto endpoint can give us new connections for as long as its accept buffer - // isn't full. This buffer is used to push these new connections while we are waiting to - // send them on the `new_connections` channel. We only call `endpoint.accept()` when we remove - // an element from this list, which guarantees that it doesn't grow unbounded. - // TODO: with_capacity? - let mut queued_new_connections = VecDeque::new(); - // Next packet waiting to be transmitted on the UDP socket, if any. - // Note that this variable isn't strictly necessary, but it reduces code duplication in the - // code below. let mut next_packet_out: Option<(SocketAddr, Vec)> = None; - let mut new_connection_waker: Option = None; + let mut is_orphaned = false; // Main loop of the task. loop { @@ -343,7 +334,7 @@ async fn background_task( } futures::select! { - message = receiver.next().fuse() => { + message = receiver.next() => { // Received a message from a different part of the code requesting us to // do something. match message { @@ -379,6 +370,13 @@ async fn background_task( let is_drained_event = event.is_drained(); if is_drained_event { alive_connections.remove(&connection_id); + if is_orphaned && alive_connections.is_empty() { + tracing::info!( + "Listener closed and no active connections remain. Shutting down the background task." + ); + return; + } + } let event_back = proto_endpoint.handle_event(connection_id, event); @@ -401,46 +399,6 @@ async fn background_task( } } } - - // The future we create here wakes up if two conditions are fulfilled: - // - // - The `new_connections` channel is ready to accept a new element. - // - `queued_new_connections` is not empty. - // - // When this happens, we pop an element from `queued_new_connections`, put it on the - // channel, and call `endpoint.accept()`, thereby allowing the QUIC state machine to - // feed a new incoming connection to us. - readiness = { - let active = !queued_new_connections.is_empty(); - let new_connections = &mut new_connections; - let new_connection_waker = &mut new_connection_waker; - future::poll_fn(move |cx| { - match new_connections.as_mut() { - Some(ref mut c) if active => { - c.poll_ready(cx) - } - _ => { - let _ = new_connection_waker.insert(cx.waker().clone()); - Poll::Pending - } - } - }) - .fuse() - } => { - if readiness.is_err() { - // new_connections channel has been dropped, meaning that the endpoint has - // been destroyed. - return; - } - - let elem = queued_new_connections.pop_front() - .expect("if queue is empty, the future above is always Pending; qed"); - let new_connections = new_connections.as_mut().expect("in case of None, the future above is always Pending; qed"); - new_connections.start_send(elem) - .expect("future is waken up only if poll_ready returned Ready; qed"); - //endpoint.accept(); - } - result = udp_socket.recv_from(&mut socket_recv_buffer).fuse() => { let (packet_len, packet_src) = match result { Ok(v) => v, @@ -473,17 +431,34 @@ async fn background_task( // A new connection has been received. `connec_id` is a newly-allocated // identifier. debug_assert_eq!(connec.side(), quinn_proto::Side::Server); + let connection_tx = match new_connections.as_mut() { + Some(tx) => tx, + None => { + tracing::warn!( + "Endpoint reported a new connection even though server capabilities are disabled." + ); + continue + } + }; + let (tx, rx) = mpsc::channel(16); - alive_connections.insert(connec_id, tx); let connection = Connection::from_quinn_connection(endpoint.clone(), connec, connec_id, rx); - - // As explained in the documentation, we put this new connection in an - // intermediary buffer. At the next loop iteration we will try to move it - // to the `new_connections` channel. We call `endpoint.accept()` only once - // the element has successfully been sent on `new_connections`. - queued_new_connections.push_back(connection); - if let Some(waker) = new_connection_waker.take() { - waker.wake(); + match connection_tx.try_send(connection) { + Ok(()) => { + alive_connections.insert(connec_id, tx); + } + Err(e) if e.is_disconnected() => { + // Listener was closed. + proto_endpoint.reject_new_connections(); + new_connections = None; + is_orphaned = true; + if alive_connections.is_empty() { + return; + } + } + _ => tracing::warn!( + "Dropping new incoming connection because the channel to the listener is full." + ) } }, } From b7103aa146621a6aa1504dc1a0b51d634883f9ae Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 20 Sep 2022 22:57:40 +0200 Subject: [PATCH 098/218] transports/quic: fix broken intra-doc link --- transports/quic/src/endpoint.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index d81c5045b4d..3c34dd3b391 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -47,7 +47,7 @@ use std::{ time::{Duration, Instant}, }; -/// Represents the configuration for the [`Endpoint`]. +/// Represents the configuration for the QUIC endpoint. #[derive(Debug, Clone)] pub struct Config { /// The client configuration to pass to `quinn_proto`. @@ -225,7 +225,7 @@ pub enum ToEndpoint { /// - One channel per each existing connection that communicates messages from the background /// task to that [`Connection`]. /// - One channel for the background task to send newly-opened connections to. The receiving -/// side is processed by the [`crate::transport::Listener`]. +/// side is processed by the [`QuicTransport`][crate::QuicTransport]. /// /// In order to avoid an unbounded buffering of events, we prioritize sending data on the UDP /// socket over everything else. If the network interface is too busy to process our packets, From 9df25f119496a74b706f7a227259a17d51f48e14 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 21 Sep 2022 13:42:35 +0200 Subject: [PATCH 099/218] transports/quic: apply suggestions from review Address leftovers from review in elenaf9#6. --- swarm-derive/src/lib.rs | 2 +- transports/quic/src/endpoint.rs | 16 ++++------------ transports/quic/src/muxer.rs | 22 +++++++++++++--------- transports/quic/src/transport.rs | 2 +- 4 files changed, 19 insertions(+), 23 deletions(-) diff --git a/swarm-derive/src/lib.rs b/swarm-derive/src/lib.rs index 5f901ed6ff8..6899ba7d79d 100644 --- a/swarm-derive/src/lib.rs +++ b/swarm-derive/src/lib.rs @@ -125,7 +125,7 @@ fn build_struct(ast: &DeriveInput, data_struct: &DataStruct) -> TokenStream { ) .unwrap(); let ty = &field.ty; - quote! {#variant(<#ty as ::libp2p::swarm::NetworkBehaviour>::OutEvent)} + quote! {#variant(<#ty as NetworkBehaviour>::OutEvent)} }) .collect::>(); let visibility = &ast.vis; diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 3c34dd3b391..0f88f9860bd 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -40,7 +40,6 @@ use futures::{ use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; use std::{ collections::HashMap, - fmt, net::{Ipv4Addr, Ipv6Addr, SocketAddr, UdpSocket}, sync::Arc, task::{Context, Poll}, @@ -88,7 +87,7 @@ impl Config { } /// Object containing all the QUIC resources shared between all connections. -#[derive(Clone)] +#[derive(Debug, Clone)] pub struct Endpoint { /// Channel to the background of the endpoint. to_endpoint: mpsc::Sender, @@ -456,8 +455,9 @@ async fn background_task( return; } } - _ => tracing::warn!( - "Dropping new incoming connection because the channel to the listener is full." + Err(_) => tracing::warn!( + "Dropping new incoming connection {:?} because the channel to the listener is full", + connec_id ) } }, @@ -466,11 +466,3 @@ async fn background_task( } } } - -impl fmt::Debug for Endpoint { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("Endpoint") - .field("socket_addr", &self.socket_addr) - .finish() - } -} diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index f2b6d3366bb..f9b5174dad9 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -195,25 +195,29 @@ impl StreamMuxer for QuicMuxer { } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let mut inner = self.inner.lock(); - if inner.connection.connection.is_drained() { + let Inner { + substreams, + connection, + .. + } = &mut *self.inner.lock(); + if connection.connection.is_drained() { return Poll::Ready(Ok(())); } - if inner.connection.connection.streams().send_streams() != 0 { - for substream in inner.substreams.keys().cloned().collect::>() { - if let Err(e) = inner.connection.finish_substream(substream) { + if connection.connection.streams().send_streams() != 0 { + for substream in substreams.keys() { + if let Err(e) = connection.finish_substream(*substream) { tracing::warn!("substream finish error on muxer close: {}", e); } } } loop { - if inner.connection.connection.streams().send_streams() == 0 - && !inner.connection.connection.is_closed() + if connection.connection.streams().send_streams() == 0 + && !connection.connection.is_closed() { - inner.connection.close() + connection.close() } - match inner.connection.poll_event(cx) { + match connection.poll_event(cx) { Poll::Ready(ConnectionEvent::ConnectionLost(_)) => return Poll::Ready(Ok(())), Poll::Ready(_) => {} Poll::Pending => break, diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index e915a29d532..40a7ba970fe 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -411,7 +411,7 @@ impl Stream for Listener { /// /// Returns `None` if the address is not the same socket family as the /// address that the endpoint is bound to. -pub fn ip_to_listenaddr(endpoint: &Endpoint, ip: IpAddr) -> Option { +fn ip_to_listenaddr(endpoint: &Endpoint, ip: IpAddr) -> Option { // True if either both addresses are Ipv4 or both Ipv6. let is_same_ip_family = endpoint.socket_addr().is_ipv4() == ip.is_ipv4(); if !is_same_ip_family { From 0aba5849489243ce23dcbd3231187a267ec66c5d Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 23 Sep 2022 12:51:39 +1000 Subject: [PATCH 100/218] Use FQN instead of renaming import This makes it easier to spot where a type is coming from. --- transports/quic/src/tls/verifier.rs | 34 ++++++++++++++--------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 2ca6fa45b6b..7d161cd72e4 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -27,7 +27,7 @@ use rustls::{ client::{HandshakeSignatureValid, ServerCertVerified, ServerCertVerifier}, internal::msgs::handshake::DigitallySignedStruct, server::{ClientCertVerified, ClientCertVerifier}, - Certificate, DistinguishedNames, Error as TlsError, SignatureScheme, + Certificate, DistinguishedNames, SignatureScheme, }; /// Implementation of the `rustls` certificate verification traits for libp2p. @@ -74,7 +74,7 @@ impl ServerCertVerifier for Libp2pCertificateVerifier { _scts: &mut dyn Iterator, _ocsp_response: &[u8], _now: std::time::SystemTime, - ) -> Result { + ) -> Result { verify_presented_certs(end_entity, intermediates).map(|_| ServerCertVerified::assertion()) } @@ -83,10 +83,10 @@ impl ServerCertVerifier for Libp2pCertificateVerifier { _message: &[u8], _cert: &Certificate, _dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { // The libp2p handshake uses TLS 1.3 (and higher). // Endpoints MUST NOT negotiate lower TLS versions. - Err(TlsError::PeerIncompatibleError( + Err(rustls::Error::PeerIncompatibleError( "Only TLS 1.3 certificates are supported".to_string(), )) } @@ -96,7 +96,7 @@ impl ServerCertVerifier for Libp2pCertificateVerifier { message: &[u8], cert: &Certificate, dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { verify_tls13_signature(cert, dss.scheme, message, dss.sig.0.as_ref()) } @@ -126,7 +126,7 @@ impl ClientCertVerifier for Libp2pCertificateVerifier { end_entity: &Certificate, intermediates: &[Certificate], _now: std::time::SystemTime, - ) -> Result { + ) -> Result { verify_presented_certs(end_entity, intermediates).map(|_| ClientCertVerified::assertion()) } @@ -135,10 +135,10 @@ impl ClientCertVerifier for Libp2pCertificateVerifier { _message: &[u8], _cert: &Certificate, _dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { // The libp2p handshake uses TLS 1.3 (and higher). // Endpoints MUST NOT negotiate lower TLS versions. - Err(TlsError::PeerIncompatibleError( + Err(rustls::Error::PeerIncompatibleError( "Only TLS 1.3 certificates are supported".to_string(), )) } @@ -148,7 +148,7 @@ impl ClientCertVerifier for Libp2pCertificateVerifier { message: &[u8], cert: &Certificate, dss: &DigitallySignedStruct, - ) -> Result { + ) -> Result { verify_tls13_signature(cert, dss.scheme, message, dss.sig.0.as_ref()) } @@ -166,9 +166,9 @@ impl ClientCertVerifier for Libp2pCertificateVerifier { fn verify_presented_certs( end_entity: &Certificate, intermediates: &[Certificate], -) -> Result<(), TlsError> { +) -> Result<(), rustls::Error> { if !intermediates.is_empty() { - return Err(TlsError::General( + return Err(rustls::Error::General( "libp2p-tls requires exactly one certificate".into(), )); } @@ -182,21 +182,21 @@ fn verify_tls13_signature( signature_scheme: SignatureScheme, message: &[u8], signature: &[u8], -) -> Result { +) -> Result { crate::tls::certificate::parse_certificate(cert.as_ref()) .and_then(|cert| cert.verify_signature(signature_scheme, message, signature)) .map(|()| HandshakeSignatureValid::assertion()) .map_err(pki_error) } -fn pki_error(error: webpki::Error) -> TlsError { +fn pki_error(error: webpki::Error) -> rustls::Error { use webpki::Error::*; match error { - BadDer | BadDerTime => TlsError::InvalidCertificateEncoding, - InvalidSignatureForPublicKey => TlsError::InvalidCertificateSignature, + BadDer | BadDerTime => rustls::Error::InvalidCertificateEncoding, + InvalidSignatureForPublicKey => rustls::Error::InvalidCertificateSignature, UnsupportedSignatureAlgorithm | UnsupportedSignatureAlgorithmForPublicKey => { - TlsError::InvalidCertificateSignatureType + rustls::Error::InvalidCertificateSignatureType } - e => TlsError::InvalidCertificateData(format!("invalid peer certificate: {}", e)), + e => rustls::Error::InvalidCertificateData(format!("invalid peer certificate: {}", e)), } } From 78a7566b72a567c42e4e9c5b00434e8430fe5cfc Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 23 Sep 2022 12:55:57 +1000 Subject: [PATCH 101/218] Import functions that are used multiple times --- transports/quic/src/tls/verifier.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index 7d161cd72e4..ea297ca15e3 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -23,6 +23,7 @@ //! This module handles a verification of a client/server certificate chain //! and signatures allegedly by the given certificates. +use crate::tls::certificate::parse_certificate; use rustls::{ client::{HandshakeSignatureValid, ServerCertVerified, ServerCertVerifier}, internal::msgs::handshake::DigitallySignedStruct, @@ -172,7 +173,7 @@ fn verify_presented_certs( "libp2p-tls requires exactly one certificate".into(), )); } - crate::tls::certificate::parse_certificate(end_entity.as_ref()) + parse_certificate(end_entity.as_ref()) .and_then(|cert| cert.verify()) .map_err(pki_error) } @@ -183,7 +184,7 @@ fn verify_tls13_signature( message: &[u8], signature: &[u8], ) -> Result { - crate::tls::certificate::parse_certificate(cert.as_ref()) + parse_certificate(cert.as_ref()) .and_then(|cert| cert.verify_signature(signature_scheme, message, signature)) .map(|()| HandshakeSignatureValid::assertion()) .map_err(pki_error) From edf3f363a50ad6608b867930a5d9c2134f60d1f8 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 23 Sep 2022 12:59:40 +1000 Subject: [PATCH 102/218] Favor `?` over `map` and `and_then` --- transports/quic/src/tls/verifier.rs | 24 +++++++++++++++++------- 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs index ea297ca15e3..f7592816917 100644 --- a/transports/quic/src/tls/verifier.rs +++ b/transports/quic/src/tls/verifier.rs @@ -76,7 +76,9 @@ impl ServerCertVerifier for Libp2pCertificateVerifier { _ocsp_response: &[u8], _now: std::time::SystemTime, ) -> Result { - verify_presented_certs(end_entity, intermediates).map(|_| ServerCertVerified::assertion()) + verify_presented_certs(end_entity, intermediates)?; + + Ok(ServerCertVerified::assertion()) } fn verify_tls12_signature( @@ -128,7 +130,9 @@ impl ClientCertVerifier for Libp2pCertificateVerifier { intermediates: &[Certificate], _now: std::time::SystemTime, ) -> Result { - verify_presented_certs(end_entity, intermediates).map(|_| ClientCertVerified::assertion()) + verify_presented_certs(end_entity, intermediates)?; + + Ok(ClientCertVerified::assertion()) } fn verify_tls12_signature( @@ -173,9 +177,13 @@ fn verify_presented_certs( "libp2p-tls requires exactly one certificate".into(), )); } + parse_certificate(end_entity.as_ref()) - .and_then(|cert| cert.verify()) - .map_err(pki_error) + .map_err(pki_error)? + .verify() + .map_err(pki_error)?; + + Ok(()) } fn verify_tls13_signature( @@ -185,9 +193,11 @@ fn verify_tls13_signature( signature: &[u8], ) -> Result { parse_certificate(cert.as_ref()) - .and_then(|cert| cert.verify_signature(signature_scheme, message, signature)) - .map(|()| HandshakeSignatureValid::assertion()) - .map_err(pki_error) + .map_err(pki_error)? + .verify_signature(signature_scheme, message, signature) + .map_err(pki_error)?; + + Ok(HandshakeSignatureValid::assertion()) } fn pki_error(error: webpki::Error) -> rustls::Error { From eea474df9b137ebbfacdefc7fac978f70f31660b Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 23 Sep 2022 13:13:20 +1000 Subject: [PATCH 103/218] Favor early return over `if let Some` --- transports/quic/src/tls/certificate.rs | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs index b856ddcc855..497c8b9130e 100644 --- a/transports/quic/src/tls/certificate.rs +++ b/transports/quic/src/tls/certificate.rs @@ -179,16 +179,14 @@ pub fn parse_certificate(der_input: &[u8]) -> Result { From 40b600df0f32e4e37a160336be294fcad3e5e12d Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 23 Sep 2022 13:22:31 +1000 Subject: [PATCH 104/218] Fix new clippy lint --- transports/quic/src/connection.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 8b3f371ffe0..a3a6cecba2c 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -269,8 +269,8 @@ impl Connection { } // The final step consists in handling the events related to the various substreams. - match self.connection.poll() { - Some(ev) => match ConnectionEvent::try_from(ev) { + if let Some(ev) = self.connection.poll() { + match ConnectionEvent::try_from(ev) { Ok(ConnectionEvent::ConnectionLost(reason)) => { // Continue in the loop once more so that we can send a // `EndpointEvent::drained` to the endpoint before returning. @@ -279,9 +279,9 @@ impl Connection { } Ok(event) => return Poll::Ready(event), Err(_) => unreachable!("We don't use datagrams or unidirectional streams."), - }, - None => {} + } } + return Poll::Pending; } } From c5fc181f058c794bea0a98c3e6fc1ec6f8be75b5 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 28 Sep 2022 01:05:09 +0200 Subject: [PATCH 105/218] transports/quic: abstract runtime Add `Provider` abstraction to support multiple runtimes. Add `Provider` implementation for `tokio` and `async-std` runtimes. --- Cargo.toml | 7 +- transports/quic/Cargo.toml | 15 ++-- transports/quic/src/connection.rs | 29 +++++--- transports/quic/src/endpoint.rs | 44 +++++++----- transports/quic/src/lib.rs | 21 +++++- transports/quic/src/transport.rs | 82 ++++++++++++++++++---- transports/quic/src/transport/async_std.rs | 52 ++++++++++++++ transports/quic/src/transport/tokio.rs | 52 ++++++++++++++ transports/quic/tests/smoke.rs | 75 ++++++++++++++------ 9 files changed, 306 insertions(+), 71 deletions(-) create mode 100644 transports/quic/src/transport/async_std.rs create mode 100644 transports/quic/src/transport/tokio.rs diff --git a/Cargo.toml b/Cargo.toml index 094d89dccbd..245a64c57d2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -25,7 +25,7 @@ default = [ "ping", "plaintext", "pnet", - "quic", + "quic-async-std", "relay", "request-response", "rendezvous", @@ -55,7 +55,8 @@ noise = ["dep:libp2p-noise"] ping = ["dep:libp2p-ping", "libp2p-metrics?/ping"] plaintext = ["dep:libp2p-plaintext"] pnet = ["dep:libp2p-pnet"] -quic = ["dep:libp2p-quic"] +quic-async-std = ["dep:libp2p-quic", "libp2p-quic?/async-std"] +quic-tokio = ["dep:libp2p-quic", "libp2p-quic?/tokio"] relay = ["dep:libp2p-relay", "libp2p-metrics?/relay"] request-response = ["dep:libp2p-request-response"] rendezvous = ["dep:libp2p-rendezvous"] @@ -111,7 +112,7 @@ smallvec = "1.6.1" libp2p-deflate = { version = "0.36.0", path = "transports/deflate", optional = true } libp2p-dns = { version = "0.36.0", path = "transports/dns", optional = true, default-features = false } libp2p-mdns = { version = "0.40.0", path = "protocols/mdns", optional = true, default-features = false } -libp2p-quic = { version = "0.7.0", path = "transports/quic", optional = true } +libp2p-quic = { version = "0.7.0", path = "transports/quic", default-features = false, optional = true } libp2p-tcp = { version = "0.37.0", path = "transports/tcp", default-features = false, optional = true } libp2p-websocket = { version = "0.38.0", path = "transports/websocket", optional = true } diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index a5bd2fad149..0f75d4d9be5 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -8,9 +8,10 @@ repository = "https://github.com/libp2p/rust-libp2p" license = "MIT" [dependencies] -async-global-executor = "2.0.2" -async-io = "1.6.0" +async-std-crate = { package = "async-std", version = "1.12.0", default-features = false, optional = true } +async-trait = "0.1.50" futures = "0.3.15" +futures-timer = "3.0.2" if-watch = "2.0.0" libp2p-core = { version = "0.36.0", path = "../../core" } parking_lot = "0.12.0" @@ -20,16 +21,22 @@ rcgen = "0.9.2" ring = "0.16.20" rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } thiserror = "1.0.26" +tokio-crate = { package = "tokio", version = "1.21.1", default-features = false, features = ["net", "rt"], optional = true } tracing = "0.1.30" webpki = "0.22.0" x509-parser = "0.13.0" yasna = "0.5.0" +[features] +default = ["async-std"] +tokio = ["tokio-crate"] +async-std = ["async-std-crate"] + [dev-dependencies] anyhow = "1.0.41" -async-std = { version = "1.10.0", features = ["attributes"] } -async-trait = "0.1.50" +async-std-crate = { package = "async-std", version = "1.12.0", features = ["attributes"] } libp2p = { version = "0.49.0", default-features = false, features = ["request-response"], path = "../.." } rand = "0.8.4" +tokio-crate = { package = "tokio", version = "1.21.1", features = ["macros", "rt-multi-thread"] } tracing-subscriber = {version = "0.3.8", features = ["env-filter"] } quickcheck = "1" diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index a3a6cecba2c..46fc67cae2d 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -28,8 +28,8 @@ use crate::endpoint::{Endpoint, ToEndpoint}; -use async_io::Timer; use futures::{channel::mpsc, prelude::*}; +use futures_timer::Delay; use libp2p_core::PeerId; use std::{ fmt, @@ -57,7 +57,7 @@ pub struct Connection { /// the endpoint. connection_id: quinn_proto::ConnectionHandle, /// `Future` that triggers at the `Instant` that `self.connection.poll_timeout()` indicates. - next_timeout: Option, + next_timeout: Option<(Delay, Instant)>, } /// Error on the connection as a whole. @@ -241,15 +241,24 @@ impl Connection { continue; } - // Timeout system. - if let Some(when) = self.connection.poll_timeout() { - let mut timer = Timer::at(when); - match timer.poll_unpin(cx) { - Poll::Ready(when) => { - self.connection.handle_timeout(when); - continue; + match self.connection.poll_timeout() { + Some(timeout) => match self.next_timeout { + Some((_, when)) if when == timeout => {} + _ => { + let now = Instant::now(); + // 0ns if now > when + let duration = timeout.duration_since(now); + let next_timeout = Delay::new(duration); + self.next_timeout = Some((next_timeout, timeout)) } - Poll::Pending => self.next_timeout = Some(timer), + }, + None => self.next_timeout = None, + } + + if let Some((timeout, when)) = self.next_timeout.as_mut() { + if timeout.poll_unpin(cx).is_ready() { + self.connection.handle_timeout(*when); + continue; } } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 0f88f9860bd..2592ddfbae5 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,7 +28,11 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`background_task`] for a thorough description. -use crate::{connection::Connection, tls, transport}; +use crate::{ + connection::Connection, + tls, + transport::{self, Provider}, +}; use futures::{ channel::{ @@ -40,7 +44,7 @@ use futures::{ use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; use std::{ collections::HashMap, - net::{Ipv4Addr, Ipv6Addr, SocketAddr, UdpSocket}, + net::{Ipv4Addr, Ipv6Addr, SocketAddr}, sync::Arc, task::{Context, Poll}, time::{Duration, Instant}, @@ -98,32 +102,36 @@ pub struct Endpoint { impl Endpoint { /// Builds a new [`Endpoint`] that is listening on the [`SocketAddr`]. - pub fn new_bidirectional( + pub fn new_bidirectional( config: Config, socket_addr: SocketAddr, ) -> Result<(Endpoint, mpsc::Receiver), transport::Error> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); - let endpoint = Self::new(config, socket_addr, Some(new_connections_tx))?; + let endpoint = Self::new::

(config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) } /// Builds a new [`Endpoint`] that only supports outbound connections. - pub fn new_dialer(config: Config, is_ipv6: bool) -> Result { + pub fn new_dialer( + config: Config, + is_ipv6: bool, + ) -> Result { let socket_addr = if is_ipv6 { SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0) } else { SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), 0) }; - Self::new(config, socket_addr, None) + Self::new::

(config, socket_addr, None) } - fn new( + fn new( config: Config, socket_addr: SocketAddr, new_connections: Option>, ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; + socket.set_nonblocking(true)?; let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(32); let endpoint = Endpoint { @@ -133,16 +141,16 @@ impl Endpoint { let server_config = new_connections.map(|c| (c, config.server_config.clone())); - // TODO: just for testing, do proper task spawning - async_global_executor::spawn(background_task( + let socket = P::from_socket(socket)?; + + P::spawn(background_task::

( config.endpoint_config, config.client_config, server_config, endpoint.clone(), - async_io::Async::::new(socket)?, + socket, to_endpoint_rx.fuse(), - )) - .detach(); + )); Ok(endpoint) } @@ -275,12 +283,12 @@ pub enum ToEndpoint { /// guarantees that the [`Endpoint`], and therefore the background task, is properly kept alive /// for as long as any QUIC connection is open. /// -async fn background_task( +async fn background_task( endpoint_config: Arc, client_config: quinn_proto::ClientConfig, server_config: Option<(mpsc::Sender, Arc)>, endpoint: Endpoint, - udp_socket: async_io::Async, + udp_socket: P::Socket, mut receiver: stream::Fuse>, ) { let (mut new_connections, server_config) = match server_config { @@ -310,7 +318,7 @@ async fn background_task( // network interface is too busy, we back-pressure all of our internal // channels. // TODO: set ECN bits; there is no support for them in the ecosystem right now - match udp_socket.send_to(&data, destination).await { + match P::send_to(&udp_socket, &data, destination).await { Ok(n) if n == data.len() => {} Ok(_) => tracing::error!( "QUIC UDP socket violated expectation that packets are always fully \ @@ -398,7 +406,7 @@ async fn background_task( } } } - result = udp_socket.recv_from(&mut socket_recv_buffer).fuse() => { + result = P::recv_from(&udp_socket, &mut socket_recv_buffer).fuse() => { let (packet_len, packet_src) = match result { Ok(v) => v, // Errors on the socket are expected to never happen, and we handle them by @@ -412,9 +420,9 @@ async fn background_task( // Received a UDP packet from the socket. debug_assert!(packet_len <= socket_recv_buffer.len()); let packet = From::from(&socket_recv_buffer[..packet_len]); - let local_ip = udp_socket.get_ref().local_addr().ok().map(|a| a.ip()); + let local_ip = endpoint.socket_addr.ip(); // TODO: ECN bits aren't handled - let event = proto_endpoint.handle(Instant::now(), packet_src, local_ip, None, packet); + let event = proto_endpoint.handle(Instant::now(), packet_src, Some(local_ip), None, packet); match event { None => {}, diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 2d9a4491bab..4bb781eee39 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -25,14 +25,25 @@ //! Example: //! //! ``` -//! use libp2p_quic::{Config, QuicTransport}; +//! # #[cfg(not(feature = "async-std"))] +//! # fn main() {} +//! # +//! # #[cfg(feature = "async-std")] +//! # fn main() -> std::io::Result<()> { +//! # +//! use libp2p_quic::{AsyncStdTransport, Config}; //! use libp2p_core::{Multiaddr, Transport}; //! //! let keypair = libp2p_core::identity::Keypair::generate_ed25519(); //! let quic_config = Config::new(&keypair).expect("could not make config"); -//! let mut quic_transport = QuicTransport::new(quic_config); +//! +//! let mut quic_transport = AsyncStdTransport::new(quic_config); +//! //! let addr = "/ip4/127.0.0.1/udp/12345/quic".parse().expect("bad address?"); //! quic_transport.listen_on(addr).expect("listen error."); +//! # +//! # Ok(()) +//! # } //! ``` //! //! The `Endpoint` struct implements the `Transport` trait of the `core` library. See the @@ -64,5 +75,9 @@ pub mod transport; pub use endpoint::Config; pub use error::Error; pub use muxer::QuicMuxer; -pub use transport::QuicTransport; +#[cfg(feature = "async-std")] +pub use transport::{AsyncStd, AsyncStdTransport}; +pub use transport::{Provider, QuicTransport}; +#[cfg(feature = "tokio")] +pub use transport::{Tokio, TokioTransport}; pub use upgrade::Upgrade; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 40a7ba970fe..3f324be06b4 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -27,6 +27,15 @@ use crate::endpoint::ToEndpoint; use crate::Config; use crate::{endpoint::Endpoint, muxer::QuicMuxer, upgrade::Upgrade}; +#[cfg(feature = "async-std")] +mod async_std; +#[cfg(feature = "tokio")] +mod tokio; +#[cfg(feature = "async-std")] +pub use async_std::{AsyncStd, AsyncStdTransport}; +#[cfg(feature = "tokio")] +pub use tokio::{Tokio, TokioTransport}; + use futures::channel::{mpsc, oneshot}; use futures::ready; use futures::stream::StreamExt; @@ -40,7 +49,9 @@ use libp2p_core::{ PeerId, Transport, }; use std::collections::VecDeque; -use std::net::IpAddr; +use std::io; +use std::marker::PhantomData; +use std::net::{IpAddr, UdpSocket}; use std::task::Waker; use std::{ net::SocketAddr, @@ -57,22 +68,25 @@ pub use quinn_proto::{ }; #[derive(Debug)] -pub struct QuicTransport { +pub struct QuicTransport

{ config: Config, listeners: SelectAll, /// Dialer for Ipv4 addresses if no matching listener exists. ipv4_dialer: Option, /// Dialer for Ipv6 addresses if no matching listener exists. ipv6_dialer: Option, + + _marker: PhantomData

, } -impl QuicTransport { +impl

QuicTransport

{ pub fn new(config: Config) -> Self { Self { listeners: SelectAll::new(), config, ipv4_dialer: None, ipv6_dialer: None, + _marker: Default::default(), } } } @@ -94,7 +108,7 @@ pub enum Error { TaskCrashed, } -impl Transport for QuicTransport { +impl Transport for QuicTransport

{ type Output = (PeerId, QuicMuxer); type Error = Error; type ListenerUpgrade = Upgrade; @@ -104,7 +118,7 @@ impl Transport for QuicTransport { let socket_addr = multiaddr_to_socketaddr(&addr).ok_or(TransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); - let listener = Listener::new(listener_id, socket_addr, self.config.clone()) + let listener = Listener::new::

(listener_id, socket_addr, self.config.clone()) .map_err(TransportError::Other)?; self.listeners.push(listener); // Drop reference to dialer endpoint so that the endpoint is dropped once the last @@ -157,7 +171,10 @@ impl Transport for QuicTransport { SocketAddr::V6(_) => &mut self.ipv6_dialer, }; if dialer.is_none() { - let _ = dialer.insert(Dialer::new(self.config.clone(), socket_addr.is_ipv6())?); + let _ = dialer.insert(Dialer::new::

( + self.config.clone(), + socket_addr.is_ipv6(), + )?); } dialer .as_mut() @@ -228,8 +245,8 @@ struct Dialer { } impl Dialer { - fn new(config: Config, is_ipv6: bool) -> Result> { - let endpoint = Endpoint::new_dialer(config, is_ipv6).map_err(TransportError::Other)?; + fn new(config: Config, is_ipv6: bool) -> Result> { + let endpoint = Endpoint::new_dialer::

(config, is_ipv6).map_err(TransportError::Other)?; Ok(Dialer { endpoint, pending_dials: VecDeque::new(), @@ -273,12 +290,12 @@ struct Listener { } impl Listener { - fn new( + fn new( listener_id: ListenerId, socket_addr: SocketAddr, config: Config, ) -> Result { - let (endpoint, new_connections_rx) = Endpoint::new_bidirectional(config, socket_addr)?; + let (endpoint, new_connections_rx) = Endpoint::new_bidirectional::

(config, socket_addr)?; let if_watcher; let pending_event; @@ -357,7 +374,7 @@ impl Listener { } impl Stream for Listener { - type Item = TransportEvent<::ListenerUpgrade, Error>; + type Item = TransportEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { if let Some(event) = self.pending_event.take() { @@ -406,6 +423,26 @@ impl Stream for Listener { } } } +#[async_trait::async_trait] +pub trait Provider: Unpin + Send + 'static { + // Wrapped socket for non-blocking I/O operations. + type Socket: Send + Sync + Unpin; + + // Wrap a socket. + // Note: The socket must be set to non-blocking. + fn from_socket(socket: UdpSocket) -> io::Result; + + // Receive a single datagram message. + // Return the number of bytes read and the address the message came from. + async fn recv_from(socket: &Self::Socket, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)>; + + // Send data on the socket to the specified address. + // Return the number of bytes written. + async fn send_to(socket: &Self::Socket, buf: &[u8], addr: SocketAddr) -> io::Result; + + // Run the given future in the background until it ends. + fn spawn(future: impl Future + Send + 'static); +} /// Turn an [`IpAddr`] into a listen-address for the endpoint. /// @@ -456,10 +493,15 @@ pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { } #[cfg(test)] +#[cfg(any(feature = "async-std", feature = "tokio"))] mod test { + #[cfg(feature = "async-std")] + use async_std_crate as async_std; use futures::future::poll_fn; use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; + #[cfg(feature = "tokio")] + use tokio_crate as tokio; use super::*; @@ -525,11 +567,25 @@ mod test { ); } + #[cfg(feature = "tokio")] + #[tokio::test] + async fn tokio_close_listener() { + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); + let config = Config::new(&keypair).unwrap(); + let transport = TokioTransport::new(config.clone()); + test_close_listener(transport).await + } + + #[cfg(feature = "async-std")] #[async_std::test] - async fn close_listener() { + async fn async_std_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); - let mut transport = QuicTransport::new(Config::new(&keypair).unwrap()); + let config = Config::new(&keypair).unwrap(); + let transport = AsyncStdTransport::new(config.clone()); + test_close_listener(transport).await + } + async fn test_close_listener(mut transport: QuicTransport

) { assert!(poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)) .now_or_never() .is_none()); diff --git a/transports/quic/src/transport/async_std.rs b/transports/quic/src/transport/async_std.rs new file mode 100644 index 00000000000..26c853ccc76 --- /dev/null +++ b/transports/quic/src/transport/async_std.rs @@ -0,0 +1,52 @@ +// Copyright 2022 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 std::{io, net::SocketAddr}; + +use async_std_crate::{net::UdpSocket, task::spawn}; +use futures::Future; + +use crate::QuicTransport; + +use super::Provider; + +pub type AsyncStdTransport = QuicTransport; +pub struct AsyncStd; + +#[async_trait::async_trait] +impl Provider for AsyncStd { + type Socket = UdpSocket; + + fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { + Ok(socket.into()) + } + + async fn recv_from(socket: &Self::Socket, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { + socket.recv_from(buf).await + } + + async fn send_to(socket: &Self::Socket, buf: &[u8], addr: SocketAddr) -> io::Result { + socket.send_to(buf, addr).await + } + + fn spawn(future: impl Future + Send + 'static) { + spawn(future); + } +} diff --git a/transports/quic/src/transport/tokio.rs b/transports/quic/src/transport/tokio.rs new file mode 100644 index 00000000000..fc9043a3343 --- /dev/null +++ b/transports/quic/src/transport/tokio.rs @@ -0,0 +1,52 @@ +// Copyright 2022 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 std::{io, net::SocketAddr}; + +use futures::Future; +use tokio_crate::net::UdpSocket; + +use crate::QuicTransport; + +use super::Provider; + +pub type TokioTransport = QuicTransport; +pub struct Tokio; + +#[async_trait::async_trait] +impl Provider for Tokio { + type Socket = UdpSocket; + + fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { + UdpSocket::from_std(socket) + } + + async fn recv_from(socket: &Self::Socket, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { + socket.recv_from(buf).await + } + + async fn send_to(socket: &Self::Socket, buf: &[u8], addr: SocketAddr) -> io::Result { + socket.send_to(buf, addr).await + } + + fn spawn(future: impl Future + Send + 'static) { + tokio_crate::spawn(future); + } +} diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index d6507e73c45..8019a9a2ab0 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -15,21 +15,31 @@ use libp2p::request_response::{ }; use libp2p::swarm::dial_opts::{DialOpts, PeerCondition}; use libp2p::swarm::{ConnectionError, DialError, Swarm, SwarmEvent}; -use libp2p_quic::{Config as QuicConfig, QuicTransport}; +use libp2p_quic as quic; +use quic::Provider; use rand::RngCore; use std::num::NonZeroU8; use std::{io, iter}; +#[cfg(feature = "async-std")] +use async_std_crate as async_std; +#[cfg(feature = "async-std")] +use libp2p_quic::AsyncStd; +#[cfg(feature = "tokio")] +use libp2p_quic::Tokio; +#[cfg(feature = "tokio")] +use tokio_crate as tokio; + fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } #[tracing::instrument] -async fn create_swarm(keylog: bool) -> Result>> { +async fn create_swarm(keylog: bool) -> Result>> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); - let config = QuicConfig::new(&keypair).unwrap(); - let transport = QuicTransport::new(config); + let config = quic::Config::new(&keypair).unwrap(); + let transport = quic::QuicTransport::

::new(config); let transport = Transport::map(transport, |(peer, muxer), _| { (peer, StreamMuxerBox::new(muxer)) @@ -51,13 +61,24 @@ fn setup_global_subscriber() { .ok(); } +#[cfg(feature = "tokio")] +#[tokio::test] +async fn tokio_smoke() -> Result<()> { + smoke::().await +} + +#[cfg(feature = "async-std")] #[async_std::test] -async fn smoke() -> Result<()> { +async fn async_std_smoke() -> Result<()> { + smoke::().await +} + +async fn smoke() -> Result<()> { setup_global_subscriber(); 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()?)?; @@ -279,12 +300,13 @@ impl RequestResponseCodec for PingCodec { } } +#[cfg(feature = "async-std")] #[async_std::test] async fn dial_failure() -> Result<()> { setup_global_subscriber(); - let mut a = create_swarm(false).await?; - let mut b = create_swarm(true).await?; + let mut a = create_swarm::(false).await?; + let mut b = create_swarm::(true).await?; Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; @@ -319,13 +341,15 @@ async fn dial_failure() -> Result<()> { #[test] fn concurrent_connections_and_streams() { - use futures::executor::block_on; use quickcheck::*; setup_global_subscriber(); #[tracing::instrument] - fn prop(number_listeners: NonZeroU8, number_streams: NonZeroU8) -> TestResult { + async fn prop( + number_listeners: NonZeroU8, + number_streams: NonZeroU8, + ) -> TestResult { let (number_listeners, number_streams): (u8, u8) = (number_listeners.into(), number_streams.into()); if number_listeners > 10 || number_streams > 10 { @@ -339,11 +363,11 @@ fn concurrent_connections_and_streams() { // Spawn the listener nodes. for _ in 0..number_listeners { - let mut listener = block_on(create_swarm(true)).unwrap(); + let mut listener = create_swarm::

(true).await.unwrap(); Swarm::listen_on(&mut listener, "/ip4/127.0.0.1/udp/0/quic".parse().unwrap()).unwrap(); // Wait to listen on address. - let addr = match block_on(listener.next()) { + let addr = match listener.next().await { Some(SwarmEvent::NewListenAddr { address, .. }) => address, e => panic!("{:?}", e), }; @@ -397,7 +421,7 @@ fn concurrent_connections_and_streams() { .unwrap(); } - let mut dialer = block_on(create_swarm(true)).unwrap(); + let mut dialer = create_swarm::

(true).await.unwrap(); // For each listener node start `number_streams` requests. for (listener_peer_id, listener_addr) in &listeners { @@ -454,17 +478,27 @@ fn concurrent_connections_and_streams() { TestResult::passed() } - prop(NonZeroU8::new(3).unwrap(), NonZeroU8::new(8).unwrap()); + let num_listener = NonZeroU8::new(3).unwrap(); + let num_streams = NonZeroU8::new(8).unwrap(); + + #[cfg(feature = "tokio")] + tokio::runtime::Runtime::new() + .unwrap() + .block_on(prop::(num_listener, num_streams)); + + #[cfg(feature = "async-std")] + async_std::task::block_on(prop::(num_listener, num_streams)); // QuickCheck::new().quickcheck(prop as fn(_, _) -> _); } -#[async_std::test] +#[cfg(feature = "tokio")] +#[tokio::test] async fn endpoint_reuse() -> Result<()> { setup_global_subscriber(); - let mut swarm_a = create_swarm(false).await?; - let mut swarm_b = create_swarm(false).await?; + let mut swarm_a = create_swarm::(false).await?; + let mut swarm_b = create_swarm::(false).await?; let b_peer_id = *swarm_b.local_peer_id(); swarm_a.listen_on("/ip4/127.0.0.1/udp/0/quic".parse()?)?; @@ -560,12 +594,13 @@ async fn endpoint_reuse() -> Result<()> { Ok(()) } +#[cfg(feature = "async-std")] #[async_std::test] async fn ipv4_dial_ipv6() -> Result<()> { setup_global_subscriber(); - let mut swarm_a = create_swarm(false).await?; - let mut swarm_b = create_swarm(false).await?; + let mut swarm_a = create_swarm::(false).await?; + let mut swarm_b = create_swarm::(false).await?; swarm_a.listen_on("/ip6/::1/udp/0/quic".parse()?)?; let a_addr = match swarm_a.next().await { From df8de300173270a3e7bdd4de833b32a7778254cf Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 28 Sep 2022 04:17:47 +0200 Subject: [PATCH 106/218] transports/quic: minor fixes --- transports/quic/src/connection.rs | 73 ++++++++++++++++++------------- transports/quic/src/endpoint.rs | 16 +++++-- transports/quic/src/muxer.rs | 20 +++------ transports/quic/src/transport.rs | 6 +-- 4 files changed, 63 insertions(+), 52 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 46fc67cae2d..d1e541d80db 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -31,6 +31,7 @@ use crate::endpoint::{Endpoint, ToEndpoint}; use futures::{channel::mpsc, prelude::*}; use futures_timer::Delay; use libp2p_core::PeerId; +use quinn_proto::{RecvStream, SendStream}; use std::{ fmt, net::SocketAddr, @@ -52,7 +53,7 @@ pub struct Connection { from_endpoint: mpsc::Receiver, /// The QUIC state machine for this specific connection. - pub connection: quinn_proto::Connection, + connection: quinn_proto::Connection, /// Identifier for this connection according to the endpoint. Used when sending messages to /// the endpoint. connection_id: quinn_proto::ConnectionHandle, @@ -63,9 +64,6 @@ pub struct Connection { /// Error on the connection as a whole. #[derive(Debug, Clone, thiserror::Error)] pub enum Error { - /// Endpoint has force-killed this connection because it was too busy. - #[error("Endpoint has force-killed our connection")] - ClosedChannel, /// The background task driving the endpoint has crashed. #[error("Background task crashed")] TaskCrashed, @@ -78,11 +76,7 @@ impl Connection { /// Crate-internal function that builds a [`Connection`] from raw components. /// /// This function assumes that there exists a background task that will process the messages - /// sent to `to_endpoint` and send us messages on `from_endpoint`. - /// - /// The `from_endpoint` can be purposefully closed by the endpoint if the connection is too - /// slow to process. - // TODO: is this necessary ^? figure out if quinn_proto doesn't forbid that situation in the first place + /// sent to `Endpoint::to_endpoint` and send us messages on `from_endpoint`. /// /// `connection_id` is used to identify the local connection in the messages sent to /// `to_endpoint`. @@ -106,24 +100,9 @@ impl Connection { } } - /// The local address which was used when the remote established the connection to us. - /// - /// `None` for client connections. - pub fn local_addr(&self) -> Option { - if self.connection.side().is_client() { - return None; - } - let endpoint_addr = self.endpoint.socket_addr(); - - // Local address may differ from the socket address if the socket is - // bound to a wildcard address. - let addr = match self.connection.local_ip() { - Some(ip) => SocketAddr::new(ip, endpoint_addr.port()), - // TODO: `quinn_proto::Connection::local_ip` is only supported for linux, - // so for other platforms we currently still return the endpoint address. - None => *endpoint_addr, - }; - Some(addr) + /// The address that the local socket is bound to. + pub fn local_addr(&self) -> &SocketAddr { + self.endpoint.socket_addr() } /// Returns the address of the node we're connected to. @@ -131,7 +110,7 @@ impl Connection { self.connection.remote_address() } - /// Returns the address of the node we're connected to. + /// Returns the ID of the node we're connected to. pub fn remote_peer_id(&self) -> PeerId { let session = self.connection.crypto_session(); let identity = session @@ -157,6 +136,19 @@ impl Connection { .close(Instant::now(), From::from(0u32), Default::default()); } + /// Whether the connection is closed. + /// A [`ConnectionEvent::ConnectionLost`] event is emitted with details when the + /// connection becomes closed. + pub fn is_closed(&self) -> bool { + self.connection.is_closed() + } + + /// Whether there is no longer any need to keep the connection around. + /// All drained connections have been closed. + pub fn is_drained(&self) -> bool { + self.connection.is_drained() + } + /// Pops a new substream opened by the remote. /// /// If `None` is returned, then a [`ConnectionEvent::StreamAvailable`] event will later be @@ -176,6 +168,21 @@ impl Connection { self.connection.streams().open(quinn_proto::Dir::Bi) } + /// Control over the stream for reading. + pub fn recv_stream(&mut self, id: quinn_proto::StreamId) -> RecvStream<'_> { + self.connection.recv_stream(id) + } + + /// Control over the stream for writing. + pub fn send_stream(&mut self, id: quinn_proto::StreamId) -> SendStream<'_> { + self.connection.send_stream(id) + } + + /// Number of streams that may have unacknowledged data. + pub fn send_stream_count(&mut self) -> usize { + self.connection.streams().send_streams() + } + /// Closes the given substream. /// /// `write_substream` must no longer be called. The substream is however still @@ -202,7 +209,7 @@ impl Connection { } Poll::Ready(None) => { if closed.is_none() { - return Poll::Ready(ConnectionEvent::ConnectionLost(Error::ClosedChannel)); + return Poll::Ready(ConnectionEvent::ConnectionLost(Error::TaskCrashed)); } } Poll::Pending => {} @@ -229,10 +236,14 @@ impl Connection { } } + // The maximum amount of segments which can be transmitted in a single Transmit + // if a platform supports Generic Send Offload (GSO). + // Set to 1 for now since not all platforms support GSO. + // TODO: Fix for platforms that support GSO. + let max_datagrams = 1; // Poll the connection for packets to send on the UDP socket and try to send them on // `to_endpoint`. - // FIXME max_datagrams - if let Some(transmit) = self.connection.poll_transmit(Instant::now(), 1) { + if let Some(transmit) = self.connection.poll_transmit(Instant::now(), max_datagrams) { // TODO: ECN bits not handled self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket { destination: transmit.destination, diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 2592ddfbae5..d888d25f33a 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -282,7 +282,6 @@ pub enum ToEndpoint { /// Keep in mind that we pass an `Endpoint` whenever we create a new connection, which /// guarantees that the [`Endpoint`], and therefore the background task, is properly kept alive /// for as long as any QUIC connection is open. -/// async fn background_task( endpoint_config: Arc, client_config: quinn_proto::ClientConfig, @@ -308,6 +307,7 @@ async fn background_task( // Next packet waiting to be transmitted on the UDP socket, if any. let mut next_packet_out: Option<(SocketAddr, Vec)> = None; + // Whether the transport dropped its handle for this endpoint. let mut is_orphaned = false; // Main loop of the task. @@ -391,7 +391,12 @@ async fn background_task( if let Some(event_back) = event_back { debug_assert!(!is_drained_event); if let Some(sender) = alive_connections.get_mut(&connection_id) { - let _ = sender.send(event_back).await; // TODO: don't await here /!\ + // We clone the sender to guarantee that there will be at least one + // free slot to send the event. + // The channel can not grow out of bound because an `event_back` is + // only sent if we prior received an event from the same connection. + // If the connection is busy, it won't sent us any more events to handle. + let _ = sender.clone().start_send(event_back); } else { tracing::error!("State mismatch: event for closed connection"); } @@ -427,9 +432,12 @@ async fn background_task( match event { None => {}, Some((connec_id, quinn_proto::DatagramEvent::ConnectionEvent(event))) => { - // Event to send to an existing connection. + // Redirect the datagram to its connection. if let Some(sender) = alive_connections.get_mut(&connec_id) { - let _ = sender.send(event).await; // TODO: don't await here /!\ + // Try to send the redirected datagramm event to the connection. + // If the connection is too busy we drop the datagram to back-pressure + // the remote. + let _ = sender.try_send(event); } else { tracing::error!("State mismatch: event for closed connection"); } diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index f9b5174dad9..312950ab131 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -200,11 +200,11 @@ impl StreamMuxer for QuicMuxer { connection, .. } = &mut *self.inner.lock(); - if connection.connection.is_drained() { + if connection.is_drained() { return Poll::Ready(Ok(())); } - if connection.connection.streams().send_streams() != 0 { + if connection.send_stream_count() != 0 { for substream in substreams.keys() { if let Err(e) = connection.finish_substream(*substream) { tracing::warn!("substream finish error on muxer close: {}", e); @@ -212,9 +212,7 @@ impl StreamMuxer for QuicMuxer { } } loop { - if connection.connection.streams().send_streams() == 0 - && !connection.connection.is_closed() - { + if connection.send_stream_count() == 0 && !connection.is_closed() { connection.close() } match connection.poll_event(cx) { @@ -247,7 +245,7 @@ impl AsyncRead for Substream { use quinn_proto::{ReadError, ReadableError}; let mut muxer = self.muxer.lock(); - let mut stream = muxer.connection.connection.recv_stream(self.id); + let mut stream = muxer.connection.recv_stream(self.id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(ReadableError::UnknownStream) => { @@ -306,7 +304,7 @@ impl AsyncWrite for Substream { ) -> Poll> { let mut muxer = self.muxer.lock(); - match muxer.connection.connection.send_stream(self.id).write(buf) { + match muxer.connection.send_stream(self.id).write(buf) { Ok(bytes) => Poll::Ready(Ok(bytes)), Err(quinn_proto::WriteError::Blocked) => { let substream = muxer @@ -353,12 +351,8 @@ impl Drop for Substream { fn drop(&mut self) { let mut muxer = self.muxer.lock(); muxer.substreams.remove(&self.id); - let _ = muxer - .connection - .connection - .recv_stream(self.id) - .stop(0u32.into()); - let mut send_stream = muxer.connection.connection.send_stream(self.id); + let _ = muxer.connection.recv_stream(self.id).stop(0u32.into()); + let mut send_stream = muxer.connection.send_stream(self.id); match send_stream.finish() { Ok(()) => {} // Already finished or reset, which is fine. diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 3f324be06b4..8d157979b58 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -399,10 +399,7 @@ impl Stream for Listener { } match self.new_connections_rx.poll_next_unpin(cx) { Poll::Ready(Some(connection)) => { - let local_addr = connection - .local_addr() - .expect("exists for server connections."); - let local_addr = socketaddr_to_multiaddr(&local_addr); + let local_addr = socketaddr_to_multiaddr(connection.local_addr()); let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); let event = TransportEvent::Incoming { upgrade: Upgrade::from_connection(connection), @@ -423,6 +420,7 @@ impl Stream for Listener { } } } + #[async_trait::async_trait] pub trait Provider: Unpin + Send + 'static { // Wrapped socket for non-blocking I/O operations. From 6dfe97af85f55a6afb1002742fbb6a80a2095818 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 28 Sep 2022 16:46:11 +0200 Subject: [PATCH 107/218] transports/quic: introduce EndpointDriver Refactor the `background_task` function into a new struct `EndpointDriver`, which implements `Future`. Split loop into smaller functions. Introduce `ReceiveStream` wrapper to implement `Stream` for the udp socket. --- transports/quic/src/connection.rs | 21 +- transports/quic/src/endpoint.rs | 553 ++++++++++++++++++------------ transports/quic/src/transport.rs | 38 +- 3 files changed, 365 insertions(+), 247 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index d1e541d80db..4b9f333d26c 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -26,7 +26,7 @@ //! All interactions with a QUIC connection should be done through this struct. // TODO: docs -use crate::endpoint::{Endpoint, ToEndpoint}; +use crate::endpoint::{EndpointChannel, ToEndpoint}; use futures::{channel::mpsc, prelude::*}; use futures_timer::Delay; @@ -42,10 +42,10 @@ use std::{ /// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Upgrade`]. /// /// Contains everything needed to process a connection with a remote. -/// Tied to a specific [`Endpoint`]. +/// Tied to a specific endpoint. pub struct Connection { - /// Endpoint this connection belongs to. - endpoint: Endpoint, + /// Channel to the endpoint this connection belongs to. + endpoint_channel: EndpointChannel, /// Pending message to be sent to the background task that is driving the endpoint. pending_to_endpoint: Option, /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. @@ -84,14 +84,14 @@ impl Connection { /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of /// its methods has ever been called. Failure to comply might lead to logic errors and panics. pub fn from_quinn_connection( - endpoint: Endpoint, + endpoint_channel: EndpointChannel, connection: quinn_proto::Connection, connection_id: quinn_proto::ConnectionHandle, from_endpoint: mpsc::Receiver, ) -> Self { debug_assert!(!connection.is_closed()); Connection { - endpoint, + endpoint_channel, pending_to_endpoint: None, connection, next_timeout: None, @@ -102,7 +102,7 @@ impl Connection { /// The address that the local socket is bound to. pub fn local_addr(&self) -> &SocketAddr { - self.endpoint.socket_addr() + self.endpoint_channel.socket_addr() } /// Returns the address of the node we're connected to. @@ -224,7 +224,7 @@ impl Connection { // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. if let Some(to_endpoint) = self.pending_to_endpoint.take() { - match self.endpoint.try_send(to_endpoint, cx) { + match self.endpoint_channel.try_send(to_endpoint, cx) { Ok(Ok(())) => {} Ok(Err(to_endpoint)) => { self.pending_to_endpoint = Some(to_endpoint); @@ -245,10 +245,7 @@ impl Connection { // `to_endpoint`. if let Some(transmit) = self.connection.poll_transmit(Instant::now(), max_datagrams) { // TODO: ECN bits not handled - self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket { - destination: transmit.destination, - data: transmit.contents, - }); + self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket(transmit)); continue; } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index d888d25f33a..b812d7e20db 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -23,10 +23,10 @@ //! Considering that all QUIC communications happen over a single UDP socket, one needs to //! maintain a unique synchronization point that holds the state of all the active connections. //! -//! The [`Endpoint`] object represents this synchronization point. It maintains a background task +//! The endpoint represents this synchronization point. It is maintained in a background task //! whose role is to interface with the UDP socket. Communication between the background task and //! the rest of the code only happens through channels. See the documentation of the -//! [`background_task`] for a thorough description. +//! [`EndpointDriver`] for a thorough description. use crate::{ connection::Connection, @@ -35,20 +35,22 @@ use crate::{ }; use futures::{ - channel::{ - mpsc::{self, SendError}, - oneshot, - }, + channel::{mpsc, oneshot}, prelude::*, + ready, }; -use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig}; +use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig, Transmit}; use std::{ collections::HashMap, + io, net::{Ipv4Addr, Ipv6Addr, SocketAddr}, + ops::ControlFlow, + pin::Pin, sync::Arc, task::{Context, Poll}, time::{Duration, Instant}, }; +use x509_parser::nom::AsBytes; /// Represents the configuration for the QUIC endpoint. #[derive(Debug, Clone)] @@ -92,7 +94,7 @@ impl Config { /// Object containing all the QUIC resources shared between all connections. #[derive(Debug, Clone)] -pub struct Endpoint { +pub struct EndpointChannel { /// Channel to the background of the endpoint. to_endpoint: mpsc::Sender, /// Address that the socket is bound to. @@ -100,22 +102,22 @@ pub struct Endpoint { socket_addr: SocketAddr, } -impl Endpoint { - /// Builds a new [`Endpoint`] that is listening on the [`SocketAddr`]. +impl EndpointChannel { + /// Builds a new endpoint that is listening on the [`SocketAddr`]. pub fn new_bidirectional( config: Config, socket_addr: SocketAddr, - ) -> Result<(Endpoint, mpsc::Receiver), transport::Error> { + ) -> Result<(EndpointChannel, mpsc::Receiver), transport::Error> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Self::new::

(config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) } - /// Builds a new [`Endpoint`] that only supports outbound connections. + /// Builds a new endpoint that only supports outbound connections. pub fn new_dialer( config: Config, is_ipv6: bool, - ) -> Result { + ) -> Result { let socket_addr = if is_ipv6 { SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0) } else { @@ -128,31 +130,33 @@ impl Endpoint { config: Config, socket_addr: SocketAddr, new_connections: Option>, - ) -> Result { + ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; socket.set_nonblocking(true)?; let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(32); - let endpoint = Endpoint { + let channel = EndpointChannel { to_endpoint: to_endpoint_tx, socket_addr: socket.local_addr()?, }; - let server_config = new_connections.map(|c| (c, config.server_config.clone())); - + let server_config = new_connections.is_some().then(|| config.server_config); let socket = P::from_socket(socket)?; - P::spawn(background_task::

( + let driver = EndpointDriver::

::new( config.endpoint_config, config.client_config, + new_connections, server_config, - endpoint.clone(), + channel.clone(), socket, - to_endpoint_rx.fuse(), - )); + to_endpoint_rx, + ); - Ok(endpoint) + P::spawn(driver); + + Ok(channel) } pub fn socket_addr(&self) -> &SocketAddr { @@ -170,7 +174,7 @@ impl Endpoint { &mut self, to_endpoint: ToEndpoint, cx: &mut Context<'_>, - ) -> Result, SendError> { + ) -> Result, mpsc::SendError> { match self.to_endpoint.poll_ready_unpin(cx) { Poll::Ready(Ok(())) => {} Poll::Ready(Err(err)) => return Err(err), @@ -198,18 +202,13 @@ pub enum ToEndpoint { event: quinn_proto::EndpointEvent, }, /// Instruct the endpoint to send a packet of data on its UDP socket. - SendUdpPacket { - /// Destination of the UDP packet. - destination: SocketAddr, - /// Packet of data to send. - data: Vec, - }, + SendUdpPacket(quinn_proto::Transmit), } -/// Task that runs in the background for as long as the endpoint is alive. Responsible for +/// Driver that runs in the background for as long as the endpoint is alive. Responsible for /// processing messages and the UDP socket. /// -/// The `receiver` parameter must be the receiving side of the `Endpoint::to_endpoint` sender. +/// The `receiver` parameter must be the receiving side of the `EndpointChannel::to_endpoint` sender. /// /// # Behaviour /// @@ -227,11 +226,11 @@ pub enum ToEndpoint { /// When it comes to channels, there exists three main multi-producer-single-consumer channels /// in play: /// -/// - One channel, represented by `Endpoint::to_endpoint` and `receiver`, that communicates -/// messages from [`Endpoint`] to the background task. -/// - One channel per each existing connection that communicates messages from the background -/// task to that [`Connection`]. -/// - One channel for the background task to send newly-opened connections to. The receiving +/// - One channel, represented by `EndpointChannel::to_endpoint` and `receiver`, that communicates +/// messages from [`EndpointChannel`] to the [`EndpointDriver`]. +/// - One channel per each existing connection that communicates messages from the [`EndpointDriver`] +/// to that [`Connection`]. +/// - One channel for the [`EndpointDriver`] to send newly-opened connections to. The receiving /// side is processed by the [`QuicTransport`][crate::QuicTransport]. /// /// In order to avoid an unbounded buffering of events, we prioritize sending data on the UDP @@ -247,10 +246,10 @@ pub enum ToEndpoint { /// /// The [`quinn_proto::Endpoint`] object contains an accept buffer, in other words a buffer of the /// incoming connections waiting to be accepted. When a new connection is signalled, we send this -/// new connection on the `new_connections` channel in an asynchronous way, and we only free a slot -/// in the accept buffer once the element has actually been enqueued on `new_connections`. There -/// are therefore in total three buffers in play: the `new_connections` channel itself, the queue -/// of elements being sent on `new_connections`, and the accept buffer of the +/// new connection on the `new_connection_tx` channel in an asynchronous way, and we only free a slot +/// in the accept buffer once the element has actually been enqueued on `new_connection_tx`. There +/// are therefore in total three buffers in play: the `new_connection_tx` channel itself, the queue +/// of elements being sent on `new_connection_tx`, and the accept buffer of the /// [`quinn_proto::Endpoint`]. /// /// ## Back-pressure on connections @@ -277,208 +276,324 @@ pub enum ToEndpoint { /// # Shutdown /// /// The background task shuts down if `endpoint_weak`, `receiver` or `new_connections` become -/// disconnected/invalid. This corresponds to the lifetime of the associated [`Endpoint`]. +/// disconnected/invalid. This corresponds to the lifetime of the associated [`quinn_proto::Endpoint`]. /// -/// Keep in mind that we pass an `Endpoint` whenever we create a new connection, which -/// guarantees that the [`Endpoint`], and therefore the background task, is properly kept alive -/// for as long as any QUIC connection is open. -async fn background_task( - endpoint_config: Arc, - client_config: quinn_proto::ClientConfig, - server_config: Option<(mpsc::Sender, Arc)>, - endpoint: Endpoint, - udp_socket: P::Socket, - mut receiver: stream::Fuse>, -) { - let (mut new_connections, server_config) = match server_config { - Some((a, b)) => (Some(a), Some(b)), - None => (None, None), - }; - +/// Keep in mind that we pass an `EndpointChannel` whenever we create a new connection, which +/// guarantees that the [`EndpointDriver`], is properly kept alive for as long as any QUIC +/// connection is open. +/// +struct EndpointDriver { // The actual QUIC state machine. - let mut proto_endpoint = quinn_proto::Endpoint::new(endpoint_config.clone(), server_config); + endpoint: quinn_proto::Endpoint, + // Config for client connections. + client_config: quinn_proto::ClientConfig, + // Copy of the channel to the endpoint driver that is passed to each new connection. + channel: EndpointChannel, + // Channel to receive messages from the transport or connections. + rx: mpsc::Receiver, + + // Socket for sending and receiving datagrams. + socket: Arc, + // Future for writing the next packet to the socket. + next_packet_out: Option<( + Pin> + Send>>, + usize, + )>, + // Stream of inbound datagrams. + receive_stream: ReceiveStream

, // List of all active connections, with a sender to notify them of events. - let mut alive_connections = HashMap::>::new(); + alive_connections: + HashMap>, + // Channel to forward new inbound connections to the transport. + // `None` if server capabilities are disabled, i.e. the endpoint is only used for dialing. + new_connection_tx: Option>, + // Whether the transport dropped its handle for this endpoint. + is_orphaned: bool, +} - // Buffer where we write packets received from the UDP socket. - let mut socket_recv_buffer = vec![0; 65536]; +impl EndpointDriver

{ + fn new( + endpoint_config: Arc, + client_config: quinn_proto::ClientConfig, + new_connection_tx: Option>, + server_config: Option>, + channel: EndpointChannel, + socket: P::Socket, + rx: mpsc::Receiver, + ) -> Self { + let socket = Arc::new(socket); + EndpointDriver { + endpoint: quinn_proto::Endpoint::new(endpoint_config, server_config), + client_config, + channel, + rx, + socket: socket.clone(), + next_packet_out: None, + receive_stream: ReceiveStream::new(socket), + alive_connections: HashMap::new(), + new_connection_tx, + is_orphaned: false, + } + } - // Next packet waiting to be transmitted on the UDP socket, if any. - let mut next_packet_out: Option<(SocketAddr, Vec)> = None; + /// Insert future to send a datagram on the socket. + fn send_packet_out(&mut self, transmit: Transmit) { + let len = transmit.contents.len(); + let socket = self.socket.clone(); + let send = + async move { P::send_to(&socket, &transmit.contents, transmit.destination).await } + .boxed(); + self.next_packet_out = Some((send, len)); + } - // Whether the transport dropped its handle for this endpoint. - let mut is_orphaned = false; - - // Main loop of the task. - loop { - // Start by flushing `next_packet_out`. - if let Some((destination, data)) = next_packet_out.take() { - // We block the current task until the packet is sent. This way, if the - // network interface is too busy, we back-pressure all of our internal - // channels. - // TODO: set ECN bits; there is no support for them in the ecosystem right now - match P::send_to(&udp_socket, &data, destination).await { - Ok(n) if n == data.len() => {} - Ok(_) => tracing::error!( - "QUIC UDP socket violated expectation that packets are always fully \ - transferred" - ), - - // Errors on the socket are expected to never happen, and we handle them by simply - // printing a log message. The packet gets discarded in case of error, but we are - // robust to packet losses and it is consequently not a logic error to process with - // normal operations. - Err(err) => tracing::error!("Error while sending on QUIC UDP socket: {:?}", err), + /// Handle a message sent from either the [`QuicTransport`](super::QuicTransport) or a [`Connection`]. + fn handle_message(&mut self, to_endpoint: ToEndpoint) -> ControlFlow<()> { + match to_endpoint { + ToEndpoint::Dial { addr, result } => { + // This `"l"` seems necessary because an empty string is an invalid domain + // name. While we don't use domain names, the underlying rustls library + // is based upon the assumption that we do. + let (connection_id, connection) = + match self.endpoint.connect(self.client_config.clone(), addr, "l") { + Ok(c) => c, + Err(err) => { + let _ = result.send(Err(err)); + return ControlFlow::Continue(()); + } + }; + + debug_assert_eq!(connection.side(), quinn_proto::Side::Client); + let (tx, rx) = mpsc::channel(16); + let connection = Connection::from_quinn_connection( + self.channel.clone(), + connection, + connection_id, + rx, + ); + self.alive_connections.insert(connection_id, tx); + let _ = result.send(Ok(connection)); } - } - // The endpoint might request packets to be sent out. This is handled in priority to avoid - // buffering up packets. - if let Some(packet) = proto_endpoint.poll_transmit() { - next_packet_out = Some((packet.destination, packet.contents)); - continue; - } + // A connection wants to notify the endpoint of something. + ToEndpoint::ProcessConnectionEvent { + connection_id, + event, + } => { + let has_key = self.alive_connections.contains_key(&connection_id); + if !has_key { + return ControlFlow::Continue(()); + } + // We "drained" event indicates that the connection no longer exists and + // its ID can be reclaimed. + let is_drained_event = event.is_drained(); + if is_drained_event { + self.alive_connections.remove(&connection_id); + if self.is_orphaned && self.alive_connections.is_empty() { + tracing::info!( + "Listener closed and no active connections remain. Shutting down the background task." + ); + return ControlFlow::Break(()); + } + } - futures::select! { - message = receiver.next() => { - // Received a message from a different part of the code requesting us to - // do something. - match message { - None => unreachable!("Sender side is never dropped or closed."), - Some(ToEndpoint::Dial { addr, result }) => { - // This `"l"` seems necessary because an empty string is an invalid domain - // name. While we don't use domain names, the underlying rustls library - // is based upon the assumption that we do. - let (connection_id, connection) = - match proto_endpoint.connect(client_config.clone(), addr, "l") { - Ok(c) => c, - Err(err) => { - let _ = result.send(Err(err)); - continue; - } - }; - - debug_assert_eq!(connection.side(), quinn_proto::Side::Client); - let (tx, rx) = mpsc::channel(16); - let connection = Connection::from_quinn_connection(endpoint.clone(), connection, connection_id, rx); - alive_connections.insert(connection_id, tx); - let _ = result.send(Ok(connection)); + let event_back = self.endpoint.handle_event(connection_id, event); + + if let Some(event_back) = event_back { + debug_assert!(!is_drained_event); + if let Some(sender) = self.alive_connections.get_mut(&connection_id) { + // We clone the sender to guarantee that there will be at least one + // free slot to send the event. + // The channel can not grow out of bound because an `event_back` is + // only sent if we prior received an event from the same connection. + // If the connection is busy, it won't sent us any more events to handle. + let _ = sender.clone().start_send(event_back); + } else { + tracing::error!("State mismatch: event for closed connection"); } + } + } - // A connection wants to notify the endpoint of something. - Some(ToEndpoint::ProcessConnectionEvent { connection_id, event }) => { - let has_key = alive_connections.contains_key(&connection_id); - if !has_key { - continue; - } - // We "drained" event indicates that the connection no longer exists and - // its ID can be reclaimed. - let is_drained_event = event.is_drained(); - if is_drained_event { - alive_connections.remove(&connection_id); - if is_orphaned && alive_connections.is_empty() { - tracing::info!( - "Listener closed and no active connections remain. Shutting down the background task." - ); - return; - } + // Data needs to be sent on the UDP socket. + ToEndpoint::SendUdpPacket(transmit) => self.send_packet_out(transmit), + } + ControlFlow::Continue(()) + } - } + /// Handle datagram received on the socket. + /// The datagram content was written into the `socket_recv_buffer`. + fn handle_datagram(&mut self, bytes: Vec, packet_src: SocketAddr) -> ControlFlow<()> { + let packet = From::from(bytes.as_bytes()); + let local_ip = self.channel.socket_addr.ip(); + // TODO: ECN bits aren't handled + let (connec_id, event) = + match self + .endpoint + .handle(Instant::now(), packet_src, Some(local_ip), None, packet) + { + Some(event) => event, + None => return ControlFlow::Continue(()), + }; + match event { + quinn_proto::DatagramEvent::ConnectionEvent(event) => { + // Redirect the datagram to its connection. + if let Some(sender) = self.alive_connections.get_mut(&connec_id) { + // Try to send the redirected datagramm event to the connection. + // If the connection is too busy we drop the datagram to back-pressure + // the remote. + let _ = sender.try_send(event); + } else { + tracing::error!("State mismatch: event for closed connection"); + } + } + quinn_proto::DatagramEvent::NewConnection(connec) => { + // A new connection has been received. `connec_id` is a newly-allocated + // identifier. + debug_assert_eq!(connec.side(), quinn_proto::Side::Server); + let connection_tx = match self.new_connection_tx.as_mut() { + Some(tx) => tx, + None => { + tracing::warn!( + "Endpoint reported a new connection even though server capabilities are disabled." + ); + return ControlFlow::Continue(()); + } + }; - let event_back = proto_endpoint.handle_event(connection_id, event); - - if let Some(event_back) = event_back { - debug_assert!(!is_drained_event); - if let Some(sender) = alive_connections.get_mut(&connection_id) { - // We clone the sender to guarantee that there will be at least one - // free slot to send the event. - // The channel can not grow out of bound because an `event_back` is - // only sent if we prior received an event from the same connection. - // If the connection is busy, it won't sent us any more events to handle. - let _ = sender.clone().start_send(event_back); - } else { - tracing::error!("State mismatch: event for closed connection"); - } + let (tx, rx) = mpsc::channel(16); + let connection = + Connection::from_quinn_connection(self.channel.clone(), connec, connec_id, rx); + match connection_tx.try_send(connection) { + Ok(()) => { + self.alive_connections.insert(connec_id, tx); + } + Err(e) if e.is_disconnected() => { + if self.alive_connections.is_empty() { + return ControlFlow::Break(()); } + // Listener was closed. + self.endpoint.reject_new_connections(); + self.new_connection_tx = None; + self.is_orphaned = true; } + Err(_) => tracing::warn!( + "Dropping new incoming connection {:?} because the channel to the listener is full", + connec_id + ) + } + } + } + ControlFlow::Continue(()) + } +} - // Data needs to be sent on the UDP socket. - Some(ToEndpoint::SendUdpPacket { destination, data }) => { - debug_assert!(next_packet_out.is_none()); - next_packet_out = Some((destination, data)); - continue; +impl Future for EndpointDriver

{ + type Output = (); + fn poll(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + loop { + if let Some((send_packet, len)) = self.next_packet_out.as_mut() { + match ready!(send_packet.poll_unpin(cx)) { + Ok(n) if n == *len => {} + Ok(_) => tracing::error!( + "QUIC UDP socket violated expectation that packets are always fully \ + transferred" + ), + // Errors on the socket are expected to never happen, and we handle them by simply + // printing a log message. The packet gets discarded in case of error, but we are + // robust to packet losses and it is consequently not a logic error to process with + // normal operations. + Err(err) => { + tracing::error!("Error while sending on QUIC UDP socket: {:?}", err) } } + self.next_packet_out = None; } - result = P::recv_from(&udp_socket, &mut socket_recv_buffer).fuse() => { - let (packet_len, packet_src) = match result { - Ok(v) => v, - // Errors on the socket are expected to never happen, and we handle them by - // simply printing a log message. - Err(err) => { - tracing::error!("Error while receive on QUIC UDP socket: {:?}", err); - continue; - }, - }; - // Received a UDP packet from the socket. - debug_assert!(packet_len <= socket_recv_buffer.len()); - let packet = From::from(&socket_recv_buffer[..packet_len]); - let local_ip = endpoint.socket_addr.ip(); - // TODO: ECN bits aren't handled - let event = proto_endpoint.handle(Instant::now(), packet_src, Some(local_ip), None, packet); - - match event { - None => {}, - Some((connec_id, quinn_proto::DatagramEvent::ConnectionEvent(event))) => { - // Redirect the datagram to its connection. - if let Some(sender) = alive_connections.get_mut(&connec_id) { - // Try to send the redirected datagramm event to the connection. - // If the connection is too busy we drop the datagram to back-pressure - // the remote. - let _ = sender.try_send(event); - } else { - tracing::error!("State mismatch: event for closed connection"); - } - }, - Some((connec_id, quinn_proto::DatagramEvent::NewConnection(connec))) => { - // A new connection has been received. `connec_id` is a newly-allocated - // identifier. - debug_assert_eq!(connec.side(), quinn_proto::Side::Server); - let connection_tx = match new_connections.as_mut() { - Some(tx) => tx, - None => { - tracing::warn!( - "Endpoint reported a new connection even though server capabilities are disabled." - ); - continue - } - }; - - let (tx, rx) = mpsc::channel(16); - let connection = Connection::from_quinn_connection(endpoint.clone(), connec, connec_id, rx); - match connection_tx.try_send(connection) { - Ok(()) => { - alive_connections.insert(connec_id, tx); - } - Err(e) if e.is_disconnected() => { - // Listener was closed. - proto_endpoint.reject_new_connections(); - new_connections = None; - is_orphaned = true; - if alive_connections.is_empty() { - return; - } - } - Err(_) => tracing::warn!( - "Dropping new incoming connection {:?} because the channel to the listener is full", - connec_id - ) - } - }, + // The endpoint might request packets to be sent out. This is handled in priority to avoid + // buffering up packets. + if let Some(transmit) = self.endpoint.poll_transmit() { + self.send_packet_out(transmit); + continue; + } + + match self.rx.poll_next_unpin(cx) { + Poll::Ready(Some(to_endpoint)) => match self.handle_message(to_endpoint) { + ControlFlow::Continue(()) => continue, + ControlFlow::Break(()) => break, + }, + Poll::Ready(None) => { + unreachable!("Sender side is never dropped or closed.") } + Poll::Pending => {} } + + match self.receive_stream.poll_next_unpin(cx) { + Poll::Ready(Some(Ok((bytes, packet_src)))) => { + match self.handle_datagram(bytes, packet_src) { + ControlFlow::Continue(()) => continue, + ControlFlow::Break(()) => break, + } + } + // Errors on the socket are expected to never happen, and we handle them by + // simply printing a log message. + Poll::Ready(Some(Err(err))) => { + tracing::error!("Error while receive on QUIC UDP socket: {:?}", err); + continue; + } + Poll::Ready(None) => { + unreachable!("ReceiveStream::poll_next never returns Poll::Ready(None)") + } + Poll::Pending => {} + } + return Poll::Pending; + } + + Poll::Ready(()) + } +} + +/// Wrapper around the socket to implement `Stream` on it socket. +/// This is needed since not all [`Provider`]s provide a poll-based receive method for their socket. +struct ReceiveStream { + fut: Pin< + Box< + dyn Future< + Output = ( + Result<(usize, SocketAddr), io::Error>, + Arc, + Vec, + ), + > + Send, + >, + >, +} + +impl ReceiveStream

{ + fn new(socket: Arc) -> Self { + let mut socket_recv_buffer = vec![0; 65536]; + let fut = async move { + let recv = P::recv_from(&socket, &mut socket_recv_buffer).await; + (recv, socket, socket_recv_buffer) + }; + Self { fut: fut.boxed() } + } +} + +impl Stream for ReceiveStream

{ + type Item = Result<(Vec, SocketAddr), io::Error>; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let (result, socket, mut buffer) = ready!(self.fut.poll_unpin(cx)); + let result = result.map(|(packet_len, packet_src)| { + debug_assert!(packet_len <= buffer.len()); + // Copies the bytes from the `socket_recv_buffer` they were written into. + (buffer[..packet_len].into(), packet_src) + }); + self.fut = async move { + let recv = P::recv_from(&socket, &mut buffer).await; + (recv, socket, buffer) } + .boxed(); + Poll::Ready(Some(result)) } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 8d157979b58..1ddd4d3c873 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -25,7 +25,7 @@ use crate::connection::Connection; use crate::endpoint::ToEndpoint; use crate::Config; -use crate::{endpoint::Endpoint, muxer::QuicMuxer, upgrade::Upgrade}; +use crate::{endpoint::EndpointChannel, muxer::QuicMuxer, upgrade::Upgrade}; #[cfg(feature = "async-std")] mod async_std; @@ -154,7 +154,7 @@ impl Transport for QuicTransport

{ .listeners .iter_mut() .filter(|l| { - let listen_addr = l.endpoint.socket_addr(); + let listen_addr = l.endpoint_channel.socket_addr(); listen_addr.is_ipv4() == socket_addr.is_ipv4() && listen_addr.ip().is_loopback() == socket_addr.ip().is_loopback() }) @@ -240,22 +240,23 @@ impl Transport for QuicTransport

{ #[derive(Debug)] struct Dialer { - endpoint: Endpoint, + endpoint_channel: EndpointChannel, pending_dials: VecDeque, } impl Dialer { fn new(config: Config, is_ipv6: bool) -> Result> { - let endpoint = Endpoint::new_dialer::

(config, is_ipv6).map_err(TransportError::Other)?; + let endpoint_channel = + EndpointChannel::new_dialer::

(config, is_ipv6).map_err(TransportError::Other)?; Ok(Dialer { - endpoint, + endpoint_channel, pending_dials: VecDeque::new(), }) } fn drive_dials(&mut self, cx: &mut Context<'_>) -> Result<(), mpsc::SendError> { if let Some(to_endpoint) = self.pending_dials.pop_front() { - match self.endpoint.try_send(to_endpoint, cx) { + match self.endpoint_channel.try_send(to_endpoint, cx) { Ok(Ok(())) => {} Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), Err(err) => { @@ -269,7 +270,7 @@ impl Dialer { #[derive(Debug)] struct Listener { - endpoint: Endpoint, + endpoint_channel: EndpointChannel, listener_id: ListenerId, @@ -295,7 +296,8 @@ impl Listener { socket_addr: SocketAddr, config: Config, ) -> Result { - let (endpoint, new_connections_rx) = Endpoint::new_bidirectional::

(config, socket_addr)?; + let (endpoint_channel, new_connections_rx) = + EndpointChannel::new_bidirectional::

(config, socket_addr)?; let if_watcher; let pending_event; @@ -304,7 +306,7 @@ impl Listener { pending_event = None; } else { if_watcher = None; - let ma = socketaddr_to_multiaddr(endpoint.socket_addr()); + let ma = socketaddr_to_multiaddr(endpoint_channel.socket_addr()); pending_event = Some(TransportEvent::NewAddress { listener_id, listen_addr: ma, @@ -312,7 +314,7 @@ impl Listener { } Ok(Listener { - endpoint, + endpoint_channel, listener_id, new_connections_rx, if_watcher, @@ -345,7 +347,9 @@ impl Listener { loop { match ready!(if_watcher.poll_if_event(cx)) { Ok(IfEvent::Up(inet)) => { - if let Some(listen_addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { + if let Some(listen_addr) = + ip_to_listenaddr(self.endpoint_channel.socket_addr(), inet.addr()) + { tracing::debug!("New listen address: {}", listen_addr); return Poll::Ready(TransportEvent::NewAddress { listener_id: self.listener_id, @@ -354,7 +358,9 @@ impl Listener { } } Ok(IfEvent::Down(inet)) => { - if let Some(listen_addr) = ip_to_listenaddr(&self.endpoint, inet.addr()) { + if let Some(listen_addr) = + ip_to_listenaddr(self.endpoint_channel.socket_addr(), inet.addr()) + { tracing::debug!("Expired listen address: {}", listen_addr); return Poll::Ready(TransportEvent::AddressExpired { listener_id: self.listener_id, @@ -388,7 +394,7 @@ impl Stream for Listener { Poll::Pending => {} } if let Some(to_endpoint) = self.pending_dials.pop_front() { - match self.endpoint.try_send(to_endpoint, cx) { + match self.endpoint_channel.try_send(to_endpoint, cx) { Ok(Ok(())) => {} Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), Err(_) => { @@ -446,13 +452,13 @@ pub trait Provider: Unpin + Send + 'static { /// /// Returns `None` if the address is not the same socket family as the /// address that the endpoint is bound to. -fn ip_to_listenaddr(endpoint: &Endpoint, ip: IpAddr) -> Option { +fn ip_to_listenaddr(endpoint_addr: &SocketAddr, ip: IpAddr) -> Option { // True if either both addresses are Ipv4 or both Ipv6. - let is_same_ip_family = endpoint.socket_addr().is_ipv4() == ip.is_ipv4(); + let is_same_ip_family = endpoint_addr.is_ipv4() == ip.is_ipv4(); if !is_same_ip_family { return None; } - let socket_addr = SocketAddr::new(ip, endpoint.socket_addr().port()); + let socket_addr = SocketAddr::new(ip, endpoint_addr.port()); Some(socketaddr_to_multiaddr(&socket_addr)) } From 43dbf456a4a00f8faa3efbe5859007fc5d9808f2 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 28 Sep 2022 18:43:15 +0200 Subject: [PATCH 108/218] transports/quic: clean error types --- transports/quic/src/connection.rs | 33 ++++++------ transports/quic/src/endpoint.rs | 13 +++-- transports/quic/src/error.rs | 90 ------------------------------- transports/quic/src/lib.rs | 9 ++-- transports/quic/src/muxer.rs | 29 +++++----- transports/quic/src/transport.rs | 65 +++++++++++----------- transports/quic/src/upgrade.rs | 4 +- 7 files changed, 74 insertions(+), 169 deletions(-) delete mode 100644 transports/quic/src/error.rs diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 4b9f333d26c..c5b388bd42b 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -31,7 +31,6 @@ use crate::endpoint::{EndpointChannel, ToEndpoint}; use futures::{channel::mpsc, prelude::*}; use futures_timer::Delay; use libp2p_core::PeerId; -use quinn_proto::{RecvStream, SendStream}; use std::{ fmt, net::SocketAddr, @@ -63,10 +62,10 @@ pub struct Connection { /// Error on the connection as a whole. #[derive(Debug, Clone, thiserror::Error)] -pub enum Error { - /// The background task driving the endpoint has crashed. - #[error("Background task crashed")] - TaskCrashed, +pub enum ConnectionError { + /// The [`EndpointDriver`] has crashed. + #[error("Endpoint driver crashed")] + EndpointDriverCrashed, /// Error in the inner state machine. #[error("{0}")] Quinn(#[from] quinn_proto::ConnectionError), @@ -75,8 +74,8 @@ pub enum Error { impl Connection { /// Crate-internal function that builds a [`Connection`] from raw components. /// - /// This function assumes that there exists a background task that will process the messages - /// sent to `Endpoint::to_endpoint` and send us messages on `from_endpoint`. + /// This function assumes that there exists a [`EndpointDriver`] that will process the messages + /// sent to `EndpointChannel::to_endpoint` and send us messages on `from_endpoint`. /// /// `connection_id` is used to identify the local connection in the messages sent to /// `to_endpoint`. @@ -169,12 +168,12 @@ impl Connection { } /// Control over the stream for reading. - pub fn recv_stream(&mut self, id: quinn_proto::StreamId) -> RecvStream<'_> { + pub fn recv_stream(&mut self, id: quinn_proto::StreamId) -> quinn_proto::RecvStream<'_> { self.connection.recv_stream(id) } /// Control over the stream for writing. - pub fn send_stream(&mut self, id: quinn_proto::StreamId) -> SendStream<'_> { + pub fn send_stream(&mut self, id: quinn_proto::StreamId) -> quinn_proto::SendStream<'_> { self.connection.send_stream(id) } @@ -209,7 +208,9 @@ impl Connection { } Poll::Ready(None) => { if closed.is_none() { - return Poll::Ready(ConnectionEvent::ConnectionLost(Error::TaskCrashed)); + return Poll::Ready(ConnectionEvent::ConnectionLost( + ConnectionError::EndpointDriverCrashed, + )); } } Poll::Pending => {} @@ -231,7 +232,9 @@ impl Connection { return Poll::Pending; } Err(_) => { - return Poll::Ready(ConnectionEvent::ConnectionLost(Error::TaskCrashed)); + return Poll::Ready(ConnectionEvent::ConnectionLost( + ConnectionError::EndpointDriverCrashed, + )); } } } @@ -317,7 +320,7 @@ pub enum ConnectionEvent { Connected, /// Connection has been closed and can no longer be used. - ConnectionLost(Error), + ConnectionLost(ConnectionError), /// Generated after [`Connection::accept_substream`] has been called and has returned /// `None`. After this event has been generated, this method is guaranteed to return `Some`. @@ -360,9 +363,9 @@ impl TryFrom for ConnectionEvent { quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { dir: quinn_proto::Dir::Bi, }) => Ok(ConnectionEvent::StreamOpened), - quinn_proto::Event::ConnectionLost { reason } => { - Ok(ConnectionEvent::ConnectionLost(Error::Quinn(reason))) - } + quinn_proto::Event::ConnectionLost { reason } => Ok(ConnectionEvent::ConnectionLost( + ConnectionError::Quinn(reason), + )), quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { Ok(ConnectionEvent::StreamFinished(id)) } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b812d7e20db..cfe57cbefc0 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -39,7 +39,6 @@ use futures::{ prelude::*, ready, }; -use quinn_proto::{ClientConfig as QuinnClientConfig, ServerConfig as QuinnServerConfig, Transmit}; use std::{ collections::HashMap, io, @@ -75,14 +74,14 @@ impl Config { let client_tls_config = tls::make_client_config(keypair).unwrap(); let server_tls_config = tls::make_server_config(keypair).unwrap(); - let mut server_config = QuinnServerConfig::with_crypto(Arc::new(server_tls_config)); + let mut server_config = quinn_proto::ServerConfig::with_crypto(Arc::new(server_tls_config)); server_config.transport = Arc::clone(&transport); // Disables connection migration. // Long-term this should be enabled, however we then need to handle address change // on connections in the `QuicMuxer`. server_config.migration(false); - let mut client_config = QuinnClientConfig::new(Arc::new(client_tls_config)); + let mut client_config = quinn_proto::ClientConfig::new(Arc::new(client_tls_config)); client_config.transport = transport; Ok(Self { client_config, @@ -107,7 +106,7 @@ impl EndpointChannel { pub fn new_bidirectional( config: Config, socket_addr: SocketAddr, - ) -> Result<(EndpointChannel, mpsc::Receiver), transport::Error> { + ) -> Result<(EndpointChannel, mpsc::Receiver), transport::TransportError> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Self::new::

(config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) @@ -117,7 +116,7 @@ impl EndpointChannel { pub fn new_dialer( config: Config, is_ipv6: bool, - ) -> Result { + ) -> Result { let socket_addr = if is_ipv6 { SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0) } else { @@ -130,7 +129,7 @@ impl EndpointChannel { config: Config, socket_addr: SocketAddr, new_connections: Option>, - ) -> Result { + ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; socket.set_nonblocking(true)?; @@ -338,7 +337,7 @@ impl EndpointDriver

{ } /// Insert future to send a datagram on the socket. - fn send_packet_out(&mut self, transmit: Transmit) { + fn send_packet_out(&mut self, transmit: quinn_proto::Transmit) { let len = transmit.contents.len(); let socket = self.socket.clone(); let send = diff --git a/transports/quic/src/error.rs b/transports/quic/src/error.rs deleted file mode 100644 index 14755fe165c..00000000000 --- a/transports/quic/src/error.rs +++ /dev/null @@ -1,90 +0,0 @@ -// 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 futures::channel::mpsc::SendError; -use io::ErrorKind; -use std::io; -use thiserror::Error; - -/// An error that can be returned by libp2p-quic. -#[derive(Error, Debug)] -pub enum Error { - /// Fatal I/O error - #[error("Fatal I/O error {0}")] - IO(#[from] std::io::Error), - /// QUIC protocol error - #[error("QUIC protocol error: {0}")] - ConnectionError(#[from] quinn_proto::ConnectionError), - /// Peer stopped receiving data - #[error("Peer stopped receiving data: code {0}")] - Stopped(quinn_proto::VarInt), - /// Connection was prematurely closed - #[error("Connection was prematurely closed: {0}")] - ConnectionLost(crate::connection::Error), - /// Error making the connection. - #[error("Connection failure: {0}")] - ConnectError(#[from] quinn_proto::ConnectError), - /// Cannot listen on the same endpoint more than once - #[error("Cannot listen on the same endpoint more than once")] - AlreadyListening, - /// The stream was reset by the peer. - #[error("Peer reset stream: code {0}")] - Reset(quinn_proto::VarInt), - /// Either an attempt was made to write to a stream that was already shut down, - /// or a previous operation on this stream failed. - #[error( - "Use of a stream that has is no longer valid. This is a \ - bug in the application." - )] - ExpiredStream, - /// Reading from a stream that has not been written to. - #[error("Reading from a stream that has not been written to.")] - CannotReadFromUnwrittenStream, - /// Fatal internal error or network failure - #[error("Fatal internal error or network failure")] - NetworkFailure, - /// Connection already being closed - #[error("Connection already being closed")] - ConnectionClosing, -} - -impl From for Error { - fn from(_: SendError) -> Error { - Error::NetworkFailure - } -} - -impl From for io::Error { - fn from(e: Error) -> Self { - match e { - Error::IO(e) => io::Error::new(e.kind(), Error::IO(e)), - Error::ConnectionError(e) => e.into(), - e @ Error::NetworkFailure - | e @ Error::ConnectionClosing - | e @ Error::ConnectError(_) => io::Error::new(ErrorKind::Other, e), - e @ Error::Stopped(_) | e @ Error::Reset(_) | e @ Error::ConnectionLost(_) => { - io::Error::new(ErrorKind::ConnectionAborted, e) - } - e @ Error::ExpiredStream => io::Error::new(ErrorKind::BrokenPipe, e), - e @ Error::AlreadyListening => io::Error::new(ErrorKind::AddrInUse, e), - e @ Error::CannotReadFromUnwrittenStream => io::Error::new(ErrorKind::NotConnected, e), - } - } -} diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 4bb781eee39..070fffc3767 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -65,19 +65,18 @@ mod connection; mod endpoint; -mod error; mod muxer; mod tls; +mod transport; mod upgrade; -pub mod transport; - +pub use connection::ConnectionError; pub use endpoint::Config; -pub use error::Error; pub use muxer::QuicMuxer; +pub use quinn_proto::ConnectError as DialError; #[cfg(feature = "async-std")] pub use transport::{AsyncStd, AsyncStdTransport}; -pub use transport::{Provider, QuicTransport}; +pub use transport::{Provider, QuicTransport, TransportError}; #[cfg(feature = "tokio")] pub use transport::{Tokio, TokioTransport}; pub use upgrade::Upgrade; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 312950ab131..a0cbe9a8d10 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -18,13 +18,14 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::connection::{Connection, ConnectionEvent}; -use crate::error::Error; +use crate::{ + connection::{Connection, ConnectionEvent}, + ConnectionError, +}; use futures::{AsyncRead, AsyncWrite}; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; -use quinn_proto::FinishError; use std::{ collections::HashMap, io::{self, Write}, @@ -36,9 +37,6 @@ use std::{ /// State for a single opened QUIC connection. #[derive(Debug)] pub struct QuicMuxer { - // Note: This could theoretically be an asynchronous future, in order to yield the current - // task if a task running in parallel is already holding the lock. However, using asynchronous - // mutexes without async/await is extremely tedious and maybe not worth the effort. inner: Arc>, } @@ -85,7 +83,7 @@ impl QuicMuxer { impl StreamMuxer for QuicMuxer { type Substream = Substream; - type Error = Error; + type Error = ConnectionError; fn poll( self: Pin<&mut Self>, @@ -103,9 +101,7 @@ impl StreamMuxer for QuicMuxer { event ); } - ConnectionEvent::ConnectionLost(err) => { - return Poll::Ready(Err(Error::ConnectionLost(err))) - } + ConnectionEvent::ConnectionLost(err) => return Poll::Ready(Err(err)), ConnectionEvent::StreamOpened => { if let Some(waker) = inner.poll_outbound_waker.take() { waker.wake(); @@ -242,16 +238,15 @@ impl AsyncRead for Substream { cx: &mut Context<'_>, mut buf: &mut [u8], ) -> Poll> { - use quinn_proto::{ReadError, ReadableError}; let mut muxer = self.muxer.lock(); let mut stream = muxer.connection.recv_stream(self.id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, - Err(ReadableError::UnknownStream) => { + Err(quinn_proto::ReadableError::UnknownStream) => { return Poll::Ready(Ok(0)); } - Err(ReadableError::IllegalOrderedRead) => { + Err(quinn_proto::ReadableError::IllegalOrderedRead) => { unreachable!( "Illegal ordered read can only happen if `stream.read(false)` is used." ); @@ -269,10 +264,10 @@ impl AsyncRead for Substream { bytes += chunk.bytes.len(); } Ok(None) => break, - Err(err @ ReadError::Reset(_)) => { + Err(err @ quinn_proto::ReadError::Reset(_)) => { return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } - Err(ReadError::Blocked) => { + Err(quinn_proto::ReadError::Blocked) => { pending = true; break; } @@ -356,8 +351,8 @@ impl Drop for Substream { match send_stream.finish() { Ok(()) => {} // Already finished or reset, which is fine. - Err(FinishError::UnknownStream) => {} - Err(FinishError::Stopped(reason)) => { + Err(quinn_proto::FinishError::UnknownStream) => {} + Err(quinn_proto::FinishError::Stopped(reason)) => { let _ = send_stream.reset(reason); } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 1ddd4d3c873..ec293a7e556 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -24,8 +24,8 @@ use crate::connection::Connection; use crate::endpoint::ToEndpoint; -use crate::Config; use crate::{endpoint::EndpointChannel, muxer::QuicMuxer, upgrade::Upgrade}; +use crate::{Config, ConnectionError}; #[cfg(feature = "async-std")] mod async_std; @@ -45,7 +45,7 @@ use if_watch::{IfEvent, IfWatcher}; use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, - transport::{ListenerId, TransportError, TransportEvent}, + transport::{ListenerId, TransportError as CoreTransportError, TransportEvent}, PeerId, Transport, }; use std::collections::VecDeque; @@ -59,14 +59,6 @@ use std::{ task::{Context, Poll}, }; -// We reexport the errors that are exposed in the API. -// All of these types use one another. -pub use crate::connection::Error as Libp2pQuicConnectionError; -pub use quinn_proto::{ - ApplicationClose, ConfigError, ConnectError, ConnectionClose, ConnectionError, - TransportError as QuinnTransportError, TransportErrorCode, -}; - #[derive(Debug)] pub struct QuicTransport

{ config: Config, @@ -93,33 +85,37 @@ impl

QuicTransport

{ /// Error that can happen on the transport. #[derive(Debug, thiserror::Error)] -pub enum Error { +pub enum TransportError { /// Error while trying to reach a remote. #[error("{0}")] - Reach(ConnectError), + Reach(quinn_proto::ConnectError), /// Error after the remote has been reached. #[error("{0}")] - Established(Libp2pQuicConnectionError), + Established(ConnectionError), #[error("{0}")] Io(#[from] std::io::Error), - #[error("Background task crashed.")] - TaskCrashed, + /// The [`EndpointDriver`] has crashed. + #[error("Endpoint driver crashed")] + EndpointDriverCrashed, } impl Transport for QuicTransport

{ type Output = (PeerId, QuicMuxer); - type Error = Error; + type Error = TransportError; type ListenerUpgrade = Upgrade; type Dial = Pin> + Send>>; - fn listen_on(&mut self, addr: Multiaddr) -> Result> { - let socket_addr = - multiaddr_to_socketaddr(&addr).ok_or(TransportError::MultiaddrNotSupported(addr))?; + fn listen_on( + &mut self, + addr: Multiaddr, + ) -> Result> { + let socket_addr = multiaddr_to_socketaddr(&addr) + .ok_or(CoreTransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); let listener = Listener::new::

(listener_id, socket_addr, self.config.clone()) - .map_err(TransportError::Other)?; + .map_err(CoreTransportError::Other)?; self.listeners.push(listener); // Drop reference to dialer endpoint so that the endpoint is dropped once the last // connection that uses it is closed. @@ -144,11 +140,11 @@ impl Transport for QuicTransport

{ Some(observed.clone()) } - fn dial(&mut self, addr: Multiaddr) -> Result> { + fn dial(&mut self, addr: Multiaddr) -> Result> { let socket_addr = multiaddr_to_socketaddr(&addr) - .ok_or_else(|| TransportError::MultiaddrNotSupported(addr.clone()))?; + .ok_or_else(|| CoreTransportError::MultiaddrNotSupported(addr.clone()))?; if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { - return Err(TransportError::MultiaddrNotSupported(addr)); + return Err(CoreTransportError::MultiaddrNotSupported(addr)); } let mut listeners = self .listeners @@ -194,8 +190,8 @@ impl Transport for QuicTransport

{ Ok(async move { let connection = rx .await - .map_err(|_| Error::TaskCrashed)? - .map_err(Error::Reach)?; + .map_err(|_| TransportError::EndpointDriverCrashed)? + .map_err(TransportError::Reach)?; let final_connec = Upgrade::from_connection(connection).await?; Ok(final_connec) } @@ -205,7 +201,7 @@ impl Transport for QuicTransport

{ fn dial_as_listener( &mut self, addr: Multiaddr, - ) -> Result> { + ) -> Result> { // TODO: As the listener of a QUIC hole punch, we need to send a random UDP packet to the // `addr`. See DCUtR specification below. // @@ -245,9 +241,12 @@ struct Dialer { } impl Dialer { - fn new(config: Config, is_ipv6: bool) -> Result> { + fn new( + config: Config, + is_ipv6: bool, + ) -> Result> { let endpoint_channel = - EndpointChannel::new_dialer::

(config, is_ipv6).map_err(TransportError::Other)?; + EndpointChannel::new_dialer::

(config, is_ipv6).map_err(CoreTransportError::Other)?; Ok(Dialer { endpoint_channel, pending_dials: VecDeque::new(), @@ -295,7 +294,7 @@ impl Listener { listener_id: ListenerId, socket_addr: SocketAddr, config: Config, - ) -> Result { + ) -> Result { let (endpoint_channel, new_connections_rx) = EndpointChannel::new_bidirectional::

(config, socket_addr)?; @@ -327,7 +326,7 @@ impl Listener { /// Report the listener as closed in a [`TransportEvent::ListenerClosed`] and /// terminate the stream. - fn close(&mut self, reason: Result<(), Error>) { + fn close(&mut self, reason: Result<(), TransportError>) { if self.is_closed { return; } @@ -380,7 +379,7 @@ impl Listener { } impl Stream for Listener { - type Item = TransportEvent; + type Item = TransportEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { if let Some(event) = self.pending_event.take() { @@ -398,7 +397,7 @@ impl Stream for Listener { Ok(Ok(())) => {} Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), Err(_) => { - self.close(Err(Error::TaskCrashed)); + self.close(Err(TransportError::EndpointDriverCrashed)); continue; } } @@ -416,7 +415,7 @@ impl Stream for Listener { return Poll::Ready(Some(event)); } Poll::Ready(None) => { - self.close(Err(Error::TaskCrashed)); + self.close(Err(TransportError::EndpointDriverCrashed)); continue; } Poll::Pending => {} diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index c1cc50110d5..f7124431308 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -49,7 +49,7 @@ impl Upgrade { } impl Future for Upgrade { - type Output = Result<(PeerId, QuicMuxer), transport::Error>; + type Output = Result<(PeerId, QuicMuxer), transport::TransportError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let connection = self @@ -65,7 +65,7 @@ impl Future for Upgrade { return Poll::Ready(Ok((peer_id, muxer))); } ConnectionEvent::ConnectionLost(err) => { - return Poll::Ready(Err(transport::Error::Established(err))) + return Poll::Ready(Err(transport::TransportError::Established(err))) } // Other items are: // - HandshakeDataReady From ba8675d419892a77e5ea1697f5f7ea7613c7eb4f Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 28 Sep 2022 18:49:47 +0200 Subject: [PATCH 109/218] lib.rs: fix quic re-export --- src/lib.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index 29b32b8abf6..623fa29b8b3 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -111,8 +111,11 @@ pub use libp2p_plaintext as plaintext; #[cfg_attr(docsrs, doc(cfg(feature = "pnet")))] #[doc(inline)] pub use libp2p_pnet as pnet; -#[cfg(feature = "quic")] -#[cfg_attr(docsrs, doc(cfg(feature = "quic")))] +#[cfg(any(feature = "quic-async-std", feature = "quic-tokio"))] +#[cfg_attr( + docsrs, + doc(cfg(any(feature = "quic-async-std", feature = "quic-tokio"))) +)] #[cfg(not(any(target_os = "emscripten", target_os = "wasi", target_os = "unknown")))] #[doc(inline)] pub use libp2p_quic as quic; From d7259ffa82e2a52d7995fe356f0aaa3726daf956 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 28 Sep 2022 18:52:39 +0200 Subject: [PATCH 110/218] transports/quic: fix intra-doc-links --- transports/quic/src/connection.rs | 7 ++++--- transports/quic/src/endpoint.rs | 2 +- transports/quic/src/transport.rs | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index c5b388bd42b..59e3eaefb29 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -63,7 +63,7 @@ pub struct Connection { /// Error on the connection as a whole. #[derive(Debug, Clone, thiserror::Error)] pub enum ConnectionError { - /// The [`EndpointDriver`] has crashed. + /// The [`EndpointDriver`](super::endpoint::EndpointDriver) has crashed. #[error("Endpoint driver crashed")] EndpointDriverCrashed, /// Error in the inner state machine. @@ -74,8 +74,9 @@ pub enum ConnectionError { impl Connection { /// Crate-internal function that builds a [`Connection`] from raw components. /// - /// This function assumes that there exists a [`EndpointDriver`] that will process the messages - /// sent to `EndpointChannel::to_endpoint` and send us messages on `from_endpoint`. + /// This function assumes that there exists a [`EndpointDriver`](super::endpoint::EndpointDriver) + /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages + /// on `from_endpoint`. /// /// `connection_id` is used to identify the local connection in the messages sent to /// `to_endpoint`. diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index cfe57cbefc0..2bd2588e65a 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -281,7 +281,7 @@ pub enum ToEndpoint { /// guarantees that the [`EndpointDriver`], is properly kept alive for as long as any QUIC /// connection is open. /// -struct EndpointDriver { +pub struct EndpointDriver { // The actual QUIC state machine. endpoint: quinn_proto::Endpoint, // Config for client connections. diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index ec293a7e556..12e4da9613e 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -96,7 +96,7 @@ pub enum TransportError { #[error("{0}")] Io(#[from] std::io::Error), - /// The [`EndpointDriver`] has crashed. + /// The [`EndpointDriver`](super::endpoint::EndpointDriver) has crashed. #[error("Endpoint driver crashed")] EndpointDriverCrashed, } From db25b275dd694aa21646b64bb92c6dd6a2861046 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 28 Sep 2022 19:00:00 +0200 Subject: [PATCH 111/218] transports/quic: fmt, fix clippy --- transports/quic/src/connection.rs | 2 +- transports/quic/src/endpoint.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 59e3eaefb29..36afcaa0c4e 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -75,7 +75,7 @@ impl Connection { /// Crate-internal function that builds a [`Connection`] from raw components. /// /// This function assumes that there exists a [`EndpointDriver`](super::endpoint::EndpointDriver) - /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages + /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages /// on `from_endpoint`. /// /// `connection_id` is used to identify the local connection in the messages sent to diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 2bd2588e65a..8ee953357c9 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -140,7 +140,7 @@ impl EndpointChannel { socket_addr: socket.local_addr()?, }; - let server_config = new_connections.is_some().then(|| config.server_config); + let server_config = new_connections.is_some().then_some(config.server_config); let socket = P::from_socket(socket)?; let driver = EndpointDriver::

::new( From c1c044812e832e1e4a6a304efe907b1b3d660cf6 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 1 Oct 2022 01:18:56 +0200 Subject: [PATCH 112/218] transports/quic: mark version as alpha Co-authored-by: Max Inden --- transports/quic/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 0f75d4d9be5..f1c0201bb76 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "libp2p-quic" -version = "0.7.0" +version = "0.7.0-alpha" authors = ["David Craven ", "Parity Technologies "] edition = "2021" description = "TLS based QUIC transport implementation for libp2p" From 5fb7369532d716fd906aa96448a9c713f70d7535 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 1 Oct 2022 01:31:18 +0200 Subject: [PATCH 113/218] transports/quic: apply suggestions from code review Co-authored-by: Max Inden --- transports/quic/src/connection.rs | 2 +- transports/quic/src/lib.rs | 4 ++-- transports/quic/src/transport.rs | 1 - 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 36afcaa0c4e..4195d95e841 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -56,7 +56,7 @@ pub struct Connection { /// Identifier for this connection according to the endpoint. Used when sending messages to /// the endpoint. connection_id: quinn_proto::ConnectionHandle, - /// `Future` that triggers at the `Instant` that `self.connection.poll_timeout()` indicates. + /// `Future` that triggers at the [`Instant`] that `self.connection.poll_timeout()` indicates. next_timeout: Option<(Delay, Instant)>, } diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 070fffc3767..048233c8d2a 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -46,8 +46,8 @@ //! # } //! ``` //! -//! The `Endpoint` struct implements the `Transport` trait of the `core` library. See the -//! documentation of `core` and of libp2p in general to learn how to use the `Transport` trait. +//! The [`Endpoint`] struct implements the [`Transport`] trait of the [`libp2p_core`] library. See the +//! documentation of [`libp2p_core`] and of libp2p in general to learn how to use the [`Transport`] trait. //! //! Note that QUIC provides transport, security, and multiplexing in a single protocol. Therefore, //! QUIC connections do not need to be upgraded. You will get a compile-time error if you try. diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 12e4da9613e..d83e91e913a 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -498,7 +498,6 @@ pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { #[cfg(test)] #[cfg(any(feature = "async-std", feature = "tokio"))] mod test { - #[cfg(feature = "async-std")] use async_std_crate as async_std; use futures::future::poll_fn; From b3b8b33a74f51599727c1a436610c0f1cefc71a9 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 1 Oct 2022 01:46:59 +0200 Subject: [PATCH 114/218] transport/quic/tests: unwrap results, rm anyhow --- transports/quic/Cargo.toml | 1 - transports/quic/tests/smoke.rs | 94 +++++++++++++--------------------- 2 files changed, 37 insertions(+), 58 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 3fcbcdc3905..454a1926dbe 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -33,7 +33,6 @@ tokio = ["tokio-crate"] async-std = ["async-std-crate"] [dev-dependencies] -anyhow = "1.0.41" async-std-crate = { package = "async-std", version = "1.12.0", features = ["attributes"] } libp2p = { version = "0.49.0", default-features = false, features = ["request-response"], path = "../.." } rand = "0.8.4" diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 8019a9a2ab0..8b16261156b 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -1,4 +1,5 @@ -use anyhow::Result; +#![cfg(any(feature = "async-std", feature = "tokio"))] + use async_trait::async_trait; use futures::channel::oneshot; use futures::future::{join, FutureExt}; @@ -15,6 +16,7 @@ use libp2p::request_response::{ }; use libp2p::swarm::dial_opts::{DialOpts, PeerCondition}; use libp2p::swarm::{ConnectionError, DialError, Swarm, SwarmEvent}; +use libp2p::Multiaddr; use libp2p_quic as quic; use quic::Provider; use rand::RngCore; @@ -35,7 +37,7 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { } #[tracing::instrument] -async fn create_swarm(keylog: bool) -> Result>> { +async fn create_swarm(keylog: bool) -> Swarm> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let config = quic::Config::new(&keypair).unwrap(); @@ -50,7 +52,15 @@ async fn create_swarm(keylog: bool) -> Result>, addr: &str) -> Multiaddr { + swarm.listen_on(addr.parse().unwrap()).unwrap(); + match swarm.next().await { + Some(SwarmEvent::NewListenAddr { address, .. }) => address, + e => panic!("{:?}", e), + } } fn setup_global_subscriber() { @@ -63,29 +73,25 @@ fn setup_global_subscriber() { #[cfg(feature = "tokio")] #[tokio::test] -async fn tokio_smoke() -> Result<()> { +async fn tokio_smoke() { smoke::().await } #[cfg(feature = "async-std")] #[async_std::test] -async fn async_std_smoke() -> Result<()> { +async fn async_std_smoke() { smoke::().await } -async fn smoke() -> Result<()> { +async fn smoke() { setup_global_subscriber(); 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()?)?; + let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; - let addr = match a.next().await { - Some(SwarmEvent::NewListenAddr { address, .. }) => address, - e => panic!("{:?}", e), - }; tracing::info!(?addr); let mut data = vec![0; 4096 * 10]; @@ -218,7 +224,6 @@ async fn smoke() -> Result<()> { }; join(fut_a, fut_b).await; - Ok(()) } #[derive(Debug, Clone)] @@ -302,18 +307,14 @@ impl RequestResponseCodec for PingCodec { #[cfg(feature = "async-std")] #[async_std::test] -async fn dial_failure() -> Result<()> { +async fn dial_failure() { setup_global_subscriber(); - let mut a = create_swarm::(false).await?; - let mut b = create_swarm::(true).await?; + let mut a = create_swarm::(false).await; + let mut b = create_swarm::(true).await; - Swarm::listen_on(&mut a, "/ip4/127.0.0.1/udp/0/quic".parse()?)?; + let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; - let addr = match a.next().await { - Some(SwarmEvent::NewListenAddr { address, .. }) => address, - e => panic!("{:?}", e), - }; let a_peer_id = &Swarm::local_peer_id(&a).clone(); drop(a); // stop a swarm so b can never reach it @@ -335,8 +336,6 @@ async fn dial_failure() -> Result<()> { Some(SwarmEvent::Behaviour(RequestResponseEvent::OutboundFailure { .. })) => {} e => panic!("{:?}", e), }; - - Ok(()) } #[test] @@ -363,14 +362,8 @@ fn concurrent_connections_and_streams() { // Spawn the listener nodes. for _ in 0..number_listeners { - let mut listener = create_swarm::

(true).await.unwrap(); - Swarm::listen_on(&mut listener, "/ip4/127.0.0.1/udp/0/quic".parse().unwrap()).unwrap(); - - // Wait to listen on address. - let addr = match listener.next().await { - Some(SwarmEvent::NewListenAddr { address, .. }) => address, - e => panic!("{:?}", e), - }; + let mut listener = create_swarm::

(true).await; + let addr = start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic").await; listeners.push((*listener.local_peer_id(), addr)); @@ -421,7 +414,7 @@ fn concurrent_connections_and_streams() { .unwrap(); } - let mut dialer = create_swarm::

(true).await.unwrap(); + let mut dialer = create_swarm::

(true).await; // For each listener node start `number_streams` requests. for (listener_peer_id, listener_addr) in &listeners { @@ -494,18 +487,14 @@ fn concurrent_connections_and_streams() { #[cfg(feature = "tokio")] #[tokio::test] -async fn endpoint_reuse() -> Result<()> { +async fn endpoint_reuse() { setup_global_subscriber(); - let mut swarm_a = create_swarm::(false).await?; - let mut swarm_b = create_swarm::(false).await?; + let mut swarm_a = create_swarm::(false).await; + let mut swarm_b = create_swarm::(false).await; let b_peer_id = *swarm_b.local_peer_id(); - swarm_a.listen_on("/ip4/127.0.0.1/udp/0/quic".parse()?)?; - let a_addr = match swarm_a.next().await { - Some(SwarmEvent::NewListenAddr { address, .. }) => address, - e => panic!("{:?}", e), - }; + let a_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/udp/0/quic").await; swarm_b.dial(a_addr.clone()).unwrap(); let b_send_back_addr = loop { @@ -547,11 +536,8 @@ async fn endpoint_reuse() -> Result<()> { _ = swarm_b.select_next_some() => {}, } } - swarm_b.listen_on("/ip4/127.0.0.1/udp/0/quic".parse()?)?; - let b_addr = match swarm_b.next().await { - Some(SwarmEvent::NewListenAddr { address, .. }) => address, - e => panic!("{:?}", e), - }; + + let b_addr = start_listening(&mut swarm_b, "/ip4/127.0.0.1/udp/0/quic").await; let dial_opts = DialOpts::peer_id(b_peer_id) .addresses(vec![b_addr.clone(), b_send_back_addr]) @@ -590,23 +576,17 @@ async fn endpoint_reuse() -> Result<()> { }, } } - - Ok(()) } #[cfg(feature = "async-std")] #[async_std::test] -async fn ipv4_dial_ipv6() -> Result<()> { +async fn ipv4_dial_ipv6() { setup_global_subscriber(); - let mut swarm_a = create_swarm::(false).await?; - let mut swarm_b = create_swarm::(false).await?; + let mut swarm_a = create_swarm::(false).await; + let mut swarm_b = create_swarm::(false).await; - swarm_a.listen_on("/ip6/::1/udp/0/quic".parse()?)?; - let a_addr = match swarm_a.next().await { - Some(SwarmEvent::NewListenAddr { address, .. }) => address, - e => panic!("{:?}", e), - }; + let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; swarm_b.dial(a_addr.clone()).unwrap(); @@ -614,7 +594,7 @@ async fn ipv4_dial_ipv6() -> Result<()> { select! { ev = swarm_a.select_next_some() => match ev { SwarmEvent::ConnectionEstablished { .. } => { - return Ok(()) + return; } SwarmEvent::IncomingConnection { local_addr, ..} => { assert!(swarm_a.listeners().any(|a| a == &local_addr)); From a91894db22594eb85037985df8f6f1ea9305c661 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 1 Oct 2022 01:51:30 +0200 Subject: [PATCH 115/218] transport/quic: suggestions from code-review --- transports/quic/src/transport.rs | 3 ++- transports/quic/src/upgrade.rs | 16 +++++++--------- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index d83e91e913a..7a8c1dc8391 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -33,6 +33,7 @@ mod async_std; mod tokio; #[cfg(feature = "async-std")] pub use async_std::{AsyncStd, AsyncStdTransport}; +use futures::future::BoxFuture; #[cfg(feature = "tokio")] pub use tokio::{Tokio, TokioTransport}; @@ -105,7 +106,7 @@ impl Transport for QuicTransport

{ type Output = (PeerId, QuicMuxer); type Error = TransportError; type ListenerUpgrade = Upgrade; - type Dial = Pin> + Send>>; + type Dial = BoxFuture<'static, Result>; fn listen_on( &mut self, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index f7124431308..12a7f1c3db8 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -67,15 +67,13 @@ impl Future for Upgrade { ConnectionEvent::ConnectionLost(err) => { return Poll::Ready(Err(transport::TransportError::Established(err))) } - // Other items are: - // - HandshakeDataReady - // - StreamAvailable - // - StreamOpened - // - StreamReadable - // - StreamWritable - // - StreamFinished - // - StreamStopped - _ => {} + ConnectionEvent::HandshakeDataReady + | ConnectionEvent::StreamAvailable + | ConnectionEvent::StreamOpened + | ConnectionEvent::StreamReadable(_) + | ConnectionEvent::StreamWritable(_) + | ConnectionEvent::StreamFinished(_) + | ConnectionEvent::StreamStopped(_) => {} } } } From 916d323741b86516adab35da91c25d8fb95ee739 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 1 Oct 2022 02:00:02 +0200 Subject: [PATCH 116/218] transport/quic/test: remove tracing --- transports/quic/Cargo.toml | 1 - transports/quic/tests/smoke.rs | 70 +++++++++------------------------- 2 files changed, 17 insertions(+), 54 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 454a1926dbe..355350350fc 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -37,5 +37,4 @@ async-std-crate = { package = "async-std", version = "1.12.0", features = ["attr libp2p = { version = "0.49.0", default-features = false, features = ["request-response"], path = "../.." } rand = "0.8.4" tokio-crate = { package = "tokio", version = "1.21.1", features = ["macros", "rt-multi-thread"] } -tracing-subscriber = {version = "0.3.8", features = ["env-filter"] } quickcheck = "1" diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 8b16261156b..d94c1000da5 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -36,8 +36,7 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } -#[tracing::instrument] -async fn create_swarm(keylog: bool) -> Swarm> { +async fn create_swarm() -> Swarm> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let config = quic::Config::new(&keypair).unwrap(); @@ -51,7 +50,6 @@ async fn create_swarm(keylog: bool) -> Swarm>, addr: &s } } -fn setup_global_subscriber() { - let filter_layer = tracing_subscriber::EnvFilter::from_default_env(); - tracing_subscriber::fmt() - .with_env_filter(filter_layer) - .try_init() - .ok(); -} - #[cfg(feature = "tokio")] #[tokio::test] async fn tokio_smoke() { @@ -84,16 +74,13 @@ async fn async_std_smoke() { } async fn smoke() { - setup_global_subscriber(); 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::

().await; + let mut b = create_swarm::

().await; let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; - tracing::info!(?addr); - let mut data = vec![0; 4096 * 10]; rng.fill_bytes(&mut data); @@ -308,10 +295,8 @@ impl RequestResponseCodec for PingCodec { #[cfg(feature = "async-std")] #[async_std::test] async fn dial_failure() { - setup_global_subscriber(); - - let mut a = create_swarm::(false).await; - let mut b = create_swarm::(true).await; + let mut a = create_swarm::().await; + let mut b = create_swarm::().await; let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; @@ -342,9 +327,6 @@ async fn dial_failure() { fn concurrent_connections_and_streams() { use quickcheck::*; - setup_global_subscriber(); - - #[tracing::instrument] async fn prop( number_listeners: NonZeroU8, number_streams: NonZeroU8, @@ -362,7 +344,7 @@ fn concurrent_connections_and_streams() { // Spawn the listener nodes. for _ in 0..number_listeners { - let mut listener = create_swarm::

(true).await; + let mut listener = create_swarm::

().await; let addr = start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic").await; listeners.push((*listener.local_peer_id(), addr)); @@ -372,12 +354,6 @@ fn concurrent_connections_and_streams() { async move { loop { match listener.next().await { - Some(SwarmEvent::ConnectionEstablished { .. }) => { - tracing::info!("listener ConnectionEstablished"); - } - Some(SwarmEvent::IncomingConnection { .. }) => { - tracing::info!("listener IncomingConnection"); - } Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { message: RequestResponseMessage::Request { @@ -387,7 +363,6 @@ fn concurrent_connections_and_streams() { }, .. })) => { - tracing::info!("listener got Message"); listener .behaviour_mut() .send_response(channel, Pong(ping)) @@ -395,10 +370,10 @@ fn concurrent_connections_and_streams() { } Some(SwarmEvent::Behaviour( RequestResponseEvent::ResponseSent { .. }, - )) => { - tracing::info!("listener ResponseSent"); - } - Some(SwarmEvent::ConnectionClosed { .. }) => {} + )) + | Some(SwarmEvent::ConnectionEstablished { .. }) + | Some(SwarmEvent::IncomingConnection { .. }) + | Some(SwarmEvent::ConnectionClosed { .. }) => {} Some(e) => { panic!("unexpected event {:?}", e); } @@ -414,7 +389,7 @@ fn concurrent_connections_and_streams() { .unwrap(); } - let mut dialer = create_swarm::

(true).await; + let mut dialer = create_swarm::

().await; // For each listener node start `number_streams` requests. for (listener_peer_id, listener_addr) in &listeners { @@ -430,11 +405,8 @@ fn concurrent_connections_and_streams() { let mut num_responses = 0; loop { match dialer.next().await { - Some(SwarmEvent::Dialing(_)) => { - tracing::info!("dialer Dialing"); - } + Some(SwarmEvent::Dialing(_)) => {} Some(SwarmEvent::ConnectionEstablished { peer_id, .. }) => { - tracing::info!("dialer Connection established"); for _ in 0..number_streams { dialer .behaviour_mut() @@ -449,18 +421,14 @@ fn concurrent_connections_and_streams() { }, .. })) => { - tracing::info!("dialer got Message"); num_responses += 1; assert_eq!(data, pong); let should_be = number_listeners as usize * (number_streams) as usize; - tracing::info!(?num_responses, ?should_be); if num_responses == should_be { break; } } - Some(SwarmEvent::ConnectionClosed { .. }) => { - tracing::info!("dialer ConnectionClosed"); - } + Some(SwarmEvent::ConnectionClosed { .. }) => {} e => { panic!("unexpected event {:?}", e); } @@ -488,10 +456,8 @@ fn concurrent_connections_and_streams() { #[cfg(feature = "tokio")] #[tokio::test] async fn endpoint_reuse() { - setup_global_subscriber(); - - let mut swarm_a = create_swarm::(false).await; - let mut swarm_b = create_swarm::(false).await; + let mut swarm_a = create_swarm::().await; + let mut swarm_b = create_swarm::().await; let b_peer_id = *swarm_b.local_peer_id(); let a_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/udp/0/quic").await; @@ -581,10 +547,8 @@ async fn endpoint_reuse() { #[cfg(feature = "async-std")] #[async_std::test] async fn ipv4_dial_ipv6() { - setup_global_subscriber(); - - let mut swarm_a = create_swarm::(false).await; - let mut swarm_b = create_swarm::(false).await; + let mut swarm_a = create_swarm::().await; + let mut swarm_b = create_swarm::().await; let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; From 2b80d05f2194cda47ca184e029d6c384e028aa90 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 1 Oct 2022 02:00:33 +0200 Subject: [PATCH 117/218] transports/quic: use `log` instead of `tracing` --- transports/quic/Cargo.toml | 2 +- transports/quic/src/endpoint.rs | 16 ++++++++-------- transports/quic/src/muxer.rs | 2 +- transports/quic/src/transport.rs | 4 ++-- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 355350350fc..eac115bec0f 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -14,6 +14,7 @@ futures = "0.3.15" futures-timer = "3.0.2" if-watch = "2.0.0" libp2p-core = { version = "0.37.0", path = "../../core" } +log = "0.4" parking_lot = "0.12.0" quinn-proto = { version = "0.8.2", default-features = false, features = ["tls-rustls"] } rand = "0.8.5" @@ -22,7 +23,6 @@ ring = "0.16.20" rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } thiserror = "1.0.26" tokio-crate = { package = "tokio", version = "1.21.1", default-features = false, features = ["net", "rt"], optional = true } -tracing = "0.1.30" webpki = "0.22.0" x509-parser = "0.13.0" yasna = "0.5.0" diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 8ee953357c9..3ba0d4773ed 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -389,7 +389,7 @@ impl EndpointDriver

{ if is_drained_event { self.alive_connections.remove(&connection_id); if self.is_orphaned && self.alive_connections.is_empty() { - tracing::info!( + log::info!( "Listener closed and no active connections remain. Shutting down the background task." ); return ControlFlow::Break(()); @@ -408,7 +408,7 @@ impl EndpointDriver

{ // If the connection is busy, it won't sent us any more events to handle. let _ = sender.clone().start_send(event_back); } else { - tracing::error!("State mismatch: event for closed connection"); + log::error!("State mismatch: event for closed connection"); } } } @@ -442,7 +442,7 @@ impl EndpointDriver

{ // the remote. let _ = sender.try_send(event); } else { - tracing::error!("State mismatch: event for closed connection"); + log::error!("State mismatch: event for closed connection"); } } quinn_proto::DatagramEvent::NewConnection(connec) => { @@ -452,7 +452,7 @@ impl EndpointDriver

{ let connection_tx = match self.new_connection_tx.as_mut() { Some(tx) => tx, None => { - tracing::warn!( + log::warn!( "Endpoint reported a new connection even though server capabilities are disabled." ); return ControlFlow::Continue(()); @@ -475,7 +475,7 @@ impl EndpointDriver

{ self.new_connection_tx = None; self.is_orphaned = true; } - Err(_) => tracing::warn!( + Err(_) => log::warn!( "Dropping new incoming connection {:?} because the channel to the listener is full", connec_id ) @@ -493,7 +493,7 @@ impl Future for EndpointDriver

{ if let Some((send_packet, len)) = self.next_packet_out.as_mut() { match ready!(send_packet.poll_unpin(cx)) { Ok(n) if n == *len => {} - Ok(_) => tracing::error!( + Ok(_) => log::error!( "QUIC UDP socket violated expectation that packets are always fully \ transferred" ), @@ -502,7 +502,7 @@ impl Future for EndpointDriver

{ // robust to packet losses and it is consequently not a logic error to process with // normal operations. Err(err) => { - tracing::error!("Error while sending on QUIC UDP socket: {:?}", err) + log::error!("Error while sending on QUIC UDP socket: {:?}", err) } } self.next_packet_out = None; @@ -536,7 +536,7 @@ impl Future for EndpointDriver

{ // Errors on the socket are expected to never happen, and we handle them by // simply printing a log message. Poll::Ready(Some(Err(err))) => { - tracing::error!("Error while receive on QUIC UDP socket: {:?}", err); + log::error!("Error while receive on QUIC UDP socket: {:?}", err); continue; } Poll::Ready(None) => { diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index a0cbe9a8d10..d3ff0c7422d 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -203,7 +203,7 @@ impl StreamMuxer for QuicMuxer { if connection.send_stream_count() != 0 { for substream in substreams.keys() { if let Err(e) = connection.finish_substream(*substream) { - tracing::warn!("substream finish error on muxer close: {}", e); + log::warn!("substream finish error on muxer close: {}", e); } } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 7a8c1dc8391..9f3481956ed 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -350,7 +350,7 @@ impl Listener { if let Some(listen_addr) = ip_to_listenaddr(self.endpoint_channel.socket_addr(), inet.addr()) { - tracing::debug!("New listen address: {}", listen_addr); + log::debug!("New listen address: {}", listen_addr); return Poll::Ready(TransportEvent::NewAddress { listener_id: self.listener_id, listen_addr, @@ -361,7 +361,7 @@ impl Listener { if let Some(listen_addr) = ip_to_listenaddr(self.endpoint_channel.socket_addr(), inet.addr()) { - tracing::debug!("Expired listen address: {}", listen_addr); + log::debug!("Expired listen address: {}", listen_addr); return Poll::Ready(TransportEvent::AddressExpired { listener_id: self.listener_id, listen_addr, From a3ed858051ee5702c7423d45670cf245ae1d0322 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 1 Oct 2022 03:34:21 +0200 Subject: [PATCH 118/218] transports/quic: fix intra-doc-link --- transports/quic/src/connection.rs | 3 ++- transports/quic/src/transport.rs | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 4195d95e841..fcd6c4839f7 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -63,9 +63,10 @@ pub struct Connection { /// Error on the connection as a whole. #[derive(Debug, Clone, thiserror::Error)] pub enum ConnectionError { - /// The [`EndpointDriver`](super::endpoint::EndpointDriver) has crashed. + /// The task driving the endpoint has crashed. #[error("Endpoint driver crashed")] EndpointDriverCrashed, + /// Error in the inner state machine. #[error("{0}")] Quinn(#[from] quinn_proto::ConnectionError), diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 9f3481956ed..395a8b6e093 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -97,7 +97,7 @@ pub enum TransportError { #[error("{0}")] Io(#[from] std::io::Error), - /// The [`EndpointDriver`](super::endpoint::EndpointDriver) has crashed. + /// The task driving the endpoint has crashed. #[error("Endpoint driver crashed")] EndpointDriverCrashed, } From ebc2c9a290d298a0fa77a2f54851e91878d30067 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 3 Oct 2022 15:37:40 +0200 Subject: [PATCH 119/218] transport/quic: Remove default feature See #2918. --- transports/quic/Cargo.toml | 1 - 1 file changed, 1 deletion(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index eac115bec0f..f921155b497 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -28,7 +28,6 @@ x509-parser = "0.13.0" yasna = "0.5.0" [features] -default = ["async-std"] tokio = ["tokio-crate"] async-std = ["async-std-crate"] From dcf61986f8ed73c89b6ad49b1cf7eee728f779b1 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 2 Oct 2022 19:03:13 +0200 Subject: [PATCH 120/218] transports/quic: poll-based Provider interface --- transports/quic/Cargo.toml | 1 + transports/quic/src/endpoint.rs | 122 +++++---------------- transports/quic/src/transport.rs | 24 ++-- transports/quic/src/transport/async_std.rs | 103 +++++++++++++++-- transports/quic/src/transport/tokio.rs | 57 ++++++++-- 5 files changed, 178 insertions(+), 129 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index f921155b497..92ba3098d8a 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -10,6 +10,7 @@ license = "MIT" [dependencies] async-std-crate = { package = "async-std", version = "1.12.0", default-features = false, optional = true } async-trait = "0.1.50" +bytes = "1.2.1" futures = "0.3.15" futures-timer = "3.0.2" if-watch = "2.0.0" diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 3ba0d4773ed..b0bfbf97070 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -34,6 +34,7 @@ use crate::{ transport::{self, Provider}, }; +use bytes::BytesMut; use futures::{ channel::{mpsc, oneshot}, prelude::*, @@ -41,10 +42,8 @@ use futures::{ }; use std::{ collections::HashMap, - io, net::{Ipv4Addr, Ipv6Addr, SocketAddr}, ops::ControlFlow, - pin::Pin, sync::Arc, task::{Context, Poll}, time::{Duration, Instant}, @@ -292,14 +291,9 @@ pub struct EndpointDriver { rx: mpsc::Receiver, // Socket for sending and receiving datagrams. - socket: Arc, + socket: P, // Future for writing the next packet to the socket. - next_packet_out: Option<( - Pin> + Send>>, - usize, - )>, - // Stream of inbound datagrams. - receive_stream: ReceiveStream

, + next_packet_out: Option, // List of all active connections, with a sender to notify them of events. alive_connections: @@ -318,34 +312,22 @@ impl EndpointDriver

{ new_connection_tx: Option>, server_config: Option>, channel: EndpointChannel, - socket: P::Socket, + socket: P, rx: mpsc::Receiver, ) -> Self { - let socket = Arc::new(socket); EndpointDriver { endpoint: quinn_proto::Endpoint::new(endpoint_config, server_config), client_config, channel, rx, - socket: socket.clone(), + socket, next_packet_out: None, - receive_stream: ReceiveStream::new(socket), alive_connections: HashMap::new(), new_connection_tx, is_orphaned: false, } } - /// Insert future to send a datagram on the socket. - fn send_packet_out(&mut self, transmit: quinn_proto::Transmit) { - let len = transmit.contents.len(); - let socket = self.socket.clone(); - let send = - async move { P::send_to(&socket, &transmit.contents, transmit.destination).await } - .boxed(); - self.next_packet_out = Some((send, len)); - } - /// Handle a message sent from either the [`QuicTransport`](super::QuicTransport) or a [`Connection`]. fn handle_message(&mut self, to_endpoint: ToEndpoint) -> ControlFlow<()> { match to_endpoint { @@ -414,15 +396,14 @@ impl EndpointDriver

{ } // Data needs to be sent on the UDP socket. - ToEndpoint::SendUdpPacket(transmit) => self.send_packet_out(transmit), + ToEndpoint::SendUdpPacket(transmit) => self.next_packet_out = Some(transmit), } ControlFlow::Continue(()) } /// Handle datagram received on the socket. /// The datagram content was written into the `socket_recv_buffer`. - fn handle_datagram(&mut self, bytes: Vec, packet_src: SocketAddr) -> ControlFlow<()> { - let packet = From::from(bytes.as_bytes()); + fn handle_datagram(&mut self, packet: BytesMut, packet_src: SocketAddr) -> ControlFlow<()> { let local_ip = self.channel.socket_addr.ip(); // TODO: ECN bits aren't handled let (connec_id, event) = @@ -490,28 +471,27 @@ impl Future for EndpointDriver

{ type Output = (); fn poll(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { - if let Some((send_packet, len)) = self.next_packet_out.as_mut() { - match ready!(send_packet.poll_unpin(cx)) { - Ok(n) if n == *len => {} - Ok(_) => log::error!( - "QUIC UDP socket violated expectation that packets are always fully \ - transferred" - ), - // Errors on the socket are expected to never happen, and we handle them by simply - // printing a log message. The packet gets discarded in case of error, but we are - // robust to packet losses and it is consequently not a logic error to process with - // normal operations. - Err(err) => { - log::error!("Error while sending on QUIC UDP socket: {:?}", err) - } + match ready!(self.socket.poll_send_flush(cx)) { + Ok(_) => {} + // Errors on the socket are expected to never happen, and we handle them by simply + // printing a log message. The packet gets discarded in case of error, but we are + // robust to packet losses and it is consequently not a logic error to process with + // normal operations. + Err(err) => { + log::error!("Error while sending on QUIC UDP socket: {:?}", err) } - self.next_packet_out = None; + } + + if let Some(transmit) = self.next_packet_out.take() { + self.socket + .start_send(transmit.contents, transmit.destination); + continue; } // The endpoint might request packets to be sent out. This is handled in priority to avoid // buffering up packets. if let Some(transmit) = self.endpoint.poll_transmit() { - self.send_packet_out(transmit); + self.next_packet_out = Some(transmit); continue; } @@ -526,22 +506,20 @@ impl Future for EndpointDriver

{ Poll::Pending => {} } - match self.receive_stream.poll_next_unpin(cx) { - Poll::Ready(Some(Ok((bytes, packet_src)))) => { - match self.handle_datagram(bytes, packet_src) { + match self.socket.poll_recv_from(cx) { + Poll::Ready(Ok((bytes, packet_src))) => { + let bytes_mut = bytes.as_bytes().into(); + match self.handle_datagram(bytes_mut, packet_src) { ControlFlow::Continue(()) => continue, ControlFlow::Break(()) => break, } } // Errors on the socket are expected to never happen, and we handle them by // simply printing a log message. - Poll::Ready(Some(Err(err))) => { + Poll::Ready(Err(err)) => { log::error!("Error while receive on QUIC UDP socket: {:?}", err); continue; } - Poll::Ready(None) => { - unreachable!("ReceiveStream::poll_next never returns Poll::Ready(None)") - } Poll::Pending => {} } return Poll::Pending; @@ -550,49 +528,3 @@ impl Future for EndpointDriver

{ Poll::Ready(()) } } - -/// Wrapper around the socket to implement `Stream` on it socket. -/// This is needed since not all [`Provider`]s provide a poll-based receive method for their socket. -struct ReceiveStream { - fut: Pin< - Box< - dyn Future< - Output = ( - Result<(usize, SocketAddr), io::Error>, - Arc, - Vec, - ), - > + Send, - >, - >, -} - -impl ReceiveStream

{ - fn new(socket: Arc) -> Self { - let mut socket_recv_buffer = vec![0; 65536]; - let fut = async move { - let recv = P::recv_from(&socket, &mut socket_recv_buffer).await; - (recv, socket, socket_recv_buffer) - }; - Self { fut: fut.boxed() } - } -} - -impl Stream for ReceiveStream

{ - type Item = Result<(Vec, SocketAddr), io::Error>; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let (result, socket, mut buffer) = ready!(self.fut.poll_unpin(cx)); - let result = result.map(|(packet_len, packet_src)| { - debug_assert!(packet_len <= buffer.len()); - // Copies the bytes from the `socket_recv_buffer` they were written into. - (buffer[..packet_len].into(), packet_src) - }); - self.fut = async move { - let recv = P::recv_from(&socket, &mut buffer).await; - (recv, socket, buffer) - } - .boxed(); - Poll::Ready(Some(result)) - } -} diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 395a8b6e093..dd285052b0a 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -427,22 +427,24 @@ impl Stream for Listener { } } -#[async_trait::async_trait] -pub trait Provider: Unpin + Send + 'static { - // Wrapped socket for non-blocking I/O operations. - type Socket: Send + Sync + Unpin; - +// Wrapped socket for non-blocking I/O operations. +pub trait Provider: Unpin + Send + Sized + 'static { // Wrap a socket. // Note: The socket must be set to non-blocking. - fn from_socket(socket: UdpSocket) -> io::Result; + fn from_socket(socket: UdpSocket) -> io::Result; // Receive a single datagram message. - // Return the number of bytes read and the address the message came from. - async fn recv_from(socket: &Self::Socket, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)>; + // Returns the message and the address the message came from. + fn poll_recv_from(&mut self, cx: &mut Context<'_>) -> Poll, SocketAddr)>>; + + // Set sending the packet on the socket. + // Since only one packet may be sent at a time, this may only be called if a preceding call + // to [`Provider::poll_send_flush`] returned [`Poll::Ready`]. + fn start_send(&mut self, data: Vec, addr: SocketAddr); - // Send data on the socket to the specified address. - // Return the number of bytes written. - async fn send_to(socket: &Self::Socket, buf: &[u8], addr: SocketAddr) -> io::Result; + // Flush a packet send in [`Provider::start_send`]. + // If [`Poll::Ready`] is returned the socket is ready for sending a new packet. + fn poll_send_flush(&mut self, cx: &mut Context<'_>) -> Poll>; // Run the given future in the background until it ends. fn spawn(future: impl Future + Send + 'static); diff --git a/transports/quic/src/transport/async_std.rs b/transports/quic/src/transport/async_std.rs index 26c853ccc76..e22fa82987f 100644 --- a/transports/quic/src/transport/async_std.rs +++ b/transports/quic/src/transport/async_std.rs @@ -18,35 +18,116 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use std::{io, net::SocketAddr}; +use std::{ + io, + net::SocketAddr, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; use async_std_crate::{net::UdpSocket, task::spawn}; -use futures::Future; +use futures::{future::BoxFuture, ready, Future, FutureExt, Stream, StreamExt}; use crate::QuicTransport; use super::Provider; pub type AsyncStdTransport = QuicTransport; -pub struct AsyncStd; +pub struct AsyncStd { + socket: Arc, + send_packet: Option>>, + recv_stream: ReceiveStream, +} -#[async_trait::async_trait] impl Provider for AsyncStd { - type Socket = UdpSocket; + fn from_socket(socket: std::net::UdpSocket) -> io::Result { + let socket = Arc::new(socket.into()); + let recv_stream = ReceiveStream::new(Arc::clone(&socket)); + Ok(AsyncStd { + socket, + send_packet: None, + recv_stream, + }) + } - fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { - Ok(socket.into()) + fn poll_recv_from(&mut self, cx: &mut Context<'_>) -> Poll, SocketAddr)>> { + match self.recv_stream.poll_next_unpin(cx) { + Poll::Ready(ready) => { + Poll::Ready(ready.expect("ReceiveStream::poll_next never returns None.")) + } + Poll::Pending => Poll::Pending, + } } - async fn recv_from(socket: &Self::Socket, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { - socket.recv_from(buf).await + fn start_send(&mut self, data: Vec, addr: SocketAddr) { + let _len = data.len(); + let socket = self.socket.clone(); + let send = async move { + let _send_len = socket.send_to(&data, addr).await?; + Ok(()) + } + .boxed(); + self.send_packet = Some(send) } - async fn send_to(socket: &Self::Socket, buf: &[u8], addr: SocketAddr) -> io::Result { - socket.send_to(buf, addr).await + fn poll_send_flush(&mut self, cx: &mut Context<'_>) -> Poll> { + let pending = match self.send_packet.as_mut() { + Some(pending) => pending, + None => return Poll::Ready(Ok(())), + }; + match pending.poll_unpin(cx) { + Poll::Ready(result) => { + self.send_packet = None; + Poll::Ready(result) + } + Poll::Pending => Poll::Pending, + } } fn spawn(future: impl Future + Send + 'static) { spawn(future); } } + +/// Wrapper around the socket to implement `Stream` on it. +struct ReceiveStream { + fut: BoxFuture< + 'static, + ( + Result<(usize, SocketAddr), io::Error>, + Arc, + Vec, + ), + >, +} + +impl ReceiveStream { + fn new(socket: Arc) -> Self { + let mut socket_recv_buffer = vec![0; 65536]; + let fut = async move { + let recv = socket.recv_from(&mut socket_recv_buffer).await; + (recv, socket, socket_recv_buffer) + }; + Self { fut: fut.boxed() } + } +} + +impl Stream for ReceiveStream { + type Item = Result<(Vec, SocketAddr), io::Error>; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let (result, socket, mut buffer) = ready!(self.fut.poll_unpin(cx)); + let result = result.map(|(packet_len, packet_src)| { + debug_assert!(packet_len <= buffer.len()); + // Copies the bytes from the `socket_recv_buffer` they were written into. + (buffer[..packet_len].into(), packet_src) + }); + self.fut = async move { + let recv = socket.recv_from(&mut buffer).await; + (recv, socket, buffer) + } + .boxed(); + Poll::Ready(Some(result)) + } +} diff --git a/transports/quic/src/transport/tokio.rs b/transports/quic/src/transport/tokio.rs index fc9043a3343..bdb6ef4df6a 100644 --- a/transports/quic/src/transport/tokio.rs +++ b/transports/quic/src/transport/tokio.rs @@ -18,32 +18,65 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use std::{io, net::SocketAddr}; +use std::{ + io, + net::SocketAddr, + task::{Context, Poll}, +}; -use futures::Future; -use tokio_crate::net::UdpSocket; +use futures::{ready, Future}; +use tokio_crate::{io::ReadBuf, net::UdpSocket}; +use x509_parser::nom::AsBytes; use crate::QuicTransport; use super::Provider; pub type TokioTransport = QuicTransport; -pub struct Tokio; +pub struct Tokio { + socket: UdpSocket, + socket_recv_buffer: Vec, + next_packet_out: Option<(Vec, SocketAddr)>, +} -#[async_trait::async_trait] impl Provider for Tokio { - type Socket = UdpSocket; + fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { + let socket = UdpSocket::from_std(socket)?; + Ok(Tokio { + socket, + socket_recv_buffer: vec![0; 65536], + next_packet_out: None, + }) + } - fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { - UdpSocket::from_std(socket) + fn poll_send_flush(&mut self, cx: &mut Context<'_>) -> Poll> { + let (data, addr) = match self.next_packet_out.as_ref() { + Some(pending) => pending, + None => return Poll::Ready(Ok(())), + }; + match self.socket.poll_send_to(cx, data.as_bytes(), *addr) { + Poll::Ready(result) => { + self.next_packet_out = None; + Poll::Ready(result.map(|_| ())) + } + Poll::Pending => Poll::Pending, + } } - async fn recv_from(socket: &Self::Socket, buf: &mut [u8]) -> io::Result<(usize, SocketAddr)> { - socket.recv_from(buf).await + fn poll_recv_from(&mut self, cx: &mut Context<'_>) -> Poll, SocketAddr)>> { + let Self { + socket, + socket_recv_buffer, + .. + } = self; + let mut read_buf = ReadBuf::new(socket_recv_buffer.as_mut_slice()); + let packet_src = ready!(socket.poll_recv_from(cx, &mut read_buf)?); + let bytes = read_buf.filled().to_vec(); + Poll::Ready(Ok((bytes, packet_src))) } - async fn send_to(socket: &Self::Socket, buf: &[u8], addr: SocketAddr) -> io::Result { - socket.send_to(buf, addr).await + fn start_send(&mut self, data: Vec, addr: SocketAddr) { + self.next_packet_out = Some((data, addr)); } fn spawn(future: impl Future + Send + 'static) { From aff6c99481ede3ed750196e10563c9131f1c72bc Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 11:32:50 +0200 Subject: [PATCH 121/218] transports/quic: suggestions from code review Co-authored-by: Thomas Eizinger --- Cargo.toml | 2 +- transports/quic/Cargo.toml | 6 +++--- transports/quic/src/muxer.rs | 19 ++++++++++--------- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index f28a9042701..fe85dba4820 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -121,7 +121,7 @@ smallvec = "1.6.1" libp2p-deflate = { version = "0.37.0", path = "transports/deflate", optional = true } libp2p-dns = { version = "0.37.0", path = "transports/dns", optional = true } libp2p-mdns = { version = "0.41.0", path = "protocols/mdns", optional = true } -libp2p-quic = { version = "0.7.0-alpha", path = "transports/quic", default-features = false, optional = true } +libp2p-quic = { version = "0.7.0-alpha", path = "transports/quic", optional = true } libp2p-tcp = { version = "0.37.0", path = "transports/tcp", optional = true } libp2p-websocket = { version = "0.39.0", path = "transports/websocket", optional = true } diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 92ba3098d8a..8fce13497f4 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -29,12 +29,12 @@ x509-parser = "0.13.0" yasna = "0.5.0" [features] -tokio = ["tokio-crate"] -async-std = ["async-std-crate"] +tokio = ["dep:tokio-crate"] +async-std = ["dep:async-std-crate"] [dev-dependencies] async-std-crate = { package = "async-std", version = "1.12.0", features = ["attributes"] } -libp2p = { version = "0.49.0", default-features = false, features = ["request-response"], path = "../.." } +libp2p = { path = "../..", features = ["request-response"] } rand = "0.8.4" tokio-crate = { package = "tokio", version = "1.21.1", features = ["macros", "rt-multi-thread"] } quickcheck = "1" diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index d3ff0c7422d..be4c311b196 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -45,7 +45,7 @@ pub struct QuicMuxer { struct Inner { /// Inner connection object that yields events. connection: Connection, - // /// State of all the substreams that the muxer reports as open. + /// State of all the substreams that the muxer reports as open. substreams: HashMap, /// Waker to wake if a new outbound substream is opened. poll_outbound_waker: Option, @@ -142,11 +142,10 @@ impl StreamMuxer for QuicMuxer { } } } - inner.poll_connection_waker = Some(cx.waker().clone()); - // TODO: If connection migration is enabled (currently disabled) address // change on the connection needs to be handled. + inner.poll_connection_waker = Some(cx.waker().clone()); Poll::Pending } @@ -155,6 +154,7 @@ impl StreamMuxer for QuicMuxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); + let substream_id = match inner.connection.accept_substream() { Some(id) => { inner.poll_outbound_waker = None; @@ -167,6 +167,7 @@ impl StreamMuxer for QuicMuxer { }; inner.substreams.insert(substream_id, Default::default()); let substream = Substream::new(substream_id, self.inner.clone()); + Poll::Ready(Ok(substream)) } @@ -258,11 +259,8 @@ impl AsyncRead for Substream { 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(); - } + let chunk = match chunks.next(buf.len()) { + Ok(Some(chunk)) => chunk, Ok(None) => break, Err(err @ quinn_proto::ReadError::Reset(_)) => { return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) @@ -271,7 +269,10 @@ impl AsyncRead for Substream { pending = true; break; } - } + }; + + buf.write_all(&chunk.bytes).expect("enough buffer space"); + bytes += chunk.bytes.len(); } if chunks.finalize().should_transmit() { if let Some(waker) = muxer.poll_connection_waker.take() { From 4d0d14363e7502e2b5b42ba44ae28d307e3ce2c7 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 11:34:43 +0200 Subject: [PATCH 122/218] transports/quic: handle connection dropped --- transports/quic/src/endpoint.rs | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b0bfbf97070..87bfd2603cc 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -418,10 +418,20 @@ impl EndpointDriver

{ quinn_proto::DatagramEvent::ConnectionEvent(event) => { // Redirect the datagram to its connection. if let Some(sender) = self.alive_connections.get_mut(&connec_id) { - // Try to send the redirected datagramm event to the connection. - // If the connection is too busy we drop the datagram to back-pressure - // the remote. - let _ = sender.try_send(event); + match sender.try_send(event) { + Ok(()) => {} + Err(err) if err.is_disconnected() => { + // Connection was dropped by the user. + // Inform the endpoint that this connection is drained. + self.endpoint + .handle_event(connec_id, quinn_proto::EndpointEvent::drained()); + self.alive_connections.remove(&connec_id); + } + Err(err) if err.is_full() => { + // Connection is too busy. Drop the datagram to back-pressure the remote. + } + Err(_) => unreachable!("Error is either `Full` or `Disconnected`."), + } } else { log::error!("State mismatch: event for closed connection"); } From bcd0718380e3c5f810ac3a0a15dae9ea393ffa90 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 11:48:20 +0200 Subject: [PATCH 123/218] transports/quic: remove outdated comments --- transports/quic/src/endpoint.rs | 1 - transports/quic/src/lib.rs | 12 +++--------- transports/quic/src/muxer.rs | 4 ++-- 3 files changed, 5 insertions(+), 12 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 87bfd2603cc..37f88f2ebe0 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -90,7 +90,6 @@ impl Config { } } -/// Object containing all the QUIC resources shared between all connections. #[derive(Debug, Clone)] pub struct EndpointChannel { /// Channel to the background of the endpoint. diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 048233c8d2a..72945741f78 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -46,20 +46,14 @@ //! # } //! ``` //! -//! The [`Endpoint`] struct implements the [`Transport`] trait of the [`libp2p_core`] library. See the -//! documentation of [`libp2p_core`] and of libp2p in general to learn how to use the [`Transport`] trait. +//! The [`QuicTransport`] struct implements the [`libp2p_core::Transport`]. See the +//! documentation of [`libp2p_core`] and of libp2p in general to learn how to use the +//! [`Transport`][libp2p_core::Transport] trait. //! //! Note that QUIC provides transport, security, and multiplexing in a single protocol. Therefore, //! QUIC connections do not need to be upgraded. You will get a compile-time error if you try. //! Instead, you must pass all needed configuration into the constructor. //! -//! # Design Notes -//! -//! The entry point is the `Endpoint` struct. It represents a single QUIC endpoint. You -//! should generally have one of these per process. -//! -//! `Endpoint` manages a background task that processes all incoming packets. Each -//! `QuicConnection` also manages a background task, which handles socket output and timer polling. #![deny(unsafe_code)] diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index be4c311b196..f1e16dae320 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -154,7 +154,7 @@ impl StreamMuxer for QuicMuxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); - + let substream_id = match inner.connection.accept_substream() { Some(id) => { inner.poll_outbound_waker = None; @@ -167,7 +167,7 @@ impl StreamMuxer for QuicMuxer { }; inner.substreams.insert(substream_id, Default::default()); let substream = Substream::new(substream_id, self.inner.clone()); - + Poll::Ready(Ok(substream)) } From 85b8bb6e003067a8dc638269da2254925fd40afd Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 12:04:47 +0200 Subject: [PATCH 124/218] transports/quic: format --- transports/quic/src/lib.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 72945741f78..72cc65cc24f 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -47,7 +47,7 @@ //! ``` //! //! The [`QuicTransport`] struct implements the [`libp2p_core::Transport`]. See the -//! documentation of [`libp2p_core`] and of libp2p in general to learn how to use the +//! documentation of [`libp2p_core`] and of libp2p in general to learn how to use the //! [`Transport`][libp2p_core::Transport] trait. //! //! Note that QUIC provides transport, security, and multiplexing in a single protocol. Therefore, From 71595d06dad498361801e69a7f7bfbf171d5df14 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 12:12:19 +0200 Subject: [PATCH 125/218] transports/quic/Cargo.toml fix authors After 4a317d the code is now completely based on #1334, thus the authorship should be set accordingly. --- transports/quic/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 8fce13497f4..402891fc46a 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -1,7 +1,7 @@ [package] name = "libp2p-quic" version = "0.7.0-alpha" -authors = ["David Craven ", "Parity Technologies "] +authors = ["Parity Technologies "] edition = "2021" description = "TLS based QUIC transport implementation for libp2p" repository = "https://github.com/libp2p/rust-libp2p" From 1a7b0ff92dd5aabb5e263182a0efc91fe6834b13 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 12:40:39 +0200 Subject: [PATCH 126/218] transports/quic: handle dropping better Send events to the [`EndpointDriver`] when `Dialer`, `Listener`, or a [`Connection`] drops. --- transports/quic/src/connection.rs | 10 ++++++++ transports/quic/src/endpoint.rs | 39 +++++++++++++++++++++---------- transports/quic/src/transport.rs | 17 +++++++++++++- 3 files changed, 53 insertions(+), 13 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index fcd6c4839f7..c0cfd4f1547 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -315,6 +315,16 @@ impl fmt::Debug for Connection { } } +impl Drop for Connection { + fn drop(&mut self) { + let to_endpoint = ToEndpoint::ProcessConnectionEvent { + connection_id: self.connection_id, + event: quinn_proto::EndpointEvent::drained(), + }; + self.endpoint_channel.send_on_drop(to_endpoint); + } +} + /// Event generated by the [`Connection`]. #[derive(Debug)] pub enum ConnectionEvent { diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 37f88f2ebe0..b865b60ea39 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -179,6 +179,14 @@ impl EndpointChannel { }; self.to_endpoint.start_send(to_endpoint).map(Ok) } + + /// Send a message to inform the [`ConnectionDriver`] about an + /// event caused by the owner of this [`EndpointChannel`] dropping. + /// This clones the sender to the endpoint to guarantee delivery. + /// It this should *not* be called for regular messages. + pub fn send_on_drop(&mut self, to_endpoint: ToEndpoint) { + let _ = self.to_endpoint.clone().try_send(to_endpoint); + } } /// Message sent to the endpoint background task. @@ -200,6 +208,9 @@ pub enum ToEndpoint { }, /// Instruct the endpoint to send a packet of data on its UDP socket. SendUdpPacket(quinn_proto::Transmit), + /// The [`QuicTransport`] [`Dialer`] or [`Listener`] coupled to this endpoint was dropped. + /// Once all pending connection closed the [`EndpointDriver`] should shut down. + Decoupled, } /// Driver that runs in the background for as long as the endpoint is alive. Responsible for @@ -301,7 +312,7 @@ pub struct EndpointDriver { // `None` if server capabilities are disabled, i.e. the endpoint is only used for dialing. new_connection_tx: Option>, // Whether the transport dropped its handle for this endpoint. - is_orphaned: bool, + is_decoupled: bool, } impl EndpointDriver

{ @@ -323,7 +334,7 @@ impl EndpointDriver

{ next_packet_out: None, alive_connections: HashMap::new(), new_connection_tx, - is_orphaned: false, + is_decoupled: false, } } @@ -369,7 +380,7 @@ impl EndpointDriver

{ let is_drained_event = event.is_drained(); if is_drained_event { self.alive_connections.remove(&connection_id); - if self.is_orphaned && self.alive_connections.is_empty() { + if self.is_decoupled && self.alive_connections.is_empty() { log::info!( "Listener closed and no active connections remain. Shutting down the background task." ); @@ -396,6 +407,7 @@ impl EndpointDriver

{ // Data needs to be sent on the UDP socket. ToEndpoint::SendUdpPacket(transmit) => self.next_packet_out = Some(transmit), + ToEndpoint::Decoupled => self.handle_decoupling()?, } ControlFlow::Continue(()) } @@ -456,15 +468,7 @@ impl EndpointDriver

{ Ok(()) => { self.alive_connections.insert(connec_id, tx); } - Err(e) if e.is_disconnected() => { - if self.alive_connections.is_empty() { - return ControlFlow::Break(()); - } - // Listener was closed. - self.endpoint.reject_new_connections(); - self.new_connection_tx = None; - self.is_orphaned = true; - } + Err(e) if e.is_disconnected() => self.handle_decoupling()?, Err(_) => log::warn!( "Dropping new incoming connection {:?} because the channel to the listener is full", connec_id @@ -474,6 +478,17 @@ impl EndpointDriver

{ } ControlFlow::Continue(()) } + + fn handle_decoupling(&mut self) -> ControlFlow<()> { + if self.alive_connections.is_empty() { + return ControlFlow::Break(()); + } + // Listener was closed. + self.endpoint.reject_new_connections(); + self.new_connection_tx = None; + self.is_decoupled = true; + ControlFlow::Continue(()) + } } impl Future for EndpointDriver

{ diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index dd285052b0a..6db15047ebf 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -118,13 +118,16 @@ impl Transport for QuicTransport

{ let listener = Listener::new::

(listener_id, socket_addr, self.config.clone()) .map_err(CoreTransportError::Other)?; self.listeners.push(listener); + // Drop reference to dialer endpoint so that the endpoint is dropped once the last // connection that uses it is closed. // New outbound connections will use a bidirectional (listener) endpoint. - match socket_addr { + let dialer = match socket_addr { SocketAddr::V4(_) => self.ipv4_dialer.take(), SocketAddr::V6(_) => self.ipv6_dialer.take(), }; + std::mem::drop(dialer); + Ok(listener_id) } @@ -268,6 +271,12 @@ impl Dialer { } } +impl Drop for Dialer { + fn drop(&mut self) { + self.endpoint_channel.send_on_drop(ToEndpoint::Decoupled); + } +} + #[derive(Debug)] struct Listener { endpoint_channel: EndpointChannel, @@ -427,6 +436,12 @@ impl Stream for Listener { } } +impl Drop for Listener { + fn drop(&mut self) { + self.endpoint_channel.send_on_drop(ToEndpoint::Decoupled); + } +} + // Wrapped socket for non-blocking I/O operations. pub trait Provider: Unpin + Send + Sized + 'static { // Wrap a socket. From f2a9ebccb6445325235859f4b2f72eadccf017b9 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 13:01:46 +0200 Subject: [PATCH 127/218] transports/quic: only translate quic addresses --- transports/quic/src/transport.rs | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 6db15047ebf..b2b5b27844e 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -140,7 +140,10 @@ impl Transport for QuicTransport

{ } } - fn address_translation(&self, _server: &Multiaddr, observed: &Multiaddr) -> Option { + fn address_translation(&self, listen: &Multiaddr, observed: &Multiaddr) -> Option { + if !is_quic_addr(listen) || !is_quic_addr(observed) { + return None; + } Some(observed.clone()) } @@ -505,6 +508,26 @@ pub fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { } } +pub fn is_quic_addr(addr: &Multiaddr) -> bool { + use Protocol::*; + let mut iter = addr.iter(); + let first = match iter.next() { + Some(p) => p, + None => return false, + }; + let second = match iter.next() { + Some(p) => p, + None => return false, + }; + let third = match iter.next() { + Some(p) => p, + None => return false, + }; + matches!(first, Ip4(_) | Ip6(_) | Dns(_) | Dns4(_) | Dns6(_)) + && matches!(second, Udp(_)) + && matches!(third, Quic) +} + /// Turns an IP address and port into the corresponding QUIC multiaddr. pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { Multiaddr::empty() From 831383a9f7cc8b5496626457ced65b456d39464d Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 14:01:14 +0200 Subject: [PATCH 128/218] Cargo.toml: add "quic-tokio" to feature `full` --- Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/Cargo.toml b/Cargo.toml index fe85dba4820..ca6cf84c990 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -31,6 +31,7 @@ full = [ "plaintext", "pnet", "quic-async-std", + "quic-tokio", "relay", "rendezvous", "request-response", From 8e729bf7148a24db026e1e81fe2532cc6c8223c8 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 14:30:16 +0200 Subject: [PATCH 129/218] transports/quic: follow naming convention of #2217 --- transports/quic/src/endpoint.rs | 6 ++--- transports/quic/src/lib.rs | 14 ++++++------ transports/quic/src/transport.rs | 22 ++++++++---------- transports/quic/src/transport/async_std.rs | 13 ++++++----- transports/quic/src/transport/tokio.rs | 12 +++++----- transports/quic/tests/smoke.rs | 26 +++++++++------------- 6 files changed, 43 insertions(+), 50 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b865b60ea39..c5d20640199 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -208,7 +208,7 @@ pub enum ToEndpoint { }, /// Instruct the endpoint to send a packet of data on its UDP socket. SendUdpPacket(quinn_proto::Transmit), - /// The [`QuicTransport`] [`Dialer`] or [`Listener`] coupled to this endpoint was dropped. + /// The [`GenTransport`] [`Dialer`] or [`Listener`] coupled to this endpoint was dropped. /// Once all pending connection closed the [`EndpointDriver`] should shut down. Decoupled, } @@ -239,7 +239,7 @@ pub enum ToEndpoint { /// - One channel per each existing connection that communicates messages from the [`EndpointDriver`] /// to that [`Connection`]. /// - One channel for the [`EndpointDriver`] to send newly-opened connections to. The receiving -/// side is processed by the [`QuicTransport`][crate::QuicTransport]. +/// side is processed by the [`GenTransport`][crate::GenTransport]. /// /// In order to avoid an unbounded buffering of events, we prioritize sending data on the UDP /// socket over everything else. If the network interface is too busy to process our packets, @@ -338,7 +338,7 @@ impl EndpointDriver

{ } } - /// Handle a message sent from either the [`QuicTransport`](super::QuicTransport) or a [`Connection`]. + /// Handle a message sent from either the [`GenTransport`](super::GenTransport) or a [`Connection`]. fn handle_message(&mut self, to_endpoint: ToEndpoint) -> ControlFlow<()> { match to_endpoint { ToEndpoint::Dial { addr, result } => { diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 72cc65cc24f..c740bb8ae71 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -31,13 +31,13 @@ //! # #[cfg(feature = "async-std")] //! # fn main() -> std::io::Result<()> { //! # -//! use libp2p_quic::{AsyncStdTransport, Config}; +//! use libp2p_quic as quic; //! use libp2p_core::{Multiaddr, Transport}; //! //! let keypair = libp2p_core::identity::Keypair::generate_ed25519(); -//! let quic_config = Config::new(&keypair).expect("could not make config"); +//! let quic_config = quic::Config::new(&keypair).expect("could not make config"); //! -//! let mut quic_transport = AsyncStdTransport::new(quic_config); +//! let mut quic_transport = quic::async_std::Transport::new(quic_config); //! //! let addr = "/ip4/127.0.0.1/udp/12345/quic".parse().expect("bad address?"); //! quic_transport.listen_on(addr).expect("listen error."); @@ -46,7 +46,7 @@ //! # } //! ``` //! -//! The [`QuicTransport`] struct implements the [`libp2p_core::Transport`]. See the +//! The [`GenTransport`] struct implements the [`libp2p_core::Transport`]. See the //! documentation of [`libp2p_core`] and of libp2p in general to learn how to use the //! [`Transport`][libp2p_core::Transport] trait. //! @@ -69,8 +69,8 @@ pub use endpoint::Config; pub use muxer::QuicMuxer; pub use quinn_proto::ConnectError as DialError; #[cfg(feature = "async-std")] -pub use transport::{AsyncStd, AsyncStdTransport}; -pub use transport::{Provider, QuicTransport, TransportError}; +pub use transport::async_std; #[cfg(feature = "tokio")] -pub use transport::{Tokio, TokioTransport}; +pub use transport::tokio; +pub use transport::{GenTransport, Provider, TransportError}; pub use upgrade::Upgrade; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index b2b5b27844e..51ec96e96e2 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -28,14 +28,10 @@ use crate::{endpoint::EndpointChannel, muxer::QuicMuxer, upgrade::Upgrade}; use crate::{Config, ConnectionError}; #[cfg(feature = "async-std")] -mod async_std; +pub mod async_std; #[cfg(feature = "tokio")] -mod tokio; -#[cfg(feature = "async-std")] -pub use async_std::{AsyncStd, AsyncStdTransport}; +pub mod tokio; use futures::future::BoxFuture; -#[cfg(feature = "tokio")] -pub use tokio::{Tokio, TokioTransport}; use futures::channel::{mpsc, oneshot}; use futures::ready; @@ -61,7 +57,7 @@ use std::{ }; #[derive(Debug)] -pub struct QuicTransport

{ +pub struct GenTransport

{ config: Config, listeners: SelectAll, /// Dialer for Ipv4 addresses if no matching listener exists. @@ -72,7 +68,7 @@ pub struct QuicTransport

{ _marker: PhantomData

, } -impl

QuicTransport

{ +impl

GenTransport

{ pub fn new(config: Config) -> Self { Self { listeners: SelectAll::new(), @@ -102,7 +98,7 @@ pub enum TransportError { EndpointDriverCrashed, } -impl Transport for QuicTransport

{ +impl Transport for GenTransport

{ type Output = (PeerId, QuicMuxer); type Error = TransportError; type ListenerUpgrade = Upgrade; @@ -615,7 +611,7 @@ mod test { async fn tokio_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); let config = Config::new(&keypair).unwrap(); - let transport = TokioTransport::new(config.clone()); + let transport = super::tokio::Transport::new(config.clone()); test_close_listener(transport).await } @@ -624,16 +620,16 @@ mod test { async fn async_std_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); let config = Config::new(&keypair).unwrap(); - let transport = AsyncStdTransport::new(config.clone()); + let transport = super::async_std::Transport::new(config.clone()); test_close_listener(transport).await } - async fn test_close_listener(mut transport: QuicTransport

) { + async fn test_close_listener(mut transport: GenTransport

) { assert!(poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)) .now_or_never() .is_none()); - // Run test twice to check that there is no unexpected behaviour if `QuicTransport.listener` + // Run test twice to check that there is no unexpected behaviour if `GenTransport.listener` // is temporarily empty. for _ in 0..2 { let listener = transport diff --git a/transports/quic/src/transport/async_std.rs b/transports/quic/src/transport/async_std.rs index e22fa82987f..34bfd22ac0e 100644 --- a/transports/quic/src/transport/async_std.rs +++ b/transports/quic/src/transport/async_std.rs @@ -29,22 +29,23 @@ use std::{ use async_std_crate::{net::UdpSocket, task::spawn}; use futures::{future::BoxFuture, ready, Future, FutureExt, Stream, StreamExt}; -use crate::QuicTransport; +use crate::GenTransport; -use super::Provider; +use super::Provider as ProviderTrait; -pub type AsyncStdTransport = QuicTransport; -pub struct AsyncStd { +pub type Transport = GenTransport; + +pub struct Provider { socket: Arc, send_packet: Option>>, recv_stream: ReceiveStream, } -impl Provider for AsyncStd { +impl ProviderTrait for Provider { fn from_socket(socket: std::net::UdpSocket) -> io::Result { let socket = Arc::new(socket.into()); let recv_stream = ReceiveStream::new(Arc::clone(&socket)); - Ok(AsyncStd { + Ok(Provider { socket, send_packet: None, recv_stream, diff --git a/transports/quic/src/transport/tokio.rs b/transports/quic/src/transport/tokio.rs index bdb6ef4df6a..09bda1e9881 100644 --- a/transports/quic/src/transport/tokio.rs +++ b/transports/quic/src/transport/tokio.rs @@ -28,21 +28,21 @@ use futures::{ready, Future}; use tokio_crate::{io::ReadBuf, net::UdpSocket}; use x509_parser::nom::AsBytes; -use crate::QuicTransport; +use crate::GenTransport; -use super::Provider; +use super::Provider as ProviderTrait; -pub type TokioTransport = QuicTransport; -pub struct Tokio { +pub type Transport = GenTransport; +pub struct Provider { socket: UdpSocket, socket_recv_buffer: Vec, next_packet_out: Option<(Vec, SocketAddr)>, } -impl Provider for Tokio { +impl ProviderTrait for Provider { fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { let socket = UdpSocket::from_std(socket)?; - Ok(Tokio { + Ok(Provider { socket, socket_recv_buffer: vec![0; 65536], next_packet_out: None, diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index d94c1000da5..2a9f7ca696f 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -25,10 +25,6 @@ use std::{io, iter}; #[cfg(feature = "async-std")] use async_std_crate as async_std; -#[cfg(feature = "async-std")] -use libp2p_quic::AsyncStd; -#[cfg(feature = "tokio")] -use libp2p_quic::Tokio; #[cfg(feature = "tokio")] use tokio_crate as tokio; @@ -40,7 +36,7 @@ async fn create_swarm() -> Swarm> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let config = quic::Config::new(&keypair).unwrap(); - let transport = quic::QuicTransport::

::new(config); + let transport = quic::GenTransport::

::new(config); let transport = Transport::map(transport, |(peer, muxer), _| { (peer, StreamMuxerBox::new(muxer)) @@ -64,13 +60,13 @@ async fn start_listening(swarm: &mut Swarm>, addr: &s #[cfg(feature = "tokio")] #[tokio::test] async fn tokio_smoke() { - smoke::().await + smoke::().await } #[cfg(feature = "async-std")] #[async_std::test] async fn async_std_smoke() { - smoke::().await + smoke::().await } async fn smoke() { @@ -295,8 +291,8 @@ impl RequestResponseCodec for PingCodec { #[cfg(feature = "async-std")] #[async_std::test] async fn dial_failure() { - let mut a = create_swarm::().await; - let mut b = create_swarm::().await; + let mut a = create_swarm::().await; + let mut b = create_swarm::().await; let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; @@ -445,10 +441,10 @@ fn concurrent_connections_and_streams() { #[cfg(feature = "tokio")] tokio::runtime::Runtime::new() .unwrap() - .block_on(prop::(num_listener, num_streams)); + .block_on(prop::(num_listener, num_streams)); #[cfg(feature = "async-std")] - async_std::task::block_on(prop::(num_listener, num_streams)); + async_std::task::block_on(prop::(num_listener, num_streams)); // QuickCheck::new().quickcheck(prop as fn(_, _) -> _); } @@ -456,8 +452,8 @@ fn concurrent_connections_and_streams() { #[cfg(feature = "tokio")] #[tokio::test] async fn endpoint_reuse() { - let mut swarm_a = create_swarm::().await; - let mut swarm_b = create_swarm::().await; + let mut swarm_a = create_swarm::().await; + let mut swarm_b = create_swarm::().await; let b_peer_id = *swarm_b.local_peer_id(); let a_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/udp/0/quic").await; @@ -547,8 +543,8 @@ async fn endpoint_reuse() { #[cfg(feature = "async-std")] #[async_std::test] async fn ipv4_dial_ipv6() { - let mut swarm_a = create_swarm::().await; - let mut swarm_b = create_swarm::().await; + let mut swarm_a = create_swarm::().await; + let mut swarm_b = create_swarm::().await; let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; From 49e359b7cd7184b4e86d58dbebcbaaa4d027519e Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 14:32:36 +0200 Subject: [PATCH 130/218] transports/quic: rename `Upgrade` -> `Connecting` --- transports/quic/src/connection.rs | 2 +- transports/quic/src/lib.rs | 2 +- transports/quic/src/transport.rs | 10 +++++----- transports/quic/src/upgrade.rs | 12 ++++++------ 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index c0cfd4f1547..7109f92cb4d 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -38,7 +38,7 @@ use std::{ time::Instant, }; -/// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Upgrade`]. +/// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Connecting`]. /// /// Contains everything needed to process a connection with a remote. /// Tied to a specific endpoint. diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index c740bb8ae71..26680fa450f 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -73,4 +73,4 @@ pub use transport::async_std; #[cfg(feature = "tokio")] pub use transport::tokio; pub use transport::{GenTransport, Provider, TransportError}; -pub use upgrade::Upgrade; +pub use upgrade::Connecting; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 51ec96e96e2..22335d11117 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -24,7 +24,7 @@ use crate::connection::Connection; use crate::endpoint::ToEndpoint; -use crate::{endpoint::EndpointChannel, muxer::QuicMuxer, upgrade::Upgrade}; +use crate::{endpoint::EndpointChannel, muxer::QuicMuxer, upgrade::Connecting}; use crate::{Config, ConnectionError}; #[cfg(feature = "async-std")] @@ -101,7 +101,7 @@ pub enum TransportError { impl Transport for GenTransport

{ type Output = (PeerId, QuicMuxer); type Error = TransportError; - type ListenerUpgrade = Upgrade; + type ListenerUpgrade = Connecting; type Dial = BoxFuture<'static, Result>; fn listen_on( @@ -195,7 +195,7 @@ impl Transport for GenTransport

{ .await .map_err(|_| TransportError::EndpointDriverCrashed)? .map_err(TransportError::Reach)?; - let final_connec = Upgrade::from_connection(connection).await?; + let final_connec = Connecting::from_connection(connection).await?; Ok(final_connec) } .boxed()) @@ -388,7 +388,7 @@ impl Listener { } impl Stream for Listener { - type Item = TransportEvent; + type Item = TransportEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { if let Some(event) = self.pending_event.take() { @@ -416,7 +416,7 @@ impl Stream for Listener { let local_addr = socketaddr_to_multiaddr(connection.local_addr()); let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); let event = TransportEvent::Incoming { - upgrade: Upgrade::from_connection(connection), + upgrade: Connecting::from_connection(connection), local_addr, send_back_addr, listener_id: self.listener_id, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 12a7f1c3db8..0926339494b 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -35,20 +35,20 @@ use std::{ }; /// A QUIC connection currently being negotiated. -pub struct Upgrade { +pub struct Connecting { connection: Option, } -impl Upgrade { - /// Builds an [`Upgrade`] that wraps around a [`Connection`]. +impl Connecting { + /// Builds an [`Connecting`] that wraps around a [`Connection`]. pub(crate) fn from_connection(connection: Connection) -> Self { - Upgrade { + Connecting { connection: Some(connection), } } } -impl Future for Upgrade { +impl Future for Connecting { type Output = Result<(PeerId, QuicMuxer), transport::TransportError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -79,7 +79,7 @@ impl Future for Upgrade { } } -impl fmt::Debug for Upgrade { +impl fmt::Debug for Connecting { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { fmt::Debug::fmt(&self.connection, f) } From 03c4da9b4caeeedf7575b3b45e1b0d3afb03d668 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 14:35:24 +0200 Subject: [PATCH 131/218] transports/quic: fix intra-doc-links --- transports/quic/src/endpoint.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index c5d20640199..91732dbac74 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -180,7 +180,7 @@ impl EndpointChannel { self.to_endpoint.start_send(to_endpoint).map(Ok) } - /// Send a message to inform the [`ConnectionDriver`] about an + /// Send a message to inform the [`EndpointDriver`] about an /// event caused by the owner of this [`EndpointChannel`] dropping. /// This clones the sender to the endpoint to guarantee delivery. /// It this should *not* be called for regular messages. @@ -208,7 +208,8 @@ pub enum ToEndpoint { }, /// Instruct the endpoint to send a packet of data on its UDP socket. SendUdpPacket(quinn_proto::Transmit), - /// The [`GenTransport`] [`Dialer`] or [`Listener`] coupled to this endpoint was dropped. + /// The [`GenTransport`][transport::GenTransport] dialer or listener coupled to this endpoint + /// was dropped. /// Once all pending connection closed the [`EndpointDriver`] should shut down. Decoupled, } From ec21fb1fcada6709a1b6a3bdb896a93d7c955ee0 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 14:49:23 +0200 Subject: [PATCH 132/218] transports/quic: derive `Debug` where possible --- transports/quic/src/connection.rs | 8 +------- transports/quic/src/endpoint.rs | 1 + transports/quic/src/muxer.rs | 1 + transports/quic/src/upgrade.rs | 8 +------- 4 files changed, 4 insertions(+), 14 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 7109f92cb4d..4b5fba32e3f 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -32,7 +32,6 @@ use futures::{channel::mpsc, prelude::*}; use futures_timer::Delay; use libp2p_core::PeerId; use std::{ - fmt, net::SocketAddr, task::{Context, Poll}, time::Instant, @@ -42,6 +41,7 @@ use std::{ /// /// Contains everything needed to process a connection with a remote. /// Tied to a specific endpoint. +#[derive(Debug)] pub struct Connection { /// Channel to the endpoint this connection belongs to. endpoint_channel: EndpointChannel, @@ -309,12 +309,6 @@ impl Connection { } } -impl fmt::Debug for Connection { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_tuple("Connection").finish() - } -} - impl Drop for Connection { fn drop(&mut self) { let to_endpoint = ToEndpoint::ProcessConnectionEvent { diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 91732dbac74..525d8ad42a5 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -291,6 +291,7 @@ pub enum ToEndpoint { /// guarantees that the [`EndpointDriver`], is properly kept alive for as long as any QUIC /// connection is open. /// +#[derive(Debug)] pub struct EndpointDriver { // The actual QUIC state machine. endpoint: quinn_proto::Endpoint, diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index f1e16dae320..26b6e21ee70 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -222,6 +222,7 @@ impl StreamMuxer for QuicMuxer { } } +#[derive(Debug)] pub struct Substream { id: quinn_proto::StreamId, muxer: Arc>, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 0926339494b..0712ebf381d 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -29,12 +29,12 @@ use crate::{ use futures::{prelude::*, ready}; use libp2p_core::PeerId; use std::{ - fmt, pin::Pin, task::{Context, Poll}, }; /// A QUIC connection currently being negotiated. +#[derive(Debug)] pub struct Connecting { connection: Option, } @@ -78,9 +78,3 @@ impl Future for Connecting { } } } - -impl fmt::Debug for Connecting { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - fmt::Debug::fmt(&self.connection, f) - } -} From 6f284ccebd8785faa3223b497692bf93b0e673ac Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 15:13:27 +0200 Subject: [PATCH 133/218] transports/quic: apply suggestions from review --- transports/quic/src/endpoint.rs | 2 +- transports/quic/src/muxer.rs | 9 +++------ transports/quic/src/transport.rs | 22 +++++++++++++--------- transports/quic/src/upgrade.rs | 4 +--- 4 files changed, 18 insertions(+), 19 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 525d8ad42a5..b16216c4032 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -208,7 +208,7 @@ pub enum ToEndpoint { }, /// Instruct the endpoint to send a packet of data on its UDP socket. SendUdpPacket(quinn_proto::Transmit), - /// The [`GenTransport`][transport::GenTransport] dialer or listener coupled to this endpoint + /// The [`GenTransport`][transport::GenTransport] dialer or listener coupled to this endpoint /// was dropped. /// Once all pending connection closed the [`EndpointDriver`] should shut down. Decoupled, diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 26b6e21ee70..3e91b995882 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -23,7 +23,7 @@ use crate::{ ConnectionError, }; -use futures::{AsyncRead, AsyncWrite}; +use futures::{ready, AsyncRead, AsyncWrite}; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ @@ -212,13 +212,10 @@ impl StreamMuxer for QuicMuxer { if connection.send_stream_count() == 0 && !connection.is_closed() { connection.close() } - match connection.poll_event(cx) { - Poll::Ready(ConnectionEvent::ConnectionLost(_)) => return Poll::Ready(Ok(())), - Poll::Ready(_) => {} - Poll::Pending => break, + if let ConnectionEvent::ConnectionLost(_) = ready!(connection.poll_event(cx)) { + return Poll::Ready(Ok(())); } } - Poll::Pending } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 22335d11117..bb151119b1e 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -84,13 +84,13 @@ impl

GenTransport

{ #[derive(Debug, thiserror::Error)] pub enum TransportError { /// Error while trying to reach a remote. - #[error("{0}")] - Reach(quinn_proto::ConnectError), + #[error(transparent)] + Reach(#[from] quinn_proto::ConnectError), /// Error after the remote has been reached. - #[error("{0}")] - Established(ConnectionError), + #[error(transparent)] + Established(#[from] ConnectionError), - #[error("{0}")] + #[error(transparent)] Io(#[from] std::io::Error), /// The task driving the endpoint has crashed. @@ -111,8 +111,7 @@ impl Transport for GenTransport

{ let socket_addr = multiaddr_to_socketaddr(&addr) .ok_or(CoreTransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); - let listener = Listener::new::

(listener_id, socket_addr, self.config.clone()) - .map_err(CoreTransportError::Other)?; + let listener = Listener::new::

(listener_id, socket_addr, self.config.clone())?; self.listeners.push(listener); // Drop reference to dialer endpoint so that the endpoint is dropped once the last @@ -193,8 +192,7 @@ impl Transport for GenTransport

{ Ok(async move { let connection = rx .await - .map_err(|_| TransportError::EndpointDriverCrashed)? - .map_err(TransportError::Reach)?; + .map_err(|_| TransportError::EndpointDriverCrashed)??; let final_connec = Connecting::from_connection(connection).await?; Ok(final_connec) } @@ -237,6 +235,12 @@ impl Transport for GenTransport

{ } } +impl From for CoreTransportError { + fn from(err: TransportError) -> Self { + CoreTransportError::Other(err) + } +} + #[derive(Debug)] struct Dialer { endpoint_channel: EndpointChannel, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 0712ebf381d..5e47f4d89d2 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -64,9 +64,7 @@ impl Future for Connecting { let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } - ConnectionEvent::ConnectionLost(err) => { - return Poll::Ready(Err(transport::TransportError::Established(err))) - } + ConnectionEvent::ConnectionLost(err) => return Poll::Ready(Err(err.into())), ConnectionEvent::HandshakeDataReady | ConnectionEvent::StreamAvailable | ConnectionEvent::StreamOpened From 7c332bb5a50e4489fdbd3a42f818b40c8a8096bf Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 17:24:26 +0200 Subject: [PATCH 134/218] transports/quic: move provider into own mod --- transports/quic/src/endpoint.rs | 6 +-- transports/quic/src/lib.rs | 10 ++-- transports/quic/src/provider.rs | 54 +++++++++++++++++++ .../src/{transport => provider}/async_std.rs | 0 .../quic/src/{transport => provider}/tokio.rs | 0 transports/quic/src/transport.rs | 38 ++----------- 6 files changed, 66 insertions(+), 42 deletions(-) create mode 100644 transports/quic/src/provider.rs rename transports/quic/src/{transport => provider}/async_std.rs (100%) rename transports/quic/src/{transport => provider}/tokio.rs (100%) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b16216c4032..31a6e12ded8 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,11 +28,7 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`EndpointDriver`] for a thorough description. -use crate::{ - connection::Connection, - tls, - transport::{self, Provider}, -}; +use crate::{connection::Connection, provider::Provider, tls, transport}; use bytes::BytesMut; use futures::{ diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 26680fa450f..c0c5a4f3b6b 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -60,6 +60,7 @@ mod connection; mod endpoint; mod muxer; +mod provider; mod tls; mod transport; mod upgrade; @@ -67,10 +68,11 @@ mod upgrade; pub use connection::ConnectionError; pub use endpoint::Config; pub use muxer::QuicMuxer; -pub use quinn_proto::ConnectError as DialError; #[cfg(feature = "async-std")] -pub use transport::async_std; +pub use provider::async_std; #[cfg(feature = "tokio")] -pub use transport::tokio; -pub use transport::{GenTransport, Provider, TransportError}; +pub use provider::tokio; +pub use provider::Provider; +pub use quinn_proto::ConnectError as DialError; +pub use transport::{GenTransport, TransportError}; pub use upgrade::Connecting; diff --git a/transports/quic/src/provider.rs b/transports/quic/src/provider.rs new file mode 100644 index 00000000000..6f067bd4c61 --- /dev/null +++ b/transports/quic/src/provider.rs @@ -0,0 +1,54 @@ +// Copyright 2022 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 futures::Future; +use std::{ + io, + net::SocketAddr, + task::{Context, Poll}, +}; + +#[cfg(feature = "async-std")] +pub mod async_std; +#[cfg(feature = "tokio")] +pub mod tokio; + +// Wrapped socket for non-blocking I/O operations. +pub trait Provider: Unpin + Send + Sized + 'static { + // Wrap a socket. + // Note: The socket must be set to non-blocking. + fn from_socket(socket: std::net::UdpSocket) -> io::Result; + + // Receive a single datagram message. + // Returns the message and the address the message came from. + fn poll_recv_from(&mut self, cx: &mut Context<'_>) -> Poll, SocketAddr)>>; + + // Set sending the packet on the socket. + // Since only one packet may be sent at a time, this may only be called if a preceding call + // to [`Provider::poll_send_flush`] returned [`Poll::Ready`]. + fn start_send(&mut self, data: Vec, addr: SocketAddr); + + // Flush a packet send in [`Provider::start_send`]. + // If [`Poll::Ready`] is returned the socket is ready for sending a new packet. + fn poll_send_flush(&mut self, cx: &mut Context<'_>) -> Poll>; + + // Run the given future in the background until it ends. + fn spawn(future: impl Future + Send + 'static); +} diff --git a/transports/quic/src/transport/async_std.rs b/transports/quic/src/provider/async_std.rs similarity index 100% rename from transports/quic/src/transport/async_std.rs rename to transports/quic/src/provider/async_std.rs diff --git a/transports/quic/src/transport/tokio.rs b/transports/quic/src/provider/tokio.rs similarity index 100% rename from transports/quic/src/transport/tokio.rs rename to transports/quic/src/provider/tokio.rs diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index bb151119b1e..e73e0500ceb 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -24,16 +24,12 @@ use crate::connection::Connection; use crate::endpoint::ToEndpoint; +use crate::provider::Provider; use crate::{endpoint::EndpointChannel, muxer::QuicMuxer, upgrade::Connecting}; use crate::{Config, ConnectionError}; -#[cfg(feature = "async-std")] -pub mod async_std; -#[cfg(feature = "tokio")] -pub mod tokio; -use futures::future::BoxFuture; - use futures::channel::{mpsc, oneshot}; +use futures::future::BoxFuture; use futures::ready; use futures::stream::StreamExt; use futures::{prelude::*, stream::SelectAll}; @@ -46,9 +42,8 @@ use libp2p_core::{ PeerId, Transport, }; use std::collections::VecDeque; -use std::io; use std::marker::PhantomData; -use std::net::{IpAddr, UdpSocket}; +use std::net::IpAddr; use std::task::Waker; use std::{ net::SocketAddr, @@ -445,29 +440,6 @@ impl Drop for Listener { } } -// Wrapped socket for non-blocking I/O operations. -pub trait Provider: Unpin + Send + Sized + 'static { - // Wrap a socket. - // Note: The socket must be set to non-blocking. - fn from_socket(socket: UdpSocket) -> io::Result; - - // Receive a single datagram message. - // Returns the message and the address the message came from. - fn poll_recv_from(&mut self, cx: &mut Context<'_>) -> Poll, SocketAddr)>>; - - // Set sending the packet on the socket. - // Since only one packet may be sent at a time, this may only be called if a preceding call - // to [`Provider::poll_send_flush`] returned [`Poll::Ready`]. - fn start_send(&mut self, data: Vec, addr: SocketAddr); - - // Flush a packet send in [`Provider::start_send`]. - // If [`Poll::Ready`] is returned the socket is ready for sending a new packet. - fn poll_send_flush(&mut self, cx: &mut Context<'_>) -> Poll>; - - // Run the given future in the background until it ends. - fn spawn(future: impl Future + Send + 'static); -} - /// Turn an [`IpAddr`] into a listen-address for the endpoint. /// /// Returns `None` if the address is not the same socket family as the @@ -615,7 +587,7 @@ mod test { async fn tokio_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); let config = Config::new(&keypair).unwrap(); - let transport = super::tokio::Transport::new(config.clone()); + let transport = crate::tokio::Transport::new(config.clone()); test_close_listener(transport).await } @@ -624,7 +596,7 @@ mod test { async fn async_std_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); let config = Config::new(&keypair).unwrap(); - let transport = super::async_std::Transport::new(config.clone()); + let transport = crate::async_std::Transport::new(config.clone()); test_close_listener(transport).await } From 0b80573c4c4b53eedccb132619cb751c2c8c2459 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 18:05:51 +0200 Subject: [PATCH 135/218] transports/quic: add `SocketFamily` enum --- transports/quic/src/endpoint.rs | 16 ++++-- transports/quic/src/transport.rs | 94 +++++++++++++++++--------------- 2 files changed, 61 insertions(+), 49 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 31a6e12ded8..73f2bdacd8d 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,7 +28,12 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`EndpointDriver`] for a thorough description. -use crate::{connection::Connection, provider::Provider, tls, transport}; +use crate::{ + connection::Connection, + provider::Provider, + tls, + transport::{self, SocketFamily}, +}; use bytes::BytesMut; use futures::{ @@ -109,12 +114,11 @@ impl EndpointChannel { /// Builds a new endpoint that only supports outbound connections. pub fn new_dialer( config: Config, - is_ipv6: bool, + socket_family: SocketFamily, ) -> Result { - let socket_addr = if is_ipv6 { - SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0) - } else { - SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), 0) + let socket_addr = match socket_family { + SocketFamily::Ipv4 => SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), 0), + SocketFamily::Ipv6 => SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0), }; Self::new::

(config, socket_addr, None) } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index e73e0500ceb..072818e2dbd 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -41,7 +41,7 @@ use libp2p_core::{ transport::{ListenerId, TransportError as CoreTransportError, TransportEvent}, PeerId, Transport, }; -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; use std::net::IpAddr; use std::task::Waker; @@ -55,10 +55,8 @@ use std::{ pub struct GenTransport

{ config: Config, listeners: SelectAll, - /// Dialer for Ipv4 addresses if no matching listener exists. - ipv4_dialer: Option, - /// Dialer for Ipv6 addresses if no matching listener exists. - ipv6_dialer: Option, + /// Dialer for each socket family if no matching listener exists. + dialer: HashMap, _marker: PhantomData

, } @@ -68,8 +66,7 @@ impl

GenTransport

{ Self { listeners: SelectAll::new(), config, - ipv4_dialer: None, - ipv6_dialer: None, + dialer: HashMap::new(), _marker: Default::default(), } } @@ -109,14 +106,10 @@ impl Transport for GenTransport

{ let listener = Listener::new::

(listener_id, socket_addr, self.config.clone())?; self.listeners.push(listener); - // Drop reference to dialer endpoint so that the endpoint is dropped once the last + // Remove dialer endpoint so that the endpoint is dropped once the last // connection that uses it is closed. // New outbound connections will use a bidirectional (listener) endpoint. - let dialer = match socket_addr { - SocketAddr::V4(_) => self.ipv4_dialer.take(), - SocketAddr::V6(_) => self.ipv6_dialer.take(), - }; - std::mem::drop(dialer); + self.dialer.remove(&socket_addr.ip().into()); Ok(listener_id) } @@ -148,7 +141,7 @@ impl Transport for GenTransport

{ .iter_mut() .filter(|l| { let listen_addr = l.endpoint_channel.socket_addr(); - listen_addr.is_ipv4() == socket_addr.is_ipv4() + SocketFamily::is_same(&listen_addr.ip(), &socket_addr.ip()) && listen_addr.ip().is_loopback() == socket_addr.ip().is_loopback() }) .collect::>(); @@ -159,21 +152,15 @@ impl Transport for GenTransport

{ result: tx, }; if listeners.is_empty() { - let dialer = match socket_addr { - SocketAddr::V4(_) => &mut self.ipv4_dialer, - SocketAddr::V6(_) => &mut self.ipv6_dialer, + let socket_family = socket_addr.ip().into(); + let dialer = match self.dialer.get_mut(&socket_family) { + Some(dialer) => dialer, + None => { + let dialer = Dialer::new::

(self.config.clone(), socket_family)?; + self.dialer.entry(socket_family).or_insert(dialer) + } }; - if dialer.is_none() { - let _ = dialer.insert(Dialer::new::

( - self.config.clone(), - socket_addr.is_ipv6(), - )?); - } - dialer - .as_mut() - .unwrap() - .pending_dials - .push_back(to_endpoint); + dialer.pending_dials.push_back(to_endpoint); } else { // Pick a random listener to use for dialing. let n = rand::random::() % listeners.len(); @@ -209,19 +196,16 @@ impl Transport for GenTransport

{ mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - if let Some(dialer) = self.ipv4_dialer.as_mut() { + let mut errored = Vec::new(); + for (key, dialer) in &mut self.dialer { if dialer.drive_dials(cx).is_err() { - // Background task of dialer crashed. - // Drop dialer and all pending dials so that the connection receiver is notified. - self.ipv4_dialer = None; + errored.push(*key); } } - if let Some(dialer) = self.ipv6_dialer.as_mut() { - if dialer.drive_dials(cx).is_err() { - // Background task of dialer crashed. - // Drop dialer and all pending dials so that the connection receiver is notified. - self.ipv4_dialer = None; - } + for key in errored { + // Background task of dialer crashed. + // Drop dialer and all pending dials so that the connection receiver is notified. + self.dialer.remove(&key); } match self.listeners.poll_next_unpin(cx) { Poll::Ready(Some(ev)) => Poll::Ready(ev), @@ -245,10 +229,10 @@ struct Dialer { impl Dialer { fn new( config: Config, - is_ipv6: bool, + socket_family: SocketFamily, ) -> Result> { - let endpoint_channel = - EndpointChannel::new_dialer::

(config, is_ipv6).map_err(CoreTransportError::Other)?; + let endpoint_channel = EndpointChannel::new_dialer::

(config, socket_family) + .map_err(CoreTransportError::Other)?; Ok(Dialer { endpoint_channel, pending_dials: VecDeque::new(), @@ -440,14 +424,38 @@ impl Drop for Listener { } } +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub enum SocketFamily { + Ipv4, + Ipv6, +} + +impl SocketFamily { + fn is_same(a: &IpAddr, b: &IpAddr) -> bool { + match (a, b) { + (IpAddr::V4(_), IpAddr::V4(_)) => true, + (IpAddr::V6(_), IpAddr::V6(_)) => true, + _ => false, + } + } +} + +impl From for SocketFamily { + fn from(ip: IpAddr) -> Self { + match ip { + IpAddr::V4(_) => SocketFamily::Ipv4, + IpAddr::V6(_) => SocketFamily::Ipv6, + } + } +} + /// Turn an [`IpAddr`] into a listen-address for the endpoint. /// /// Returns `None` if the address is not the same socket family as the /// address that the endpoint is bound to. fn ip_to_listenaddr(endpoint_addr: &SocketAddr, ip: IpAddr) -> Option { // True if either both addresses are Ipv4 or both Ipv6. - let is_same_ip_family = endpoint_addr.is_ipv4() == ip.is_ipv4(); - if !is_same_ip_family { + if !SocketFamily::is_same(&endpoint_addr.ip(), &ip) { return None; } let socket_addr = SocketAddr::new(ip, endpoint_addr.port()); From eafa89edbf3cb7d77fc4d08efbbcfebb31a93364 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Tue, 4 Oct 2022 22:42:52 +0200 Subject: [PATCH 136/218] transports/quic: wake on new pending dial --- transports/quic/src/endpoint.rs | 2 +- transports/quic/src/transport.rs | 37 ++++++++++++++++++-------------- 2 files changed, 22 insertions(+), 17 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 73f2bdacd8d..6dd6a8e0121 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -384,7 +384,7 @@ impl EndpointDriver

{ self.alive_connections.remove(&connection_id); if self.is_decoupled && self.alive_connections.is_empty() { log::info!( - "Listener closed and no active connections remain. Shutting down the background task." + "Driver is decoupled and no active connections remain. Shutting down." ); return ControlFlow::Break(()); } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 072818e2dbd..bf4b25c1d50 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -151,7 +151,7 @@ impl Transport for GenTransport

{ addr: socket_addr, result: tx, }; - if listeners.is_empty() { + let (pending_dials, waker) = if listeners.is_empty() { let socket_family = socket_addr.ip().into(); let dialer = match self.dialer.get_mut(&socket_family) { Some(dialer) => dialer, @@ -160,17 +160,19 @@ impl Transport for GenTransport

{ self.dialer.entry(socket_family).or_insert(dialer) } }; - dialer.pending_dials.push_back(to_endpoint); + (&mut dialer.pending_dials, &mut dialer.waker) } else { // Pick a random listener to use for dialing. let n = rand::random::() % listeners.len(); let listener = listeners.get_mut(n).expect("Can not be out of bound."); - listener.pending_dials.push_back(to_endpoint); - if let Some(waker) = listener.waker.take() { - waker.wake() - } + (&mut listener.pending_dials, &mut listener.waker) }; + pending_dials.push_back(to_endpoint); + if let Some(waker) = waker.take() { + waker.wake() + } + Ok(async move { let connection = rx .await @@ -203,7 +205,7 @@ impl Transport for GenTransport

{ } } for key in errored { - // Background task of dialer crashed. + // Endpoint driver of dialer crashed. // Drop dialer and all pending dials so that the connection receiver is notified. self.dialer.remove(&key); } @@ -224,6 +226,7 @@ impl From for CoreTransportError { struct Dialer { endpoint_channel: EndpointChannel, pending_dials: VecDeque, + waker: Option, } impl Dialer { @@ -236,19 +239,22 @@ impl Dialer { Ok(Dialer { endpoint_channel, pending_dials: VecDeque::new(), + waker: None, }) } fn drive_dials(&mut self, cx: &mut Context<'_>) -> Result<(), mpsc::SendError> { - if let Some(to_endpoint) = self.pending_dials.pop_front() { + while let Some(to_endpoint) = self.pending_dials.pop_front() { match self.endpoint_channel.try_send(to_endpoint, cx) { Ok(Ok(())) => {} - Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), - Err(err) => { - return Err(err); + Ok(Err(to_endpoint)) => { + self.pending_dials.push_front(to_endpoint); + break; } + Err(err) => return Err(err), } } + self.waker = Some(cx.waker().clone()); Ok(()) } } @@ -432,11 +438,10 @@ pub enum SocketFamily { impl SocketFamily { fn is_same(a: &IpAddr, b: &IpAddr) -> bool { - match (a, b) { - (IpAddr::V4(_), IpAddr::V4(_)) => true, - (IpAddr::V6(_), IpAddr::V6(_)) => true, - _ => false, - } + matches!( + (a, b), + (IpAddr::V4(_), IpAddr::V4(_)) | (IpAddr::V6(_), IpAddr::V6(_)) + ) } } From 112345fa85b64481124543bbb42ff858a64603f4 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Wed, 5 Oct 2022 09:47:46 +1100 Subject: [PATCH 137/218] Introduce dedicated `dial` functions --- transports/quic/src/transport.rs | 82 ++++++++++++++++++++++---------- 1 file changed, 57 insertions(+), 25 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index bf4b25c1d50..a5eb4f33c90 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -41,6 +41,9 @@ use libp2p_core::{ transport::{ListenerId, TransportError as CoreTransportError, TransportEvent}, PeerId, Transport, }; +use rand::prelude::SliceRandom; +use rand::thread_rng; +use std::collections::hash_map::Entry; use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; use std::net::IpAddr; @@ -146,32 +149,23 @@ impl Transport for GenTransport

{ }) .collect::>(); - let (tx, rx) = oneshot::channel(); - let to_endpoint = ToEndpoint::Dial { - addr: socket_addr, - result: tx, - }; - let (pending_dials, waker) = if listeners.is_empty() { - let socket_family = socket_addr.ip().into(); - let dialer = match self.dialer.get_mut(&socket_family) { - Some(dialer) => dialer, - None => { - let dialer = Dialer::new::

(self.config.clone(), socket_family)?; - self.dialer.entry(socket_family).or_insert(dialer) - } - }; - (&mut dialer.pending_dials, &mut dialer.waker) - } else { - // Pick a random listener to use for dialing. - let n = rand::random::() % listeners.len(); - let listener = listeners.get_mut(n).expect("Can not be out of bound."); - (&mut listener.pending_dials, &mut listener.waker) - }; + // Try to use pick a random listener to use for dialing. + let rx = match listeners.choose_mut(&mut thread_rng()) { + Some(listener) => listener.dial(socket_addr), + None => { + // No listener? Get or create an explicit dialer. + + let socket_family = socket_addr.ip().into(); + let dialer = match self.dialer.entry(socket_family) { + Entry::Occupied(occupied) => occupied.into_mut(), + Entry::Vacant(vacant) => { + vacant.insert(Dialer::new::

(self.config.clone(), socket_family)?) + } + }; - pending_dials.push_back(to_endpoint); - if let Some(waker) = waker.take() { - waker.wake() - } + dialer.dial(socket_addr) + } + }; Ok(async move { let connection = rx @@ -243,6 +237,26 @@ impl Dialer { }) } + fn dial( + &mut self, + address: SocketAddr, + ) -> oneshot::Receiver> { + let (rx, tx) = oneshot::channel(); + + let message = ToEndpoint::Dial { + addr: address, + result: rx, + }; + + self.pending_dials.push_back(message); + + if let Some(waker) = self.waker.take() { + waker.wake(); + } + + tx + } + fn drive_dials(&mut self, cx: &mut Context<'_>) -> Result<(), mpsc::SendError> { while let Some(to_endpoint) = self.pending_dials.pop_front() { match self.endpoint_channel.try_send(to_endpoint, cx) { @@ -322,6 +336,24 @@ impl Listener { }) } + fn dial( + &mut self, + address: SocketAddr, + ) -> oneshot::Receiver> { + let (rx, tx) = oneshot::channel(); + + self.pending_dials.push_back(ToEndpoint::Dial { + addr: address, + result: rx, + }); + + if let Some(waker) = self.waker.take() { + waker.wake(); + } + + tx + } + /// Report the listener as closed in a [`TransportEvent::ListenerClosed`] and /// terminate the stream. fn close(&mut self, reason: Result<(), TransportError>) { From 7a86e308c62747ee94b03c61ebef9d3f98c7a26f Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Wed, 5 Oct 2022 10:15:04 +1100 Subject: [PATCH 138/218] Extract dedicated `DialerState` Adding this abstraction makes sure we don't forget to register a waker every time we work off the queue. Previously, a listener did not register a waker. --- transports/quic/src/transport.rs | 99 +++++++++++++++++--------------- 1 file changed, 53 insertions(+), 46 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index a5eb4f33c90..d7e259cdb80 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -151,7 +151,7 @@ impl Transport for GenTransport

{ // Try to use pick a random listener to use for dialing. let rx = match listeners.choose_mut(&mut thread_rng()) { - Some(listener) => listener.dial(socket_addr), + Some(listener) => listener.dialer_state.new_dial(socket_addr), None => { // No listener? Get or create an explicit dialer. @@ -163,7 +163,7 @@ impl Transport for GenTransport

{ } }; - dialer.dial(socket_addr) + dialer.state.new_dial(socket_addr) } }; @@ -194,7 +194,7 @@ impl Transport for GenTransport

{ ) -> Poll> { let mut errored = Vec::new(); for (key, dialer) in &mut self.dialer { - if dialer.drive_dials(cx).is_err() { + if let Poll::Ready(_error) = dialer.poll(cx) { errored.push(*key); } } @@ -219,8 +219,7 @@ impl From for CoreTransportError { #[derive(Debug)] struct Dialer { endpoint_channel: EndpointChannel, - pending_dials: VecDeque, - waker: Option, + state: DialerState, } impl Dialer { @@ -232,12 +231,29 @@ impl Dialer { .map_err(CoreTransportError::Other)?; Ok(Dialer { endpoint_channel, - pending_dials: VecDeque::new(), - waker: None, + state: DialerState::default(), }) } - fn dial( + fn poll(&mut self, cx: &mut Context<'_>) -> Poll { + self.state.poll(&mut self.endpoint_channel, cx) + } +} + +impl Drop for Dialer { + fn drop(&mut self) { + self.endpoint_channel.send_on_drop(ToEndpoint::Decoupled); + } +} + +#[derive(Default, Debug)] +struct DialerState { + pending_dials: VecDeque, + waker: Option, +} + +impl DialerState { + fn new_dial( &mut self, address: SocketAddr, ) -> oneshot::Receiver> { @@ -257,25 +273,26 @@ impl Dialer { tx } - fn drive_dials(&mut self, cx: &mut Context<'_>) -> Result<(), mpsc::SendError> { + /// Send all pending dials into the given [`EndpointChannel`]. + /// + /// This only ever returns [`Poll::Pending`] or an error in case the channel is closed. + fn poll( + &mut self, + channel: &mut EndpointChannel, + cx: &mut Context<'_>, + ) -> Poll { while let Some(to_endpoint) = self.pending_dials.pop_front() { - match self.endpoint_channel.try_send(to_endpoint, cx) { + match channel.try_send(to_endpoint, cx) { Ok(Ok(())) => {} Ok(Err(to_endpoint)) => { self.pending_dials.push_front(to_endpoint); break; } - Err(err) => return Err(err), + Err(_) => return Poll::Ready(TransportError::EndpointDriverCrashed), } } self.waker = Some(cx.waker().clone()); - Ok(()) - } -} - -impl Drop for Dialer { - fn drop(&mut self) { - self.endpoint_channel.send_on_drop(ToEndpoint::Decoupled); + Poll::Pending } } @@ -296,7 +313,7 @@ struct Listener { /// Pending event to reported. pending_event: Option<::Item>, - pending_dials: VecDeque, + dialer_state: DialerState, waker: Option, } @@ -331,29 +348,11 @@ impl Listener { if_watcher, is_closed: false, pending_event, - pending_dials: VecDeque::new(), + dialer_state: DialerState::default(), waker: None, }) } - fn dial( - &mut self, - address: SocketAddr, - ) -> oneshot::Receiver> { - let (rx, tx) = oneshot::channel(); - - self.pending_dials.push_back(ToEndpoint::Dial { - addr: address, - result: rx, - }); - - if let Some(waker) = self.waker.take() { - waker.wake(); - } - - tx - } - /// Report the listener as closed in a [`TransportEvent::ListenerClosed`] and /// terminate the stream. fn close(&mut self, reason: Result<(), TransportError>) { @@ -406,6 +405,17 @@ impl Listener { } } } + + /// Poll for a next If Event. + fn poll_dialer(&mut self, cx: &mut Context<'_>) -> Poll { + let Self { + dialer_state, + endpoint_channel, + .. + } = &mut *self; + + dialer_state.poll(endpoint_channel, cx) + } } impl Stream for Listener { @@ -422,15 +432,12 @@ impl Stream for Listener { Poll::Ready(event) => return Poll::Ready(Some(event)), Poll::Pending => {} } - if let Some(to_endpoint) = self.pending_dials.pop_front() { - match self.endpoint_channel.try_send(to_endpoint, cx) { - Ok(Ok(())) => {} - Ok(Err(to_endpoint)) => self.pending_dials.push_front(to_endpoint), - Err(_) => { - self.close(Err(TransportError::EndpointDriverCrashed)); - continue; - } + match self.poll_dialer(cx) { + Poll::Ready(error) => { + self.close(Err(error)); + continue; } + Poll::Pending => {} } match self.new_connections_rx.poll_next_unpin(cx) { Poll::Ready(Some(connection)) => { From 4dbb086f69e993feb7be6e6727a664be856ff8f2 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 11:15:53 +1100 Subject: [PATCH 139/218] Don't print inner Display of error This leads to double printing of errors because it is returned from `source` as well. --- transports/quic/src/connection.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 4b5fba32e3f..9fd10722200 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -68,7 +68,7 @@ pub enum ConnectionError { EndpointDriverCrashed, /// Error in the inner state machine. - #[error("{0}")] + #[error(transparent)] Quinn(#[from] quinn_proto::ConnectionError), } From 4010be06cf16ff6ae03d79e2d8ea90d02a685105 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 11:22:47 +1100 Subject: [PATCH 140/218] Remove unnecessary clone --- 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 6dd6a8e0121..72e0235442a 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -185,7 +185,7 @@ impl EndpointChannel { /// This clones the sender to the endpoint to guarantee delivery. /// It this should *not* be called for regular messages. pub fn send_on_drop(&mut self, to_endpoint: ToEndpoint) { - let _ = self.to_endpoint.clone().try_send(to_endpoint); + let _ = self.to_endpoint.try_send(to_endpoint); } } From 40e4545b6b2a87767193c40549648e124e1a6ed3 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 11:40:15 +1100 Subject: [PATCH 141/218] Use `Self` to refer to impl type --- transports/quic/src/endpoint.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 72e0235442a..c1d54e0a6a3 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -105,7 +105,7 @@ impl EndpointChannel { pub fn new_bidirectional( config: Config, socket_addr: SocketAddr, - ) -> Result<(EndpointChannel, mpsc::Receiver), transport::TransportError> { + ) -> Result<(Self, mpsc::Receiver), transport::TransportError> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Self::new::

(config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) @@ -115,7 +115,7 @@ impl EndpointChannel { pub fn new_dialer( config: Config, socket_family: SocketFamily, - ) -> Result { + ) -> Result { let socket_addr = match socket_family { SocketFamily::Ipv4 => SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), 0), SocketFamily::Ipv6 => SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0), @@ -127,13 +127,13 @@ impl EndpointChannel { config: Config, socket_addr: SocketAddr, new_connections: Option>, - ) -> Result { + ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; socket.set_nonblocking(true)?; let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(32); - let channel = EndpointChannel { + let channel = Self { to_endpoint: to_endpoint_tx, socket_addr: socket.local_addr()?, }; From 4d2c457cc9f3388747de27e2dd8c21caddcdea3a Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 11:42:24 +1100 Subject: [PATCH 142/218] Don't repeat module name in type --- transports/quic/src/connection.rs | 7 +++---- transports/quic/src/endpoint.rs | 8 ++++---- transports/quic/src/transport.rs | 12 ++++++------ 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 9fd10722200..f830856831e 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -26,8 +26,7 @@ //! All interactions with a QUIC connection should be done through this struct. // TODO: docs -use crate::endpoint::{EndpointChannel, ToEndpoint}; - +use crate::endpoint::{self, ToEndpoint}; use futures::{channel::mpsc, prelude::*}; use futures_timer::Delay; use libp2p_core::PeerId; @@ -44,7 +43,7 @@ use std::{ #[derive(Debug)] pub struct Connection { /// Channel to the endpoint this connection belongs to. - endpoint_channel: EndpointChannel, + endpoint_channel: endpoint::Channel, /// Pending message to be sent to the background task that is driving the endpoint. pending_to_endpoint: Option, /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. @@ -85,7 +84,7 @@ impl Connection { /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of /// its methods has ever been called. Failure to comply might lead to logic errors and panics. pub fn from_quinn_connection( - endpoint_channel: EndpointChannel, + endpoint_channel: endpoint::Channel, connection: quinn_proto::Connection, connection_id: quinn_proto::ConnectionHandle, from_endpoint: mpsc::Receiver, diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index c1d54e0a6a3..32e6beb8d02 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -92,7 +92,7 @@ impl Config { } #[derive(Debug, Clone)] -pub struct EndpointChannel { +pub struct Channel { /// Channel to the background of the endpoint. to_endpoint: mpsc::Sender, /// Address that the socket is bound to. @@ -100,7 +100,7 @@ pub struct EndpointChannel { socket_addr: SocketAddr, } -impl EndpointChannel { +impl Channel { /// Builds a new endpoint that is listening on the [`SocketAddr`]. pub fn new_bidirectional( config: Config, @@ -298,7 +298,7 @@ pub struct EndpointDriver { // Config for client connections. client_config: quinn_proto::ClientConfig, // Copy of the channel to the endpoint driver that is passed to each new connection. - channel: EndpointChannel, + channel: Channel, // Channel to receive messages from the transport or connections. rx: mpsc::Receiver, @@ -323,7 +323,7 @@ impl EndpointDriver

{ client_config: quinn_proto::ClientConfig, new_connection_tx: Option>, server_config: Option>, - channel: EndpointChannel, + channel: Channel, socket: P, rx: mpsc::Receiver, ) -> Self { diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index d7e259cdb80..ff3ee831580 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -25,7 +25,7 @@ use crate::connection::Connection; use crate::endpoint::ToEndpoint; use crate::provider::Provider; -use crate::{endpoint::EndpointChannel, muxer::QuicMuxer, upgrade::Connecting}; +use crate::{endpoint, muxer::QuicMuxer, upgrade::Connecting}; use crate::{Config, ConnectionError}; use futures::channel::{mpsc, oneshot}; @@ -218,7 +218,7 @@ impl From for CoreTransportError { #[derive(Debug)] struct Dialer { - endpoint_channel: EndpointChannel, + endpoint_channel: endpoint::Channel, state: DialerState, } @@ -227,7 +227,7 @@ impl Dialer { config: Config, socket_family: SocketFamily, ) -> Result> { - let endpoint_channel = EndpointChannel::new_dialer::

(config, socket_family) + let endpoint_channel = endpoint::Channel::new_dialer::

(config, socket_family) .map_err(CoreTransportError::Other)?; Ok(Dialer { endpoint_channel, @@ -278,7 +278,7 @@ impl DialerState { /// This only ever returns [`Poll::Pending`] or an error in case the channel is closed. fn poll( &mut self, - channel: &mut EndpointChannel, + channel: &mut endpoint::Channel, cx: &mut Context<'_>, ) -> Poll { while let Some(to_endpoint) = self.pending_dials.pop_front() { @@ -298,7 +298,7 @@ impl DialerState { #[derive(Debug)] struct Listener { - endpoint_channel: EndpointChannel, + endpoint_channel: endpoint::Channel, listener_id: ListenerId, @@ -325,7 +325,7 @@ impl Listener { config: Config, ) -> Result { let (endpoint_channel, new_connections_rx) = - EndpointChannel::new_bidirectional::

(config, socket_addr)?; + endpoint::Channel::new_bidirectional::

(config, socket_addr)?; let if_watcher; let pending_event; From 8693b4e7adef3eff0d809279a695f04286f63f7b Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 12:06:23 +1100 Subject: [PATCH 143/218] Call `map_err` inside `new_dial` This encapsulates the use of our oneshots better. --- transports/quic/src/transport.rs | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index ff3ee831580..91d00929d89 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -29,18 +29,20 @@ use crate::{endpoint, muxer::QuicMuxer, upgrade::Connecting}; use crate::{Config, ConnectionError}; use futures::channel::{mpsc, oneshot}; -use futures::future::BoxFuture; +use futures::future::{BoxFuture, MapErr}; use futures::ready; use futures::stream::StreamExt; use futures::{prelude::*, stream::SelectAll}; use if_watch::{IfEvent, IfWatcher}; +use futures::channel::oneshot::{Canceled, Receiver}; use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, transport::{ListenerId, TransportError as CoreTransportError, TransportEvent}, PeerId, Transport, }; +use quinn_proto::ConnectError; use rand::prelude::SliceRandom; use rand::thread_rng; use std::collections::hash_map::Entry; @@ -168,9 +170,7 @@ impl Transport for GenTransport

{ }; Ok(async move { - let connection = rx - .await - .map_err(|_| TransportError::EndpointDriverCrashed)??; + let connection = rx.await??; let final_connec = Connecting::from_connection(connection).await?; Ok(final_connec) } @@ -253,10 +253,11 @@ struct DialerState { } impl DialerState { + // With TAIP, this return signature would be a bit nicer. fn new_dial( &mut self, address: SocketAddr, - ) -> oneshot::Receiver> { + ) -> MapErr>, fn(Canceled) -> TransportError> { let (rx, tx) = oneshot::channel(); let message = ToEndpoint::Dial { @@ -270,7 +271,7 @@ impl DialerState { waker.wake(); } - tx + tx.map_err(|_| TransportError::EndpointDriverCrashed) } /// Send all pending dials into the given [`EndpointChannel`]. From 6d7a8a6a41c8e4e3d965d0b6a32328dcaf70c997 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 12:07:14 +1100 Subject: [PATCH 144/218] Better name for returned future --- transports/quic/src/transport.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 91d00929d89..535938fd289 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -152,7 +152,7 @@ impl Transport for GenTransport

{ .collect::>(); // Try to use pick a random listener to use for dialing. - let rx = match listeners.choose_mut(&mut thread_rng()) { + let dialing = match listeners.choose_mut(&mut thread_rng()) { Some(listener) => listener.dialer_state.new_dial(socket_addr), None => { // No listener? Get or create an explicit dialer. @@ -170,7 +170,7 @@ impl Transport for GenTransport

{ }; Ok(async move { - let connection = rx.await??; + let connection = dialing.await??; let final_connec = Connecting::from_connection(connection).await?; Ok(final_connec) } From a43b12d4ff74546f28d4a60d6004fd7537e88a8b Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 12:11:47 +1100 Subject: [PATCH 145/218] Better type-safety for disconnected background task The `mpsc::Error` type is also used for expressing that the channel is full which is something we guard against in the `try_send` impl. Introducing a dedicated `Disconnected` error type expresses this invariant in the type system. --- transports/quic/src/connection.rs | 2 +- transports/quic/src/endpoint.rs | 24 +++++++++++++++++++++--- transports/quic/src/transport.rs | 5 ++++- 3 files changed, 26 insertions(+), 5 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index f830856831e..b20b0483dc9 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -232,7 +232,7 @@ impl Connection { self.pending_to_endpoint = Some(to_endpoint); return Poll::Pending; } - Err(_) => { + Err(endpoint::Disconnected {}) => { return Poll::Ready(ConnectionEvent::ConnectionLost( ConnectionError::EndpointDriverCrashed, )); diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 32e6beb8d02..392e0f3aa85 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -171,13 +171,27 @@ impl Channel { &mut self, to_endpoint: ToEndpoint, cx: &mut Context<'_>, - ) -> Result, mpsc::SendError> { + ) -> Result, Disconnected> { match self.to_endpoint.poll_ready_unpin(cx) { Poll::Ready(Ok(())) => {} - Poll::Ready(Err(err)) => return Err(err), + Poll::Ready(Err(e)) => { + debug_assert!( + e.is_disconnected(), + "mpsc::Sender can only be disconnected when calling `poll_ready_unpin" + ); + + return Err(Disconnected {}); + } Poll::Pending => return Ok(Err(to_endpoint)), }; - self.to_endpoint.start_send(to_endpoint).map(Ok) + + if let Err(e) = self.to_endpoint.start_send(to_endpoint) { + debug_assert!(e.is_disconnected(), "We called `Sink::poll_ready` so we are guaranteed to have a slot. If this fails, it means we are disconnected."); + + return Err(Disconnected {}); + } + + Ok(Ok(())) } /// Send a message to inform the [`EndpointDriver`] about an @@ -189,6 +203,10 @@ impl Channel { } } +#[derive(Debug, thiserror::Error)] +#[error("Background task disconnected")] +pub struct Disconnected {} + /// Message sent to the endpoint background task. #[derive(Debug)] pub enum ToEndpoint { diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 535938fd289..95fbca697a2 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -271,6 +271,7 @@ impl DialerState { waker.wake(); } + // Our oneshot getting dropped means the message didn't make it to the endpoint driver. tx.map_err(|_| TransportError::EndpointDriverCrashed) } @@ -289,7 +290,9 @@ impl DialerState { self.pending_dials.push_front(to_endpoint); break; } - Err(_) => return Poll::Ready(TransportError::EndpointDriverCrashed), + Err(endpoint::Disconnected {}) => { + return Poll::Ready(TransportError::EndpointDriverCrashed) + } } } self.waker = Some(cx.waker().clone()); From 1c0e0f1bc5c2789dc403fe7c65ee867abf3ba15a Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 12:17:08 +1100 Subject: [PATCH 146/218] Fix docs --- 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 95fbca697a2..ff617b5adfa 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -410,7 +410,7 @@ impl Listener { } } - /// Poll for a next If Event. + /// Poll [`DialerState`] to initiate requested dials. fn poll_dialer(&mut self, cx: &mut Context<'_>) -> Poll { let Self { dialer_state, From 42e58a8cbb06e2c90360150a407959079363c292 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 13:41:02 +1100 Subject: [PATCH 147/218] Fix broken intra-doc links See https://github.com/libp2p/rust-libp2p/pull/2990. --- transports/quic/src/endpoint.rs | 4 ++-- transports/quic/src/transport.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 392e0f3aa85..f30a0da5208 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -195,7 +195,7 @@ impl Channel { } /// Send a message to inform the [`EndpointDriver`] about an - /// event caused by the owner of this [`EndpointChannel`] dropping. + /// event caused by the owner of this [`Channel`] dropping. /// This clones the sender to the endpoint to guarantee delivery. /// It this should *not* be called for regular messages. pub fn send_on_drop(&mut self, to_endpoint: ToEndpoint) { @@ -254,7 +254,7 @@ pub enum ToEndpoint { /// in play: /// /// - One channel, represented by `EndpointChannel::to_endpoint` and `receiver`, that communicates -/// messages from [`EndpointChannel`] to the [`EndpointDriver`]. +/// messages from [`Channel`] to the [`EndpointDriver`]. /// - One channel per each existing connection that communicates messages from the [`EndpointDriver`] /// to that [`Connection`]. /// - One channel for the [`EndpointDriver`] to send newly-opened connections to. The receiving diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index ff617b5adfa..2712a29e08e 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -275,7 +275,7 @@ impl DialerState { tx.map_err(|_| TransportError::EndpointDriverCrashed) } - /// Send all pending dials into the given [`EndpointChannel`]. + /// Send all pending dials into the given [`endpoint::Channel`]. /// /// This only ever returns [`Poll::Pending`] or an error in case the channel is closed. fn poll( From 716fa16ddef1e03d29c1d7fa0e0cc846dc7ccd6d Mon Sep 17 00:00:00 2001 From: Roman Date: Thu, 6 Oct 2022 21:00:05 +0300 Subject: [PATCH 148/218] Update transports/quic/src/endpoint.rs Co-authored-by: Marco Munizaga --- 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 f30a0da5208..f554f791e6c 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -197,7 +197,7 @@ impl Channel { /// Send a message to inform the [`EndpointDriver`] about an /// event caused by the owner of this [`Channel`] dropping. /// This clones the sender to the endpoint to guarantee delivery. - /// It this should *not* be called for regular messages. + /// This should *not* be called for regular messages. pub fn send_on_drop(&mut self, to_endpoint: ToEndpoint) { let _ = self.to_endpoint.try_send(to_endpoint); } From 2264ab83377758d0644e668caba1ae9048cc384c Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 12 Oct 2022 12:54:28 +0200 Subject: [PATCH 149/218] transports/quic: expose config params, add handshake timeout Rename existing `Config` to `QuinnConfig`. Add new `Config` for params that the user may configure. Add timeout for the initial handshake. --- transports/quic/src/endpoint.rs | 102 +++++++++++++++++++++++-------- transports/quic/src/lib.rs | 2 +- transports/quic/src/transport.rs | 44 ++++++++----- transports/quic/src/upgrade.rs | 38 ++++++++---- transports/quic/tests/smoke.rs | 4 +- 5 files changed, 135 insertions(+), 55 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index f554f791e6c..21281bdc56f 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -51,43 +51,89 @@ use std::{ }; use x509_parser::nom::AsBytes; -/// Represents the configuration for the QUIC endpoint. -#[derive(Debug, Clone)] +/// Config for the transport. +#[derive(Clone)] pub struct Config { - /// The client configuration to pass to `quinn_proto`. + /// Timeout for the initial handshake when establishing a connection. + /// The actual timeout is the minimum of this an the [`Config::max_idle_timeout`]. + pub handshake_timeout: Duration, + /// Maximum duration of inactivity to accept before timing out the connection. + pub max_idle_timeout: Duration, + /// Period of inactivity before sending a keep-alive packet. + /// Must be set lower than the idle_timeout of both + /// peers to be effective. + /// + /// See [`quinn_proto::TransportConfig::keep_alive_interval`] for more + /// info. + pub keep_alive_interval: Duration, + /// Maximum number of incoming bidirectional streams that may be open + /// concurrently by the remote peer. + pub max_concurrent_stream_limit: u32, + + client_tls_config: Arc, + server_tls_config: Arc, +} + +impl Config { + /// Creates a new configuration object with default values. + pub fn new(keypair: &libp2p_core::identity::Keypair) -> Self { + let client_tls_config = Arc::new(tls::make_client_config(keypair).unwrap()); + let server_tls_config = Arc::new(tls::make_server_config(keypair).unwrap()); + Self { + client_tls_config, + server_tls_config, + handshake_timeout: Duration::from_secs(5), + max_idle_timeout: Duration::from_secs(30), + max_concurrent_stream_limit: 256, + keep_alive_interval: Duration::from_secs(15), + } + } +} + +/// Represents the inner configuration for [`quinn_proto`]. +#[derive(Debug, Clone)] +pub struct QuinnConfig { client_config: quinn_proto::ClientConfig, - /// The server configuration to pass to `quinn_proto`. server_config: Arc, - /// The endpoint configuration to pass to `quinn_proto`. endpoint_config: Arc, } -impl Config { - /// Creates a new configuration object with default values. - pub fn new(keypair: &libp2p_core::identity::Keypair) -> Result { +impl From for QuinnConfig { + fn from(config: Config) -> QuinnConfig { + let Config { + client_tls_config, + server_tls_config, + max_idle_timeout, + max_concurrent_stream_limit, + keep_alive_interval, + handshake_timeout: _, + } = config; let mut transport = quinn_proto::TransportConfig::default(); - transport.max_concurrent_uni_streams(0u32.into()); // Can only panic if value is out of range. + transport.max_concurrent_uni_streams(0u32.into()); + transport.max_concurrent_bidi_streams(max_concurrent_stream_limit.into()); transport.datagram_receive_buffer_size(None); - transport.keep_alive_interval(Some(Duration::from_millis(10))); + transport.keep_alive_interval(Some(keep_alive_interval)); + transport.max_idle_timeout(Some(max_idle_timeout.try_into().expect("is < 2^62"))); + transport.allow_spin(false); let transport = Arc::new(transport); - let client_tls_config = tls::make_client_config(keypair).unwrap(); - let server_tls_config = tls::make_server_config(keypair).unwrap(); - - let mut server_config = quinn_proto::ServerConfig::with_crypto(Arc::new(server_tls_config)); + let mut server_config = quinn_proto::ServerConfig::with_crypto(server_tls_config); server_config.transport = Arc::clone(&transport); // Disables connection migration. // Long-term this should be enabled, however we then need to handle address change // on connections in the `QuicMuxer`. server_config.migration(false); - let mut client_config = quinn_proto::ClientConfig::new(Arc::new(client_tls_config)); + let mut client_config = quinn_proto::ClientConfig::new(client_tls_config); client_config.transport = transport; - Ok(Self { + + let endpoint_config = quinn_proto::EndpointConfig::default(); + + QuinnConfig { client_config, server_config: Arc::new(server_config), - endpoint_config: Default::default(), - }) + endpoint_config: Arc::new(endpoint_config), + } } } @@ -103,28 +149,28 @@ pub struct Channel { impl Channel { /// Builds a new endpoint that is listening on the [`SocketAddr`]. pub fn new_bidirectional( - config: Config, + quinn_config: QuinnConfig, socket_addr: SocketAddr, ) -> Result<(Self, mpsc::Receiver), transport::TransportError> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); - let endpoint = Self::new::

(config, socket_addr, Some(new_connections_tx))?; + let endpoint = Self::new::

(quinn_config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) } /// Builds a new endpoint that only supports outbound connections. pub fn new_dialer( - config: Config, + quinn_config: QuinnConfig, socket_family: SocketFamily, ) -> Result { let socket_addr = match socket_family { SocketFamily::Ipv4 => SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), 0), SocketFamily::Ipv6 => SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0), }; - Self::new::

(config, socket_addr, None) + Self::new::

(quinn_config, socket_addr, None) } fn new( - config: Config, + quinn_config: QuinnConfig, socket_addr: SocketAddr, new_connections: Option>, ) -> Result { @@ -138,12 +184,14 @@ impl Channel { socket_addr: socket.local_addr()?, }; - let server_config = new_connections.is_some().then_some(config.server_config); + let server_config = new_connections + .is_some() + .then_some(quinn_config.server_config); let socket = P::from_socket(socket)?; let driver = EndpointDriver::

::new( - config.endpoint_config, - config.client_config, + quinn_config.endpoint_config, + quinn_config.client_config, new_connections, server_config, channel.clone(), @@ -313,7 +361,7 @@ pub enum ToEndpoint { pub struct EndpointDriver { // The actual QUIC state machine. endpoint: quinn_proto::Endpoint, - // Config for client connections. + // QuinnConfig for client connections. client_config: quinn_proto::ClientConfig, // Copy of the channel to the endpoint driver that is passed to each new connection. channel: Channel, diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index c0c5a4f3b6b..001b6d94901 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -35,7 +35,7 @@ //! use libp2p_core::{Multiaddr, Transport}; //! //! let keypair = libp2p_core::identity::Keypair::generate_ed25519(); -//! let quic_config = quic::Config::new(&keypair).expect("could not make config"); +//! let quic_config = quic::Config::new(&keypair); //! //! let mut quic_transport = quic::async_std::Transport::new(quic_config); //! diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 2712a29e08e..bbd96aa7b0d 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -22,11 +22,10 @@ //! //! Combines all the objects in the other modules to implement the trait. -use crate::connection::Connection; -use crate::endpoint::ToEndpoint; +use crate::connection::{Connection, ConnectionError}; +use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; use crate::provider::Provider; use crate::{endpoint, muxer::QuicMuxer, upgrade::Connecting}; -use crate::{Config, ConnectionError}; use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, MapErr}; @@ -50,6 +49,7 @@ use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; use std::net::IpAddr; use std::task::Waker; +use std::time::Duration; use std::{ net::SocketAddr, pin::Pin, @@ -58,7 +58,9 @@ use std::{ #[derive(Debug)] pub struct GenTransport

{ - config: Config, + quinn_config: QuinnConfig, + handshake_timeout: Duration, + listeners: SelectAll, /// Dialer for each socket family if no matching listener exists. dialer: HashMap, @@ -68,9 +70,12 @@ pub struct GenTransport

{ impl

GenTransport

{ pub fn new(config: Config) -> Self { + let handshake_timeout = config.handshake_timeout; + let quinn_config = config.into(); Self { listeners: SelectAll::new(), - config, + quinn_config, + handshake_timeout, dialer: HashMap::new(), _marker: Default::default(), } @@ -93,6 +98,9 @@ pub enum TransportError { /// The task driving the endpoint has crashed. #[error("Endpoint driver crashed")] EndpointDriverCrashed, + + #[error("Handshake with the remote timed out.")] + HandshakeTimedOut, } impl Transport for GenTransport

{ @@ -108,7 +116,12 @@ impl Transport for GenTransport

{ let socket_addr = multiaddr_to_socketaddr(&addr) .ok_or(CoreTransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); - let listener = Listener::new::

(listener_id, socket_addr, self.config.clone())?; + let listener = Listener::new::

( + listener_id, + socket_addr, + self.quinn_config.clone(), + self.handshake_timeout, + )?; self.listeners.push(listener); // Remove dialer endpoint so that the endpoint is dropped once the last @@ -161,17 +174,17 @@ impl Transport for GenTransport

{ let dialer = match self.dialer.entry(socket_family) { Entry::Occupied(occupied) => occupied.into_mut(), Entry::Vacant(vacant) => { - vacant.insert(Dialer::new::

(self.config.clone(), socket_family)?) + vacant.insert(Dialer::new::

(self.quinn_config.clone(), socket_family)?) } }; dialer.state.new_dial(socket_addr) } }; - + let handshake_timeout = self.handshake_timeout; Ok(async move { let connection = dialing.await??; - let final_connec = Connecting::from_connection(connection).await?; + let final_connec = Connecting::from_connection(connection, handshake_timeout).await?; Ok(final_connec) } .boxed()) @@ -224,7 +237,7 @@ struct Dialer { impl Dialer { fn new( - config: Config, + config: QuinnConfig, socket_family: SocketFamily, ) -> Result> { let endpoint_channel = endpoint::Channel::new_dialer::

(config, socket_family) @@ -308,6 +321,7 @@ struct Listener { /// Channel where new connections are being sent. new_connections_rx: mpsc::Receiver, + handshake_timeout: Duration, if_watcher: Option, @@ -326,7 +340,8 @@ impl Listener { fn new( listener_id: ListenerId, socket_addr: SocketAddr, - config: Config, + config: QuinnConfig, + handshake_timeout: Duration, ) -> Result { let (endpoint_channel, new_connections_rx) = endpoint::Channel::new_bidirectional::

(config, socket_addr)?; @@ -349,6 +364,7 @@ impl Listener { endpoint_channel, listener_id, new_connections_rx, + handshake_timeout, if_watcher, is_closed: false, pending_event, @@ -448,7 +464,7 @@ impl Stream for Listener { let local_addr = socketaddr_to_multiaddr(connection.local_addr()); let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); let event = TransportEvent::Incoming { - upgrade: Connecting::from_connection(connection), + upgrade: Connecting::from_connection(connection, self.handshake_timeout), local_addr, send_back_addr, listener_id: self.listener_id, @@ -642,7 +658,7 @@ mod test { #[tokio::test] async fn tokio_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); - let config = Config::new(&keypair).unwrap(); + let config = Config::new(&keypair); let transport = crate::tokio::Transport::new(config.clone()); test_close_listener(transport).await } @@ -651,7 +667,7 @@ mod test { #[async_std::test] async fn async_std_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); - let config = Config::new(&keypair).unwrap(); + let config = Config::new(&keypair); let transport = crate::async_std::Transport::new(config.clone()); test_close_listener(transport).await } diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 5e47f4d89d2..7866aefac29 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -26,24 +26,28 @@ use crate::{ transport, }; -use futures::{prelude::*, ready}; +use futures::prelude::*; +use futures_timer::Delay; use libp2p_core::PeerId; use std::{ pin::Pin, task::{Context, Poll}, + time::Duration, }; /// A QUIC connection currently being negotiated. #[derive(Debug)] pub struct Connecting { connection: Option, + timeout: Delay, } impl Connecting { /// Builds an [`Connecting`] that wraps around a [`Connection`]. - pub(crate) fn from_connection(connection: Connection) -> Self { + pub(crate) fn from_connection(connection: Connection, timeout: Duration) -> Self { Connecting { connection: Some(connection), + timeout: Delay::new(timeout), } } } @@ -58,21 +62,31 @@ impl Future for Connecting { .expect("Future polled after it has completed"); loop { - match ready!(connection.poll_event(cx)) { - ConnectionEvent::Connected => { + match connection.poll_event(cx) { + Poll::Ready(ConnectionEvent::Connected) => { let peer_id = connection.remote_peer_id(); let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } - ConnectionEvent::ConnectionLost(err) => return Poll::Ready(Err(err.into())), - ConnectionEvent::HandshakeDataReady - | ConnectionEvent::StreamAvailable - | ConnectionEvent::StreamOpened - | ConnectionEvent::StreamReadable(_) - | ConnectionEvent::StreamWritable(_) - | ConnectionEvent::StreamFinished(_) - | ConnectionEvent::StreamStopped(_) => {} + Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { + return Poll::Ready(Err(err.into())) + } + Poll::Ready(ConnectionEvent::HandshakeDataReady) + | Poll::Ready(ConnectionEvent::StreamAvailable) + | Poll::Ready(ConnectionEvent::StreamOpened) + | Poll::Ready(ConnectionEvent::StreamReadable(_)) + | Poll::Ready(ConnectionEvent::StreamWritable(_)) + | Poll::Ready(ConnectionEvent::StreamFinished(_)) + | Poll::Ready(ConnectionEvent::StreamStopped(_)) => continue, + Poll::Pending => {} + } + match self.timeout.poll_unpin(cx) { + Poll::Ready(()) => { + return Poll::Ready(Err(transport::TransportError::HandshakeTimedOut)) + } + Poll::Pending => {} } + return Poll::Pending; } } } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 2a9f7ca696f..0e5af3d2cf4 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -21,6 +21,7 @@ use libp2p_quic as quic; use quic::Provider; use rand::RngCore; use std::num::NonZeroU8; +use std::time::Duration; use std::{io, iter}; #[cfg(feature = "async-std")] @@ -35,7 +36,8 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { async fn create_swarm() -> Swarm> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); - let config = quic::Config::new(&keypair).unwrap(); + let mut config = quic::Config::new(&keypair); + config.handshake_timeout = Duration::from_secs(1); let transport = quic::GenTransport::

::new(config); let transport = Transport::map(transport, |(peer, muxer), _| { From f121350b3b981b8acddd2a82aa15ee8c68cec623 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 12 Oct 2022 17:52:32 +0200 Subject: [PATCH 150/218] transports/quic/tests: use quickcheck --- transports/quic/Cargo.toml | 1 + transports/quic/tests/smoke.rs | 30 ++++++++++++++---------------- 2 files changed, 15 insertions(+), 16 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 402891fc46a..917c65f0158 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -35,6 +35,7 @@ async-std = ["dep:async-std-crate"] [dev-dependencies] async-std-crate = { package = "async-std", version = "1.12.0", features = ["attributes"] } libp2p = { path = "../..", features = ["request-response"] } +env_logger = "0.9.0" rand = "0.8.4" tokio-crate = { package = "tokio", version = "1.21.1", features = ["macros", "rt-multi-thread"] } quickcheck = "1" diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 0e5af3d2cf4..01fae2289c1 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -72,6 +72,7 @@ async fn async_std_smoke() { } async fn smoke() { + let _ = env_logger::try_init(); let mut rng = rand::thread_rng(); let mut a = create_swarm::

().await; @@ -293,6 +294,7 @@ impl RequestResponseCodec for PingCodec { #[cfg(feature = "async-std")] #[async_std::test] async fn dial_failure() { + let _ = env_logger::try_init(); let mut a = create_swarm::().await; let mut b = create_swarm::().await; @@ -325,10 +327,7 @@ async fn dial_failure() { fn concurrent_connections_and_streams() { use quickcheck::*; - async fn prop( - number_listeners: NonZeroU8, - number_streams: NonZeroU8, - ) -> TestResult { + fn prop(number_listeners: NonZeroU8, number_streams: NonZeroU8) -> TestResult { let (number_listeners, number_streams): (u8, u8) = (number_listeners.into(), number_streams.into()); if number_listeners > 10 || number_streams > 10 { @@ -342,8 +341,8 @@ fn concurrent_connections_and_streams() { // Spawn the listener nodes. for _ in 0..number_listeners { - let mut listener = create_swarm::

().await; - let addr = start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic").await; + let mut listener = pool.run_until(create_swarm::

()); + let addr = pool.run_until(start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic")); listeners.push((*listener.local_peer_id(), addr)); @@ -387,7 +386,7 @@ fn concurrent_connections_and_streams() { .unwrap(); } - let mut dialer = create_swarm::

().await; + let mut dialer = pool.run_until(create_swarm::

()); // For each listener node start `number_streams` requests. for (listener_peer_id, listener_addr) in &listeners { @@ -437,23 +436,21 @@ fn concurrent_connections_and_streams() { TestResult::passed() } - let num_listener = NonZeroU8::new(3).unwrap(); - let num_streams = NonZeroU8::new(8).unwrap(); - #[cfg(feature = "tokio")] - tokio::runtime::Runtime::new() - .unwrap() - .block_on(prop::(num_listener, num_streams)); + { + let rt = tokio::runtime::Runtime::new().unwrap(); + let _guard = rt.enter(); + QuickCheck::new().quickcheck(prop:: as fn(_, _) -> _); + } #[cfg(feature = "async-std")] - async_std::task::block_on(prop::(num_listener, num_streams)); - - // QuickCheck::new().quickcheck(prop as fn(_, _) -> _); + QuickCheck::new().quickcheck(prop:: as fn(_, _) -> _); } #[cfg(feature = "tokio")] #[tokio::test] async fn endpoint_reuse() { + let _ = env_logger::try_init(); let mut swarm_a = create_swarm::().await; let mut swarm_b = create_swarm::().await; let b_peer_id = *swarm_b.local_peer_id(); @@ -545,6 +542,7 @@ async fn endpoint_reuse() { #[cfg(feature = "async-std")] #[async_std::test] async fn ipv4_dial_ipv6() { + let _ = env_logger::try_init(); let mut swarm_a = create_swarm::().await; let mut swarm_b = create_swarm::().await; From 963d3dd30fac8fe54b25e7e98e3041b809918b3a Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 12 Oct 2022 20:13:03 +0200 Subject: [PATCH 151/218] transports/quic: report PeerId in Connected event --- transports/quic/src/connection.rs | 117 ++++++++++++++---------------- transports/quic/src/muxer.rs | 2 +- transports/quic/src/upgrade.rs | 3 +- 3 files changed, 58 insertions(+), 64 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index b20b0483dc9..5b9cb99b0ab 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -110,23 +110,6 @@ impl Connection { self.connection.remote_address() } - /// Returns the ID of the node we're connected to. - pub fn remote_peer_id(&self) -> PeerId { - let session = self.connection.crypto_session(); - let identity = session - .peer_identity() - .expect("connection got identity because it passed TLS handshake; qed"); - let certificates: Box> = - identity.downcast().expect("we rely on rustls feature; qed"); - let end_entity = certificates - .get(0) - .expect("there should be exactly one certificate; qed"); - let end_entity_der = end_entity.as_ref(); - let p2p_cert = crate::tls::certificate::parse_certificate(end_entity_der) - .expect("the certificate was validated during TLS handshake; qed"); - PeerId::from_public_key(&p2p_cert.extension.public_key) - } - /// Start closing the connection. A [`ConnectionEvent::ConnectionLost`] event will be /// produced in the future. pub fn close(&mut self) { @@ -291,21 +274,72 @@ impl Connection { // The final step consists in handling the events related to the various substreams. if let Some(ev) = self.connection.poll() { - match ConnectionEvent::try_from(ev) { - Ok(ConnectionEvent::ConnectionLost(reason)) => { + match self.parse_connection_event(ev) { + ConnectionEvent::ConnectionLost(reason) => { // Continue in the loop once more so that we can send a // `EndpointEvent::drained` to the endpoint before returning. closed = Some(reason); continue; } - Ok(event) => return Poll::Ready(event), - Err(_) => unreachable!("We don't use datagrams or unidirectional streams."), + event => return Poll::Ready(event), } } return Poll::Pending; } } + + fn parse_connection_event(&self, event: quinn_proto::Event) -> ConnectionEvent { + match event { + quinn_proto::Event::Connected => { + let session = self.connection.crypto_session(); + let identity = session + .peer_identity() + .expect("connection got identity because it passed TLS handshake; qed"); + let certificates: Box> = + identity.downcast().expect("we rely on rustls feature; qed"); + let end_entity = certificates + .get(0) + .expect("there should be exactly one certificate; qed"); + let end_entity_der = end_entity.as_ref(); + let p2p_cert = crate::tls::certificate::parse_certificate(end_entity_der) + .expect("the certificate was validated during TLS handshake; qed"); + let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); + ConnectionEvent::Connected(peer_id) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { + ConnectionEvent::StreamReadable(id) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Writable { id }) => { + ConnectionEvent::StreamWritable(id) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { id, .. }) => { + ConnectionEvent::StreamStopped(id) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Bi, + }) => ConnectionEvent::StreamAvailable, + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Bi, + }) => ConnectionEvent::StreamOpened, + quinn_proto::Event::ConnectionLost { reason } => { + ConnectionEvent::ConnectionLost(ConnectionError::Quinn(reason)) + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { + ConnectionEvent::StreamFinished(id) + } + quinn_proto::Event::HandshakeDataReady => ConnectionEvent::HandshakeDataReady, + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Uni, + }) + | quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Uni, + }) + | quinn_proto::Event::DatagramReceived => { + unreachable!("We don't use datagrams or unidirectional streams.") + } + } + } } impl Drop for Connection { @@ -322,7 +356,7 @@ impl Drop for Connection { #[derive(Debug)] pub enum ConnectionEvent { /// Now connected to the remote and certificates are available. - Connected, + Connected(PeerId), /// Connection has been closed and can no longer be used. ConnectionLost(ConnectionError), @@ -347,42 +381,3 @@ pub enum ConnectionEvent { HandshakeDataReady, } - -impl TryFrom for ConnectionEvent { - type Error = quinn_proto::Event; - - fn try_from(event: quinn_proto::Event) -> Result { - match event { - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { - Ok(ConnectionEvent::StreamReadable(id)) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Writable { id }) => { - Ok(ConnectionEvent::StreamWritable(id)) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { id, .. }) => { - Ok(ConnectionEvent::StreamStopped(id)) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { - dir: quinn_proto::Dir::Bi, - }) => Ok(ConnectionEvent::StreamAvailable), - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { - dir: quinn_proto::Dir::Bi, - }) => Ok(ConnectionEvent::StreamOpened), - quinn_proto::Event::ConnectionLost { reason } => Ok(ConnectionEvent::ConnectionLost( - ConnectionError::Quinn(reason), - )), - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { - Ok(ConnectionEvent::StreamFinished(id)) - } - quinn_proto::Event::Connected => Ok(ConnectionEvent::Connected), - quinn_proto::Event::HandshakeDataReady => Ok(ConnectionEvent::HandshakeDataReady), - ev @ quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { - dir: quinn_proto::Dir::Uni, - }) - | ev @ quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { - dir: quinn_proto::Dir::Uni, - }) - | ev @ quinn_proto::Event::DatagramReceived => Err(ev), - } - } -} diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 3e91b995882..e36c138a148 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -94,7 +94,7 @@ impl StreamMuxer for QuicMuxer { // the wakers of related poll-based methods. while let Poll::Ready(event) = inner.connection.poll_event(cx) { match event { - ConnectionEvent::Connected | ConnectionEvent::HandshakeDataReady => { + ConnectionEvent::Connected(_) | ConnectionEvent::HandshakeDataReady => { debug_assert!( false, "Unexpected event {:?} on established QUIC connection", diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 7866aefac29..65008f1797a 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -63,8 +63,7 @@ impl Future for Connecting { loop { match connection.poll_event(cx) { - Poll::Ready(ConnectionEvent::Connected) => { - let peer_id = connection.remote_peer_id(); + Poll::Ready(ConnectionEvent::Connected(peer_id)) => { let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } From 9e394b3f7e7c31df2e1db6ea19d16a80d914d885 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 12 Oct 2022 20:51:37 +0200 Subject: [PATCH 152/218] transports/quic: minor changes on closing logic - Remove `closed` flag in `Connection::poll_event`, instead return `ConnectionLost` events directly. - Explicitly close inner connection and wake all substreams if connection closed due to an connection error. --- transports/quic/src/connection.rs | 26 ++++------------ transports/quic/src/muxer.rs | 52 ++++++++++++++++++------------- 2 files changed, 37 insertions(+), 41 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 5b9cb99b0ab..2707808a9c5 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -183,7 +183,6 @@ impl Connection { /// Polls the connection for an event that happened on it. pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { - let mut closed = None; loop { match self.from_endpoint.poll_next_unpin(cx) { Poll::Ready(Some(event)) => { @@ -191,11 +190,9 @@ impl Connection { continue; } Poll::Ready(None) => { - if closed.is_none() { - return Poll::Ready(ConnectionEvent::ConnectionLost( - ConnectionError::EndpointDriverCrashed, - )); - } + return Poll::Ready(ConnectionEvent::ConnectionLost( + ConnectionError::EndpointDriverCrashed, + )); } Poll::Pending => {} } @@ -210,7 +207,7 @@ impl Connection { // being full to the user. if let Some(to_endpoint) = self.pending_to_endpoint.take() { match self.endpoint_channel.try_send(to_endpoint, cx) { - Ok(Ok(())) => {} + Ok(Ok(())) => continue, // The endpoint may send back an event. Ok(Err(to_endpoint)) => { self.pending_to_endpoint = Some(to_endpoint); return Poll::Pending; @@ -268,21 +265,10 @@ impl Connection { continue; } - if let Some(closed) = closed { - return Poll::Ready(ConnectionEvent::ConnectionLost(closed)); - } - // The final step consists in handling the events related to the various substreams. if let Some(ev) = self.connection.poll() { - match self.parse_connection_event(ev) { - ConnectionEvent::ConnectionLost(reason) => { - // Continue in the loop once more so that we can send a - // `EndpointEvent::drained` to the endpoint before returning. - closed = Some(reason); - continue; - } - event => return Poll::Ready(event), - } + let event = self.parse_connection_event(ev); + return Poll::Ready(event); } return Poll::Pending; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index e36c138a148..ee2d88d3c39 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -55,17 +55,6 @@ struct Inner { poll_connection_waker: Option, } -/// State of a single substream. -#[derive(Debug, Default, Clone)] -struct SubstreamState { - /// Waker to wake if the substream becomes readable or stopped. - read_waker: Option, - /// Waker to wake if the substream becomes writable or stopped. - write_waker: Option, - /// Waker to wake if the substream becomes closed or stopped. - finished_waker: Option, -} - impl QuicMuxer { /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. pub(crate) fn from_connection(connection: Connection) -> Self { @@ -101,7 +90,11 @@ impl StreamMuxer for QuicMuxer { event ); } - ConnectionEvent::ConnectionLost(err) => return Poll::Ready(Err(err)), + ConnectionEvent::ConnectionLost(err) => { + inner.connection.close(); + inner.substreams.values_mut().for_each(|s| s.wake_all()); + return Poll::Ready(Err(err)); + } ConnectionEvent::StreamOpened => { if let Some(waker) = inner.poll_outbound_waker.take() { waker.wake(); @@ -124,15 +117,7 @@ impl StreamMuxer for QuicMuxer { ConnectionEvent::StreamFinished(substream) | ConnectionEvent::StreamStopped(substream) => { if let Some(substream) = inner.substreams.get_mut(&substream) { - if let Some(waker) = substream.read_waker.take() { - waker.wake(); - } - if let Some(waker) = substream.write_waker.take() { - waker.wake(); - } - if let Some(waker) = substream.finished_waker.take() { - waker.wake(); - } + substream.wake_all(); } } ConnectionEvent::StreamAvailable => { @@ -219,6 +204,31 @@ impl StreamMuxer for QuicMuxer { } } +/// State of a single substream. +#[derive(Debug, Default, Clone)] +struct SubstreamState { + /// Waker to wake if the substream becomes readable or stopped. + read_waker: Option, + /// Waker to wake if the substream becomes writable or stopped. + write_waker: Option, + /// Waker to wake if the substream becomes closed or stopped. + finished_waker: Option, +} + +impl SubstreamState { + fn wake_all(&mut self) { + if let Some(waker) = self.read_waker.take() { + waker.wake(); + } + if let Some(waker) = self.write_waker.take() { + waker.wake(); + } + if let Some(waker) = self.finished_waker.take() { + waker.wake(); + } + } +} + #[derive(Debug)] pub struct Substream { id: quinn_proto::StreamId, From 7e48fe47d7bfa16d0ebfe51a870da8cb72f1ef0b Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 12 Oct 2022 22:36:38 +0200 Subject: [PATCH 153/218] transports/quic: wake connection on substream write --- transports/quic/src/muxer.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index ee2d88d3c39..337caad09e2 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -309,7 +309,12 @@ impl AsyncWrite for Substream { let mut muxer = self.muxer.lock(); match muxer.connection.send_stream(self.id).write(buf) { - Ok(bytes) => Poll::Ready(Ok(bytes)), + Ok(bytes) => { + if let Some(waker) = muxer.poll_connection_waker.take() { + waker.wake(); + } + Poll::Ready(Ok(bytes)) + } Err(quinn_proto::WriteError::Blocked) => { let substream = muxer .substreams From fdca477c381a71cd8c52be341c3f74bdc8fd0198 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Wed, 12 Oct 2022 23:33:11 +0200 Subject: [PATCH 154/218] transports/quic: suggestions from review --- transports/quic/src/endpoint.rs | 16 ++++++++-------- transports/quic/src/lib.rs | 2 +- transports/quic/src/transport.rs | 7 +++++-- 3 files changed, 14 insertions(+), 11 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 21281bdc56f..1f6c1fffd27 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -187,7 +187,7 @@ impl Channel { let server_config = new_connections .is_some() .then_some(quinn_config.server_config); - let socket = P::from_socket(socket)?; + let provider_socket = P::from_socket(socket)?; let driver = EndpointDriver::

::new( quinn_config.endpoint_config, @@ -195,7 +195,7 @@ impl Channel { new_connections, server_config, channel.clone(), - socket, + provider_socket, to_endpoint_rx, ); @@ -276,7 +276,7 @@ pub enum ToEndpoint { SendUdpPacket(quinn_proto::Transmit), /// The [`GenTransport`][transport::GenTransport] dialer or listener coupled to this endpoint /// was dropped. - /// Once all pending connection closed the [`EndpointDriver`] should shut down. + /// Once all pending connections are closed, the [`EndpointDriver`] should shut down. Decoupled, } @@ -369,7 +369,7 @@ pub struct EndpointDriver { rx: mpsc::Receiver, // Socket for sending and receiving datagrams. - socket: P, + provider_socket: P, // Future for writing the next packet to the socket. next_packet_out: Option, @@ -398,7 +398,7 @@ impl EndpointDriver

{ client_config, channel, rx, - socket, + provider_socket: socket, next_packet_out: None, alive_connections: HashMap::new(), new_connection_tx, @@ -563,7 +563,7 @@ impl Future for EndpointDriver

{ type Output = (); fn poll(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { - match ready!(self.socket.poll_send_flush(cx)) { + match ready!(self.provider_socket.poll_send_flush(cx)) { Ok(_) => {} // Errors on the socket are expected to never happen, and we handle them by simply // printing a log message. The packet gets discarded in case of error, but we are @@ -575,7 +575,7 @@ impl Future for EndpointDriver

{ } if let Some(transmit) = self.next_packet_out.take() { - self.socket + self.provider_socket .start_send(transmit.contents, transmit.destination); continue; } @@ -598,7 +598,7 @@ impl Future for EndpointDriver

{ Poll::Pending => {} } - match self.socket.poll_recv_from(cx) { + match self.provider_socket.poll_recv_from(cx) { Poll::Ready(Ok((bytes, packet_src))) => { let bytes_mut = bytes.as_bytes().into(); match self.handle_datagram(bytes_mut, packet_src) { diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 001b6d94901..49244bd217f 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -39,7 +39,7 @@ //! //! let mut quic_transport = quic::async_std::Transport::new(quic_config); //! -//! let addr = "/ip4/127.0.0.1/udp/12345/quic".parse().expect("bad address?"); +//! let addr = "/ip4/127.0.0.1/udp/12345/quic".parse().expect("address should be valid"); //! quic_transport.listen_on(addr).expect("listen error."); //! # //! # Ok(()) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index bbd96aa7b0d..6ce8893515d 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -158,6 +158,9 @@ impl Transport for GenTransport

{ .listeners .iter_mut() .filter(|l| { + if l.is_closed { + return false; + } let listen_addr = l.endpoint_channel.socket_addr(); SocketFamily::is_same(&listen_addr.ip(), &socket_addr.ip()) && listen_addr.ip().is_loopback() == socket_addr.ip().is_loopback() @@ -198,7 +201,7 @@ impl Transport for GenTransport

{ // `addr`. See DCUtR specification below. // // https://github.com/libp2p/specs/blob/master/relay/DCUtR.md#the-protocol - self.dial(addr) + Err(CoreTransportError::MultiaddrNotSupported(addr)) } fn poll( @@ -432,7 +435,7 @@ impl Listener { dialer_state, endpoint_channel, .. - } = &mut *self; + } = self; dialer_state.poll(endpoint_channel, cx) } From 8d3147a24c41a82fde7cb2ea78201844f31cb557 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 13 Oct 2022 00:06:24 +0200 Subject: [PATCH 155/218] transports/quic: suggestions from code review Co-authored-by: Marco Munizaga --- transports/quic/src/endpoint.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 1f6c1fffd27..897c64b7d52 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -464,7 +464,7 @@ impl EndpointDriver

{ // We clone the sender to guarantee that there will be at least one // free slot to send the event. // The channel can not grow out of bound because an `event_back` is - // only sent if we prior received an event from the same connection. + // only sent if we previously received an event from the same connection. // If the connection is busy, it won't sent us any more events to handle. let _ = sender.clone().start_send(event_back); } else { @@ -567,7 +567,7 @@ impl Future for EndpointDriver

{ Ok(_) => {} // Errors on the socket are expected to never happen, and we handle them by simply // printing a log message. The packet gets discarded in case of error, but we are - // robust to packet losses and it is consequently not a logic error to process with + // robust to packet losses and it is consequently not a logic error to proceed with // normal operations. Err(err) => { log::error!("Error while sending on QUIC UDP socket: {:?}", err) From 82b6a9d078c09ae25bc4fc9bf34c8240783c5431 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 13 Oct 2022 00:43:00 +0200 Subject: [PATCH 156/218] transports/quic: suggestions from code review --- transports/quic/src/muxer.rs | 61 ++++++++++++++++++++---------------- 1 file changed, 34 insertions(+), 27 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 337caad09e2..117201adf66 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -55,6 +55,14 @@ struct Inner { poll_connection_waker: Option, } +impl Inner { + fn unchecked_substream_state(&mut self, id: quinn_proto::StreamId) -> &mut SubstreamState { + self.substreams + .get_mut(&id) + .expect("Substream should be known.") + } +} + impl QuicMuxer { /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. pub(crate) fn from_connection(connection: Connection) -> Self { @@ -114,8 +122,13 @@ impl StreamMuxer for QuicMuxer { } } } - ConnectionEvent::StreamFinished(substream) - | ConnectionEvent::StreamStopped(substream) => { + ConnectionEvent::StreamFinished(substream) => { + if let Some(substream) = inner.substreams.get_mut(&substream) { + substream.wake_all(); + substream.is_write_closed = true; + } + } + ConnectionEvent::StreamStopped(substream) => { if let Some(substream) = inner.substreams.get_mut(&substream) { substream.wake_all(); } @@ -186,13 +199,10 @@ impl StreamMuxer for QuicMuxer { return Poll::Ready(Ok(())); } - if connection.send_stream_count() != 0 { - for substream in substreams.keys() { - if let Err(e) = connection.finish_substream(*substream) { - log::warn!("substream finish error on muxer close: {}", e); - } - } + for substream in substreams.keys() { + let _ = connection.finish_substream(*substream); } + loop { if connection.send_stream_count() == 0 && !connection.is_closed() { connection.close() @@ -213,6 +223,8 @@ struct SubstreamState { write_waker: Option, /// Waker to wake if the substream becomes closed or stopped. finished_waker: Option, + + is_write_closed: bool, } impl SubstreamState { @@ -264,12 +276,9 @@ impl AsyncRead for Substream { let mut bytes = 0; let mut pending = false; loop { - if buf.is_empty() { - break; - } let chunk = match chunks.next(buf.len()) { - Ok(Some(chunk)) => chunk, - Ok(None) => break, + Ok(Some(chunk)) if !chunk.bytes.is_empty() => chunk, + Ok(_) => break, Err(err @ quinn_proto::ReadError::Reset(_)) => { return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } @@ -288,10 +297,7 @@ impl AsyncRead for Substream { } } if pending && bytes == 0 { - let substream_state = muxer - .substreams - .get_mut(&self.id) - .expect("known substream; qed"); + let substream_state = muxer.unchecked_substream_state(self.id); substream_state.read_waker = Some(cx.waker().clone()); Poll::Pending } else { @@ -316,11 +322,8 @@ impl AsyncWrite for Substream { Poll::Ready(Ok(bytes)) } Err(quinn_proto::WriteError::Blocked) => { - let substream = muxer - .substreams - .get_mut(&self.id) - .expect("known substream; qed"); - substream.write_waker = Some(cx.waker().clone()); + let substream_state = muxer.unchecked_substream_state(self.id); + substream_state.write_waker = Some(cx.waker().clone()); Poll::Pending } Err(err @ quinn_proto::WriteError::Stopped(_)) => { @@ -339,19 +342,23 @@ impl AsyncWrite for Substream { fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { let mut muxer = self.muxer.lock(); + + if muxer.unchecked_substream_state(self.id).is_write_closed { + return Poll::Ready(Ok(())); + } + match muxer.connection.finish_substream(self.id) { Ok(()) => { - let substream_state = muxer - .substreams - .get_mut(&self.id) - .expect("Substream is not finished."); + let substream_state = muxer.unchecked_substream_state(self.id); substream_state.finished_waker = Some(cx.waker().clone()); Poll::Pending } Err(err @ quinn_proto::FinishError::Stopped(_)) => { Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } - Err(quinn_proto::FinishError::UnknownStream) => Poll::Ready(Ok(())), + Err(quinn_proto::FinishError::UnknownStream) => { + Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())) + } } } } From b89ef3388a3fbf911ef35f1da9c3c0f43652359e Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 13 Oct 2022 14:58:19 +0200 Subject: [PATCH 157/218] transport/quic: select listener deterministically --- transports/quic/src/transport.rs | 39 ++++++++++++++++---------------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 6ce8893515d..dda60ce5894 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -42,10 +42,9 @@ use libp2p_core::{ PeerId, Transport, }; use quinn_proto::ConnectError; -use rand::prelude::SliceRandom; -use rand::thread_rng; -use std::collections::hash_map::Entry; +use std::collections::hash_map::{DefaultHasher, Entry}; use std::collections::{HashMap, VecDeque}; +use std::hash::{Hash, Hasher}; use std::marker::PhantomData; use std::net::IpAddr; use std::task::Waker; @@ -167,23 +166,25 @@ impl Transport for GenTransport

{ }) .collect::>(); - // Try to use pick a random listener to use for dialing. - let dialing = match listeners.choose_mut(&mut thread_rng()) { - Some(listener) => listener.dialer_state.new_dial(socket_addr), - None => { - // No listener? Get or create an explicit dialer. - - let socket_family = socket_addr.ip().into(); - let dialer = match self.dialer.entry(socket_family) { - Entry::Occupied(occupied) => occupied.into_mut(), - Entry::Vacant(vacant) => { - vacant.insert(Dialer::new::

(self.quinn_config.clone(), socket_family)?) - } - }; - - dialer.state.new_dial(socket_addr) - } + let dialing = if !listeners.is_empty() { + // Pick any listener to use for dialing. + // We hash the socket address to achieve determinism. + let mut hasher = DefaultHasher::new(); + socket_addr.hash(&mut hasher); + let index = hasher.finish() as usize % listeners.len(); + listeners[index].dialer_state.new_dial(socket_addr) + } else { + // No listener? Get or create an explicit dialer. + let socket_family = socket_addr.ip().into(); + let dialer = match self.dialer.entry(socket_family) { + Entry::Occupied(occupied) => occupied.into_mut(), + Entry::Vacant(vacant) => { + vacant.insert(Dialer::new::

(self.quinn_config.clone(), socket_family)?) + } + }; + dialer.state.new_dial(socket_addr) }; + let handshake_timeout = self.handshake_timeout; Ok(async move { let connection = dialing.await??; From 3fa3e9ef635b991a14d49898fd72570eea612594 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 13 Oct 2022 15:16:25 +0200 Subject: [PATCH 158/218] transports/quic: enhance docs --- transports/quic/src/transport.rs | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index dda60ce5894..b08e75dcffa 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -133,6 +133,8 @@ impl Transport for GenTransport

{ fn remove_listener(&mut self, id: ListenerId) -> bool { if let Some(listener) = self.listeners.iter_mut().find(|l| l.listener_id == id) { + // Close the listener, which will eventually finish its stream. + // `SelectAll` removes streams once they are finished. listener.close(Ok(())); true } else { @@ -517,9 +519,13 @@ impl From for SocketFamily { } } -/// Turn an [`IpAddr`] into a listen-address for the endpoint. +/// Turn an [`IpAddr`] reported byt the [`IfWatcher`] into a +/// listen-address for the endpoint. /// -/// Returns `None` if the address is not the same socket family as the +/// For this, the `ip` is combined with the port that the endpoint +/// is actually bound. +/// +/// Returns `None` if the `ip` is not the same socket family as the /// address that the endpoint is bound to. fn ip_to_listenaddr(endpoint_addr: &SocketAddr, ip: IpAddr) -> Option { // True if either both addresses are Ipv4 or both Ipv6. @@ -681,7 +687,7 @@ mod test { .now_or_never() .is_none()); - // Run test twice to check that there is no unexpected behaviour if `GenTransport.listener` + // Run test twice to check that there is no unexpected behaviour if `Transport.listener` // is temporarily empty. for _ in 0..2 { let listener = transport From 6430af0d458436c75a258446ccd69369b383d684 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 12:19:30 +1100 Subject: [PATCH 159/218] Remove unnecessary `pub` qualifiers of utility functions --- transports/quic/src/transport.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index b08e75dcffa..b5bc603f990 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -538,7 +538,7 @@ fn ip_to_listenaddr(endpoint_addr: &SocketAddr, ip: IpAddr) -> Option /// Tries to turn a QUIC multiaddress into a UDP [`SocketAddr`]. Returns None if the format /// of the multiaddr is wrong. -pub fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { +fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { let mut iter = addr.iter(); let proto1 = iter.next()?; let proto2 = iter.next()?; @@ -562,7 +562,7 @@ pub fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { } } -pub fn is_quic_addr(addr: &Multiaddr) -> bool { +fn is_quic_addr(addr: &Multiaddr) -> bool { use Protocol::*; let mut iter = addr.iter(); let first = match iter.next() { @@ -583,7 +583,7 @@ pub fn is_quic_addr(addr: &Multiaddr) -> bool { } /// Turns an IP address and port into the corresponding QUIC multiaddr. -pub(crate) fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { +fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { Multiaddr::empty() .with(socket_addr.ip().into()) .with(Protocol::Udp(socket_addr.port())) From b77ee8ca4e745672a6d8b6d87c911c50438979f6 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 12:25:49 +1100 Subject: [PATCH 160/218] Don't repeat protocol name in muxer struct --- transports/quic/src/lib.rs | 2 +- transports/quic/src/muxer.rs | 8 ++++---- transports/quic/src/transport.rs | 4 ++-- transports/quic/src/upgrade.rs | 6 +++--- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 49244bd217f..699444990db 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -67,7 +67,7 @@ mod upgrade; pub use connection::ConnectionError; pub use endpoint::Config; -pub use muxer::QuicMuxer; +pub use muxer::Muxer; #[cfg(feature = "async-std")] pub use provider::async_std; #[cfg(feature = "tokio")] diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 117201adf66..962cf7b6872 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -36,7 +36,7 @@ use std::{ /// State for a single opened QUIC connection. #[derive(Debug)] -pub struct QuicMuxer { +pub struct Muxer { inner: Arc>, } @@ -63,10 +63,10 @@ impl Inner { } } -impl QuicMuxer { +impl Muxer { /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. pub(crate) fn from_connection(connection: Connection) -> Self { - QuicMuxer { + Muxer { inner: Arc::new(Mutex::new(Inner { connection, substreams: Default::default(), @@ -78,7 +78,7 @@ impl QuicMuxer { } } -impl StreamMuxer for QuicMuxer { +impl StreamMuxer for Muxer { type Substream = Substream; type Error = ConnectionError; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index b5bc603f990..edf989d6b6c 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -25,7 +25,7 @@ use crate::connection::{Connection, ConnectionError}; use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; use crate::provider::Provider; -use crate::{endpoint, muxer::QuicMuxer, upgrade::Connecting}; +use crate::{endpoint, muxer::Muxer, upgrade::Connecting}; use futures::channel::{mpsc, oneshot}; use futures::future::{BoxFuture, MapErr}; @@ -103,7 +103,7 @@ pub enum TransportError { } impl Transport for GenTransport

{ - type Output = (PeerId, QuicMuxer); + type Output = (PeerId, Muxer); type Error = TransportError; type ListenerUpgrade = Connecting; type Dial = BoxFuture<'static, Result>; diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 65008f1797a..189a1d52ec5 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -22,7 +22,7 @@ use crate::{ connection::{Connection, ConnectionEvent}, - muxer::QuicMuxer, + muxer::Muxer, transport, }; @@ -53,7 +53,7 @@ impl Connecting { } impl Future for Connecting { - type Output = Result<(PeerId, QuicMuxer), transport::TransportError>; + type Output = Result<(PeerId, Muxer), transport::TransportError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let connection = self @@ -64,7 +64,7 @@ impl Future for Connecting { loop { match connection.poll_event(cx) { Poll::Ready(ConnectionEvent::Connected(peer_id)) => { - let muxer = QuicMuxer::from_connection(self.connection.take().unwrap()); + let muxer = Muxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { From 60c2761e03d1fe44c14314f14862dde62959c304 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Thu, 6 Oct 2022 12:54:40 +1100 Subject: [PATCH 161/218] Have `new_dial` return a future --- transports/quic/src/transport.rs | 33 +++++++++++++++++--------------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index edf989d6b6c..07bd7c3ab06 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -28,20 +28,18 @@ use crate::provider::Provider; use crate::{endpoint, muxer::Muxer, upgrade::Connecting}; use futures::channel::{mpsc, oneshot}; -use futures::future::{BoxFuture, MapErr}; +use futures::future::BoxFuture; use futures::ready; use futures::stream::StreamExt; use futures::{prelude::*, stream::SelectAll}; use if_watch::{IfEvent, IfWatcher}; -use futures::channel::oneshot::{Canceled, Receiver}; use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, transport::{ListenerId, TransportError as CoreTransportError, TransportEvent}, PeerId, Transport, }; -use quinn_proto::ConnectError; use std::collections::hash_map::{DefaultHasher, Entry}; use std::collections::{HashMap, VecDeque}; use std::hash::{Hash, Hasher}; @@ -174,7 +172,9 @@ impl Transport for GenTransport

{ let mut hasher = DefaultHasher::new(); socket_addr.hash(&mut hasher); let index = hasher.finish() as usize % listeners.len(); - listeners[index].dialer_state.new_dial(socket_addr) + listeners[index] + .dialer_state + .new_dial(socket_addr, self.handshake_timeout) } else { // No listener? Get or create an explicit dialer. let socket_family = socket_addr.ip().into(); @@ -184,16 +184,10 @@ impl Transport for GenTransport

{ vacant.insert(Dialer::new::

(self.quinn_config.clone(), socket_family)?) } }; - dialer.state.new_dial(socket_addr) + dialer.state.new_dial(socket_addr, self.handshake_timeout) }; - let handshake_timeout = self.handshake_timeout; - Ok(async move { - let connection = dialing.await??; - let final_connec = Connecting::from_connection(connection, handshake_timeout).await?; - Ok(final_connec) - } - .boxed()) + Ok(dialing) } fn dial_as_listener( @@ -276,7 +270,8 @@ impl DialerState { fn new_dial( &mut self, address: SocketAddr, - ) -> MapErr>, fn(Canceled) -> TransportError> { + timeout: Duration, + ) -> BoxFuture<'static, Result<(PeerId, Muxer), TransportError>> { let (rx, tx) = oneshot::channel(); let message = ToEndpoint::Dial { @@ -290,8 +285,16 @@ impl DialerState { waker.wake(); } - // Our oneshot getting dropped means the message didn't make it to the endpoint driver. - tx.map_err(|_| TransportError::EndpointDriverCrashed) + async move { + // Our oneshot getting dropped means the message didn't make it to the endpoint driver. + let connection = tx + .await + .map_err(|_| TransportError::EndpointDriverCrashed)??; + let (peer, muxer) = Connecting::from_connection(connection, timeout).await?; + + Ok((peer, muxer)) + } + .boxed() } /// Send all pending dials into the given [`endpoint::Channel`]. From c0b8a55c100346c49b4428dca8642304f6238c5f Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 13 Oct 2022 16:27:15 +0200 Subject: [PATCH 162/218] transports/quic: add single `Error` for crate Add single `Error` instead of `TransportError` and `ConnectionError`. Add `ConnectionError` and `ConnectError` newtypes to wrap `quinn_proto` errors. --- transports/quic/src/connection.rs | 25 +++------ transports/quic/src/endpoint.rs | 15 +++--- transports/quic/src/error.rs | 49 ++++++++++++++++++ transports/quic/src/lib.rs | 6 +-- transports/quic/src/muxer.rs | 4 +- transports/quic/src/transport.rs | 86 ++++++++++--------------------- transports/quic/src/upgrade.rs | 13 ++--- 7 files changed, 99 insertions(+), 99 deletions(-) create mode 100644 transports/quic/src/error.rs diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 2707808a9c5..0c8bd6dd6a3 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -26,7 +26,10 @@ //! All interactions with a QUIC connection should be done through this struct. // TODO: docs -use crate::endpoint::{self, ToEndpoint}; +use crate::{ + endpoint::{self, ToEndpoint}, + ConnectionError, Error, +}; use futures::{channel::mpsc, prelude::*}; use futures_timer::Delay; use libp2p_core::PeerId; @@ -59,18 +62,6 @@ pub struct Connection { next_timeout: Option<(Delay, Instant)>, } -/// Error on the connection as a whole. -#[derive(Debug, Clone, thiserror::Error)] -pub enum ConnectionError { - /// The task driving the endpoint has crashed. - #[error("Endpoint driver crashed")] - EndpointDriverCrashed, - - /// Error in the inner state machine. - #[error(transparent)] - Quinn(#[from] quinn_proto::ConnectionError), -} - impl Connection { /// Crate-internal function that builds a [`Connection`] from raw components. /// @@ -191,7 +182,7 @@ impl Connection { } Poll::Ready(None) => { return Poll::Ready(ConnectionEvent::ConnectionLost( - ConnectionError::EndpointDriverCrashed, + Error::EndpointDriverCrashed, )); } Poll::Pending => {} @@ -214,7 +205,7 @@ impl Connection { } Err(endpoint::Disconnected {}) => { return Poll::Ready(ConnectionEvent::ConnectionLost( - ConnectionError::EndpointDriverCrashed, + Error::EndpointDriverCrashed, )); } } @@ -309,7 +300,7 @@ impl Connection { dir: quinn_proto::Dir::Bi, }) => ConnectionEvent::StreamOpened, quinn_proto::Event::ConnectionLost { reason } => { - ConnectionEvent::ConnectionLost(ConnectionError::Quinn(reason)) + ConnectionEvent::ConnectionLost(ConnectionError::from(reason).into()) } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { ConnectionEvent::StreamFinished(id) @@ -345,7 +336,7 @@ pub enum ConnectionEvent { Connected(PeerId), /// Connection has been closed and can no longer be used. - ConnectionLost(ConnectionError), + ConnectionLost(Error), /// Generated after [`Connection::accept_substream`] has been called and has returned /// `None`. After this event has been generated, this method is guaranteed to return `Some`. diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 897c64b7d52..45abe3d7da8 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -29,10 +29,7 @@ //! [`EndpointDriver`] for a thorough description. use crate::{ - connection::Connection, - provider::Provider, - tls, - transport::{self, SocketFamily}, + connection::Connection, provider::Provider, tls, transport::SocketFamily, ConnectError, Error, }; use bytes::BytesMut; @@ -151,7 +148,7 @@ impl Channel { pub fn new_bidirectional( quinn_config: QuinnConfig, socket_addr: SocketAddr, - ) -> Result<(Self, mpsc::Receiver), transport::TransportError> { + ) -> Result<(Self, mpsc::Receiver), Error> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Self::new::

(quinn_config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) @@ -161,7 +158,7 @@ impl Channel { pub fn new_dialer( quinn_config: QuinnConfig, socket_family: SocketFamily, - ) -> Result { + ) -> Result { let socket_addr = match socket_family { SocketFamily::Ipv4 => SocketAddr::new(Ipv4Addr::UNSPECIFIED.into(), 0), SocketFamily::Ipv6 => SocketAddr::new(Ipv6Addr::UNSPECIFIED.into(), 0), @@ -173,7 +170,7 @@ impl Channel { quinn_config: QuinnConfig, socket_addr: SocketAddr, new_connections: Option>, - ) -> Result { + ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; socket.set_nonblocking(true)?; @@ -263,7 +260,7 @@ pub enum ToEndpoint { /// UDP address to connect to. addr: SocketAddr, /// Channel to return the result of the dialing to. - result: oneshot::Sender>, + result: 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. Only `quinn_proto` knows what is in @@ -417,7 +414,7 @@ impl EndpointDriver

{ match self.endpoint.connect(self.client_config.clone(), addr, "l") { Ok(c) => c, Err(err) => { - let _ = result.send(Err(err)); + let _ = result.send(Err(ConnectError::from(err).into())); return ControlFlow::Continue(()); } }; diff --git a/transports/quic/src/error.rs b/transports/quic/src/error.rs new file mode 100644 index 00000000000..9fbb074f11c --- /dev/null +++ b/transports/quic/src/error.rs @@ -0,0 +1,49 @@ +// Copyright 2022 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. + +/// Error that can happen on the transport. +#[derive(Debug, thiserror::Error)] +pub enum Error { + /// Error while trying to reach a remote. + #[error(transparent)] + Reach(#[from] ConnectError), + + /// Error after the remote has been reached. + #[error(transparent)] + Connection(#[from] ConnectionError), + + #[error(transparent)] + Io(#[from] std::io::Error), + + /// The task driving the endpoint has crashed. + #[error("Endpoint driver crashed")] + EndpointDriverCrashed, + + #[error("Handshake with the remote timed out.")] + HandshakeTimedOut, +} + +#[derive(Debug, thiserror::Error)] +#[error(transparent)] +pub struct ConnectError(#[from] quinn_proto::ConnectError); + +#[derive(Debug, thiserror::Error)] +#[error(transparent)] +pub struct ConnectionError(#[from] quinn_proto::ConnectionError); diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 699444990db..ac39293e090 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -59,20 +59,20 @@ mod connection; mod endpoint; +mod error; mod muxer; mod provider; mod tls; mod transport; mod upgrade; -pub use connection::ConnectionError; pub use endpoint::Config; +pub use error::{ConnectError, ConnectionError, Error}; pub use muxer::Muxer; #[cfg(feature = "async-std")] pub use provider::async_std; #[cfg(feature = "tokio")] pub use provider::tokio; pub use provider::Provider; -pub use quinn_proto::ConnectError as DialError; -pub use transport::{GenTransport, TransportError}; +pub use transport::GenTransport; pub use upgrade::Connecting; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 962cf7b6872..e9c2b87b132 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -20,7 +20,7 @@ use crate::{ connection::{Connection, ConnectionEvent}, - ConnectionError, + Error, }; use futures::{ready, AsyncRead, AsyncWrite}; @@ -80,7 +80,7 @@ impl Muxer { impl StreamMuxer for Muxer { type Substream = Substream; - type Error = ConnectionError; + type Error = Error; fn poll( self: Pin<&mut Self>, diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 07bd7c3ab06..d1844065920 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -22,10 +22,10 @@ //! //! Combines all the objects in the other modules to implement the trait. -use crate::connection::{Connection, ConnectionError}; +use crate::connection::Connection; use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; use crate::provider::Provider; -use crate::{endpoint, muxer::Muxer, upgrade::Connecting}; +use crate::{endpoint, muxer::Muxer, upgrade::Connecting, Error}; use futures::channel::{mpsc, oneshot}; use futures::future::BoxFuture; @@ -37,7 +37,7 @@ use if_watch::{IfEvent, IfWatcher}; use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, - transport::{ListenerId, TransportError as CoreTransportError, TransportEvent}, + transport::{ListenerId, TransportError, TransportEvent}, PeerId, Transport, }; use std::collections::hash_map::{DefaultHasher, Entry}; @@ -79,39 +79,15 @@ impl

GenTransport

{ } } -/// Error that can happen on the transport. -#[derive(Debug, thiserror::Error)] -pub enum TransportError { - /// Error while trying to reach a remote. - #[error(transparent)] - Reach(#[from] quinn_proto::ConnectError), - /// Error after the remote has been reached. - #[error(transparent)] - Established(#[from] ConnectionError), - - #[error(transparent)] - Io(#[from] std::io::Error), - - /// The task driving the endpoint has crashed. - #[error("Endpoint driver crashed")] - EndpointDriverCrashed, - - #[error("Handshake with the remote timed out.")] - HandshakeTimedOut, -} - impl Transport for GenTransport

{ type Output = (PeerId, Muxer); - type Error = TransportError; + type Error = Error; type ListenerUpgrade = Connecting; type Dial = BoxFuture<'static, Result>; - fn listen_on( - &mut self, - addr: Multiaddr, - ) -> Result> { - let socket_addr = multiaddr_to_socketaddr(&addr) - .ok_or(CoreTransportError::MultiaddrNotSupported(addr))?; + fn listen_on(&mut self, addr: Multiaddr) -> Result> { + let socket_addr = + multiaddr_to_socketaddr(&addr).ok_or(TransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); let listener = Listener::new::

( listener_id, @@ -147,11 +123,11 @@ impl Transport for GenTransport

{ Some(observed.clone()) } - fn dial(&mut self, addr: Multiaddr) -> Result> { + fn dial(&mut self, addr: Multiaddr) -> Result> { let socket_addr = multiaddr_to_socketaddr(&addr) - .ok_or_else(|| CoreTransportError::MultiaddrNotSupported(addr.clone()))?; + .ok_or_else(|| TransportError::MultiaddrNotSupported(addr.clone()))?; if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { - return Err(CoreTransportError::MultiaddrNotSupported(addr)); + return Err(TransportError::MultiaddrNotSupported(addr)); } let mut listeners = self .listeners @@ -193,12 +169,12 @@ impl Transport for GenTransport

{ fn dial_as_listener( &mut self, addr: Multiaddr, - ) -> Result> { + ) -> Result> { // TODO: As the listener of a QUIC hole punch, we need to send a random UDP packet to the // `addr`. See DCUtR specification below. // // https://github.com/libp2p/specs/blob/master/relay/DCUtR.md#the-protocol - Err(CoreTransportError::MultiaddrNotSupported(addr)) + Err(TransportError::MultiaddrNotSupported(addr)) } fn poll( @@ -223,9 +199,9 @@ impl Transport for GenTransport

{ } } -impl From for CoreTransportError { - fn from(err: TransportError) -> Self { - CoreTransportError::Other(err) +impl From for TransportError { + fn from(err: Error) -> Self { + TransportError::Other(err) } } @@ -239,16 +215,16 @@ impl Dialer { fn new( config: QuinnConfig, socket_family: SocketFamily, - ) -> Result> { + ) -> Result> { let endpoint_channel = endpoint::Channel::new_dialer::

(config, socket_family) - .map_err(CoreTransportError::Other)?; + .map_err(TransportError::Other)?; Ok(Dialer { endpoint_channel, state: DialerState::default(), }) } - fn poll(&mut self, cx: &mut Context<'_>) -> Poll { + fn poll(&mut self, cx: &mut Context<'_>) -> Poll { self.state.poll(&mut self.endpoint_channel, cx) } } @@ -271,7 +247,7 @@ impl DialerState { &mut self, address: SocketAddr, timeout: Duration, - ) -> BoxFuture<'static, Result<(PeerId, Muxer), TransportError>> { + ) -> BoxFuture<'static, Result<(PeerId, Muxer), Error>> { let (rx, tx) = oneshot::channel(); let message = ToEndpoint::Dial { @@ -287,9 +263,7 @@ impl DialerState { async move { // Our oneshot getting dropped means the message didn't make it to the endpoint driver. - let connection = tx - .await - .map_err(|_| TransportError::EndpointDriverCrashed)??; + let connection = tx.await.map_err(|_| Error::EndpointDriverCrashed)??; let (peer, muxer) = Connecting::from_connection(connection, timeout).await?; Ok((peer, muxer)) @@ -300,11 +274,7 @@ impl DialerState { /// Send all pending dials into the given [`endpoint::Channel`]. /// /// This only ever returns [`Poll::Pending`] or an error in case the channel is closed. - fn poll( - &mut self, - channel: &mut endpoint::Channel, - cx: &mut Context<'_>, - ) -> Poll { + fn poll(&mut self, channel: &mut endpoint::Channel, cx: &mut Context<'_>) -> Poll { while let Some(to_endpoint) = self.pending_dials.pop_front() { match channel.try_send(to_endpoint, cx) { Ok(Ok(())) => {} @@ -312,9 +282,7 @@ impl DialerState { self.pending_dials.push_front(to_endpoint); break; } - Err(endpoint::Disconnected {}) => { - return Poll::Ready(TransportError::EndpointDriverCrashed) - } + Err(endpoint::Disconnected {}) => return Poll::Ready(Error::EndpointDriverCrashed), } } self.waker = Some(cx.waker().clone()); @@ -351,7 +319,7 @@ impl Listener { socket_addr: SocketAddr, config: QuinnConfig, handshake_timeout: Duration, - ) -> Result { + ) -> Result { let (endpoint_channel, new_connections_rx) = endpoint::Channel::new_bidirectional::

(config, socket_addr)?; @@ -384,7 +352,7 @@ impl Listener { /// Report the listener as closed in a [`TransportEvent::ListenerClosed`] and /// terminate the stream. - fn close(&mut self, reason: Result<(), TransportError>) { + fn close(&mut self, reason: Result<(), Error>) { if self.is_closed { return; } @@ -436,7 +404,7 @@ impl Listener { } /// Poll [`DialerState`] to initiate requested dials. - fn poll_dialer(&mut self, cx: &mut Context<'_>) -> Poll { + fn poll_dialer(&mut self, cx: &mut Context<'_>) -> Poll { let Self { dialer_state, endpoint_channel, @@ -448,7 +416,7 @@ impl Listener { } impl Stream for Listener { - type Item = TransportEvent; + type Item = TransportEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { if let Some(event) = self.pending_event.take() { @@ -481,7 +449,7 @@ impl Stream for Listener { return Poll::Ready(Some(event)); } Poll::Ready(None) => { - self.close(Err(TransportError::EndpointDriverCrashed)); + self.close(Err(Error::EndpointDriverCrashed)); continue; } Poll::Pending => {} diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 189a1d52ec5..68fa4d1f91f 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -22,8 +22,7 @@ use crate::{ connection::{Connection, ConnectionEvent}, - muxer::Muxer, - transport, + Error, Muxer, }; use futures::prelude::*; @@ -53,7 +52,7 @@ impl Connecting { } impl Future for Connecting { - type Output = Result<(PeerId, Muxer), transport::TransportError>; + type Output = Result<(PeerId, Muxer), Error>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let connection = self @@ -67,9 +66,7 @@ impl Future for Connecting { let muxer = Muxer::from_connection(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } - Poll::Ready(ConnectionEvent::ConnectionLost(err)) => { - return Poll::Ready(Err(err.into())) - } + Poll::Ready(ConnectionEvent::ConnectionLost(err)) => return Poll::Ready(Err(err)), Poll::Ready(ConnectionEvent::HandshakeDataReady) | Poll::Ready(ConnectionEvent::StreamAvailable) | Poll::Ready(ConnectionEvent::StreamOpened) @@ -80,9 +77,7 @@ impl Future for Connecting { Poll::Pending => {} } match self.timeout.poll_unpin(cx) { - Poll::Ready(()) => { - return Poll::Ready(Err(transport::TransportError::HandshakeTimedOut)) - } + Poll::Ready(()) => return Poll::Ready(Err(Error::HandshakeTimedOut)), Poll::Pending => {} } return Poll::Pending; From 2eafbe6d430534ee3f0275b02209768375519ec5 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 14 Oct 2022 12:35:57 +0200 Subject: [PATCH 163/218] transports/quic: test EndpointDriver drop on listener close --- transports/quic/src/endpoint.rs | 2 +- transports/quic/src/transport.rs | 30 +++++++++++++++++++++++------- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 45abe3d7da8..6572c9772b8 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -248,7 +248,7 @@ impl Channel { } } -#[derive(Debug, thiserror::Error)] +#[derive(Debug, thiserror::Error, Clone, PartialEq, Eq)] #[error("Background task disconnected")] pub struct Disconnected {} diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index d1844065920..3c32d80079c 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -567,6 +567,7 @@ mod test { #[cfg(feature = "async-std")] use async_std_crate as async_std; use futures::future::poll_fn; + use futures_timer::Delay; use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; #[cfg(feature = "tokio")] use tokio_crate as tokio; @@ -661,15 +662,25 @@ mod test { // Run test twice to check that there is no unexpected behaviour if `Transport.listener` // is temporarily empty. for _ in 0..2 { - let listener = transport + let id = transport .listen_on("/ip4/0.0.0.0/udp/0/quic".parse().unwrap()) .unwrap(); + + // Copy channel to use it later. + let mut channel = transport + .listeners + .iter() + .next() + .unwrap() + .endpoint_channel + .clone(); + match poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)).await { TransportEvent::NewAddress { listener_id, listen_addr, } => { - assert_eq!(listener_id, listener); + assert_eq!(listener_id, id); assert!( matches!(listen_addr.iter().next(), Some(Protocol::Ip4(a)) if !a.is_unspecified()) ); @@ -680,16 +691,13 @@ mod test { } e => panic!("Unexpected event: {:?}", e), } - assert!( - transport.remove_listener(listener), - "Expect listener to exist." - ); + assert!(transport.remove_listener(id), "Expect listener to exist."); match poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)).await { TransportEvent::ListenerClosed { listener_id, reason: Ok(()), } => { - assert_eq!(listener_id, listener); + assert_eq!(listener_id, id); } e => panic!("Unexpected event: {:?}", e), } @@ -699,6 +707,14 @@ mod test { .now_or_never() .is_none()); assert!(transport.listeners.is_empty()); + + // Check that the [`EndpointDriver`] has shut down. + Delay::new(Duration::from_millis(10)).await; + poll_fn(|cx| { + assert!(channel.try_send(ToEndpoint::Decoupled, cx).is_err()); + Poll::Ready(()) + }) + .await; } } } From f880d70da117a57623c3e2095206ae5a923f9680 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 14 Oct 2022 12:37:09 +0200 Subject: [PATCH 164/218] transports/quic: fix intra-doc link --- transports/quic/src/connection.rs | 2 +- transports/quic/src/endpoint.rs | 4 ++-- transports/quic/src/muxer.rs | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 0c8bd6dd6a3..4e9baf8bf92 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -39,7 +39,7 @@ use std::{ time::Instant, }; -/// Underlying structure for both [`crate::QuicMuxer`] and [`crate::Connecting`]. +/// Underlying structure for both [`crate::Muxer`] and [`crate::Connecting`]. /// /// Contains everything needed to process a connection with a remote. /// Tied to a specific endpoint. diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 6572c9772b8..3fc6c0e46e6 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -118,7 +118,7 @@ impl From for QuinnConfig { server_config.transport = Arc::clone(&transport); // Disables connection migration. // Long-term this should be enabled, however we then need to handle address change - // on connections in the `QuicMuxer`. + // on connections in the `Muxer`. server_config.migration(false); let mut client_config = quinn_proto::ClientConfig::new(client_tls_config); @@ -271,7 +271,7 @@ pub enum ToEndpoint { }, /// Instruct the endpoint to send a packet of data on its UDP socket. SendUdpPacket(quinn_proto::Transmit), - /// The [`GenTransport`][transport::GenTransport] dialer or listener coupled to this endpoint + /// The [`GenTransport`][crate::GenTransport] dialer or listener coupled to this endpoint /// was dropped. /// Once all pending connections are closed, the [`EndpointDriver`] should shut down. Decoupled, diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index e9c2b87b132..0b67a818f00 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -40,7 +40,7 @@ pub struct Muxer { inner: Arc>, } -/// Mutex-protected fields of [`QuicMuxer`]. +/// Mutex-protected fields of [`Muxer`]. #[derive(Debug)] struct Inner { /// Inner connection object that yields events. @@ -64,7 +64,7 @@ impl Inner { } impl Muxer { - /// Crate-internal function that builds a [`QuicMuxer`] from a raw connection. + /// Crate-internal function that builds a [`Muxer`] from a raw connection. pub(crate) fn from_connection(connection: Connection) -> Self { Muxer { inner: Arc::new(Mutex::new(Inner { From c33d17f370923f0675eee2ff9434eb28f7d35f15 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 14 Oct 2022 13:08:06 +0200 Subject: [PATCH 165/218] transports/quic: test dialer drop --- transports/quic/src/transport.rs | 68 +++++++++++++++++++++++++------- 1 file changed, 53 insertions(+), 15 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 3c32d80079c..46ce561cf3e 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -636,25 +636,12 @@ mod test { ); } - #[cfg(feature = "tokio")] - #[tokio::test] - async fn tokio_close_listener() { - let keypair = libp2p_core::identity::Keypair::generate_ed25519(); - let config = Config::new(&keypair); - let transport = crate::tokio::Transport::new(config.clone()); - test_close_listener(transport).await - } - #[cfg(feature = "async-std")] #[async_std::test] - async fn async_std_close_listener() { + async fn test_close_listener() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); let config = Config::new(&keypair); - let transport = crate::async_std::Transport::new(config.clone()); - test_close_listener(transport).await - } - - async fn test_close_listener(mut transport: GenTransport

) { + let mut transport = crate::async_std::Transport::new(config); assert!(poll_fn(|cx| Pin::new(&mut transport).as_mut().poll(cx)) .now_or_never() .is_none()); @@ -717,4 +704,55 @@ mod test { .await; } } + + #[cfg(feature = "tokio")] + #[tokio::test] + async fn test_dialer_drop() { + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); + let config = Config::new(&keypair); + let mut transport = crate::async_std::Transport::new(config); + + let _dial = transport + .dial("/ip4/123.45.67.8/udp/1234/quic".parse().unwrap()) + .unwrap(); + + // Expect a dialer and its background task to exist. + let mut channel = transport + .dialer + .get(&SocketFamily::Ipv4) + .unwrap() + .endpoint_channel + .clone(); + assert!(!transport.dialer.contains_key(&SocketFamily::Ipv6)); + + // Send dummy dial to check that the endpoint driver is running. + poll_fn(|cx| { + let (tx, _) = oneshot::channel(); + let _ = channel + .try_send( + ToEndpoint::Dial { + addr: SocketAddr::new(IpAddr::V4(Ipv4Addr::UNSPECIFIED), 0), + result: tx, + }, + cx, + ) + .unwrap(); + Poll::Ready(()) + }) + .await; + + // Start listening so that the dialer and driver are dropped. + let _ = transport + .listen_on("/ip4/0.0.0.0/udp/0/quic".parse().unwrap()) + .unwrap(); + assert!(!transport.dialer.contains_key(&SocketFamily::Ipv4)); + + // Check that the [`EndpointDriver`] has shut down. + Delay::new(Duration::from_millis(10)).await; + poll_fn(|cx| { + assert!(channel.try_send(ToEndpoint::Decoupled, cx).is_err()); + Poll::Ready(()) + }) + .await; + } } From 1f83766059fec549906a180c0391f08db2272de6 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 14 Oct 2022 13:30:20 +0200 Subject: [PATCH 166/218] transports/quic: better debugging statements --- transports/quic/src/endpoint.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 3fc6c0e46e6..b2e7b4e6c76 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -505,6 +505,10 @@ impl EndpointDriver

{ } Err(err) if err.is_full() => { // Connection is too busy. Drop the datagram to back-pressure the remote. + log::debug!( + "Dropping {:?} because the connection's channel is full.", + err.into_inner() + ); } Err(_) => unreachable!("Error is either `Full` or `Disconnected`."), } @@ -519,9 +523,7 @@ impl EndpointDriver

{ let connection_tx = match self.new_connection_tx.as_mut() { Some(tx) => tx, None => { - log::warn!( - "Endpoint reported a new connection even though server capabilities are disabled." - ); + debug_assert!(false, "Endpoint reported a new connection even though server capabilities are disabled."); return ControlFlow::Continue(()); } }; From 6af111627576954b289805bf40657940305e442c Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 14 Oct 2022 13:55:44 +0200 Subject: [PATCH 167/218] transports/quic: improve logic for picking a dialer --- transports/quic/src/transport.rs | 42 +++++++++++++++++--------------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 46ce561cf3e..0f0d9d51813 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -142,28 +142,30 @@ impl Transport for GenTransport

{ }) .collect::>(); - let dialing = if !listeners.is_empty() { - // Pick any listener to use for dialing. - // We hash the socket address to achieve determinism. - let mut hasher = DefaultHasher::new(); - socket_addr.hash(&mut hasher); - let index = hasher.finish() as usize % listeners.len(); - listeners[index] - .dialer_state - .new_dial(socket_addr, self.handshake_timeout) - } else { - // No listener? Get or create an explicit dialer. - let socket_family = socket_addr.ip().into(); - let dialer = match self.dialer.entry(socket_family) { - Entry::Occupied(occupied) => occupied.into_mut(), - Entry::Vacant(vacant) => { - vacant.insert(Dialer::new::

(self.quinn_config.clone(), socket_family)?) - } - }; - dialer.state.new_dial(socket_addr, self.handshake_timeout) + let dialer_state = match listeners.len() { + 0 => { + // No listener. Get or create an explicit dialer. + let socket_family = socket_addr.ip().into(); + let dialer = match self.dialer.entry(socket_family) { + Entry::Occupied(occupied) => occupied.into_mut(), + Entry::Vacant(vacant) => { + vacant.insert(Dialer::new::

(self.quinn_config.clone(), socket_family)?) + } + }; + &mut dialer.state + } + 1 => &mut listeners[0].dialer_state, + _ => { + // Pick any listener to use for dialing. + // We hash the socket address to achieve determinism. + let mut hasher = DefaultHasher::new(); + socket_addr.hash(&mut hasher); + let index = hasher.finish() as usize % listeners.len(); + &mut listeners[index].dialer_state + } }; - Ok(dialing) + Ok(dialer_state.new_dial(socket_addr, self.handshake_timeout)) } fn dial_as_listener( From 3b0bd06c9d3355cecd9078a359428fad1d216177 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 14 Oct 2022 14:17:00 +0200 Subject: [PATCH 168/218] transports/quic: test dialing wrong PeerId` --- transports/quic/tests/smoke.rs | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 01fae2289c1..9f876c25a91 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -568,3 +568,35 @@ async fn ipv4_dial_ipv6() { } } } + +#[cfg(feature = "async-std")] +#[async_std::test] +async fn wrong_peerod() { + use libp2p::PeerId; + + let _ = env_logger::try_init(); + let mut swarm_a = create_swarm::().await; + let mut swarm_b = create_swarm::().await; + + let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; + let a_id = *swarm_a.local_peer_id(); + + let wrong_id = PeerId::random(); + let dial_ops = DialOpts::peer_id(wrong_id).addresses(vec![a_addr]).build(); + swarm_b.dial(dial_ops).unwrap(); + + loop { + select! { + _ = swarm_a.select_next_some() => {}, + ev = swarm_b.select_next_some() => match ev { + SwarmEvent::Dialing(peer_id) => assert_eq!(peer_id, wrong_id), + SwarmEvent::OutgoingConnectionError {peer_id: Some(peer_id), error: DialError::WrongPeerId { obtained, .. }} => { + assert_eq!(peer_id, wrong_id); + assert_eq!(obtained, a_id); + break; + }, + e => panic!("{:?}", e), + } + } + } +} From a41205703a9e082d264eec2fb5662a9bbbe23386 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 11:54:19 +0200 Subject: [PATCH 169/218] transports/quic: remove deny(unsafe-code) flag --- transports/quic/src/lib.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index ac39293e090..b554ecf971c 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -55,8 +55,6 @@ //! Instead, you must pass all needed configuration into the constructor. //! -#![deny(unsafe_code)] - mod connection; mod endpoint; mod error; From 4325f5cc053b5a9ec394d1a5838c0db0fda97a7a Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 12:22:55 +0200 Subject: [PATCH 170/218] transports/quic: follow libp2p#2962 --- Cargo.toml | 10 ++++------ src/lib.rs | 13 ++++++------- 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 2c2c83b986b..ba3d653b546 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -33,8 +33,7 @@ full = [ "ping", "plaintext", "pnet", - "quic-async-std", - "quic-tokio", + "quic", "relay", "rendezvous", "request-response", @@ -71,8 +70,7 @@ noise = ["dep:libp2p-noise"] ping = ["dep:libp2p-ping", "libp2p-metrics?/ping"] plaintext = ["dep:libp2p-plaintext"] pnet = ["dep:libp2p-pnet"] -quic-async-std = ["dep:libp2p-quic", "libp2p-quic?/async-std"] -quic-tokio = ["dep:libp2p-quic", "libp2p-quic?/tokio"] +quic = ["dep:libp2p-quic"] relay = ["dep:libp2p-relay", "libp2p-metrics?/relay"] request-response = ["dep:libp2p-request-response"] rendezvous = ["dep:libp2p-rendezvous"] @@ -89,8 +87,8 @@ secp256k1 = ["libp2p-core/secp256k1"] rsa = ["libp2p-core/rsa"] ecdsa = ["libp2p-core/ecdsa"] serde = ["libp2p-core/serde", "libp2p-kad?/serde", "libp2p-gossipsub?/serde"] -tokio = ["libp2p-mdns?/tokio", "libp2p-tcp?/tokio", "libp2p-dns?/tokio"] -async-std = ["libp2p-mdns?/async-io", "libp2p-tcp?/async-io", "libp2p-dns?/async-std"] +tokio = ["libp2p-mdns?/tokio", "libp2p-tcp?/tokio", "libp2p-dns?/tokio", "libp2p-quic?/tokio"] +async-std = ["libp2p-mdns?/async-io", "libp2p-tcp?/async-io", "libp2p-dns?/async-std", "libp2p-quic?/async-std"] [package.metadata.docs.rs] all-features = true diff --git a/src/lib.rs b/src/lib.rs index 36193112378..5a819dbc555 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -145,14 +145,13 @@ pub use libp2p_plaintext as plaintext; #[cfg_attr(docsrs, doc(cfg(feature = "pnet")))] #[doc(inline)] pub use libp2p_pnet as pnet; -#[cfg(any(feature = "quic-async-std", feature = "quic-tokio"))] -#[cfg_attr( - docsrs, - doc(cfg(any(feature = "quic-async-std", feature = "quic-tokio"))) -)] +#[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; +pub mod quic { + #[doc(inline)] + pub use libp2p_quic::*; +} #[cfg(feature = "relay")] #[cfg_attr(docsrs, doc(cfg(feature = "relay")))] #[doc(inline)] From 3982a6cd2feec3b23bda0a0d1d3ef982fe4688d5 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 12:24:35 +0200 Subject: [PATCH 171/218] transports/quic: follow libp2p#2983 --- transports/quic/Cargo.toml | 7 +++++++ transports/quic/src/lib.rs | 2 ++ 2 files changed, 9 insertions(+) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 917c65f0158..f4d4624b460 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -32,6 +32,13 @@ yasna = "0.5.0" tokio = ["dep:tokio-crate"] async-std = ["dep:async-std-crate"] +# Passing arguments to the docsrs builder in order to properly document cfg's. +# More information: https://docs.rs/about/builds#cross-compiling +[package.metadata.docs.rs] +all-features = true +rustdoc-args = ["--cfg", "docsrs"] +rustc-args = ["--cfg", "docsrs"] + [dev-dependencies] async-std-crate = { package = "async-std", version = "1.12.0", features = ["attributes"] } libp2p = { path = "../..", features = ["request-response"] } diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index b554ecf971c..9508274276e 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -55,6 +55,8 @@ //! Instead, you must pass all needed configuration into the constructor. //! +#![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))] + mod connection; mod endpoint; mod error; From 8bcf9df02e3c9a136d9d01ee6cb089d8c006d528 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 13:15:18 +0200 Subject: [PATCH 172/218] transports/quic: move struct within file --- transports/quic/src/muxer.rs | 46 ++++++++++++++++++------------------ 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 0b67a818f00..f55f378e632 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -40,29 +40,6 @@ pub struct Muxer { inner: Arc>, } -/// Mutex-protected fields of [`Muxer`]. -#[derive(Debug)] -struct Inner { - /// Inner connection object that yields events. - connection: Connection, - /// State of all the substreams that the muxer reports as open. - substreams: HashMap, - /// Waker to wake if a new outbound substream is opened. - poll_outbound_waker: Option, - /// Waker to wake if a new inbound substream was happened. - poll_inbound_waker: Option, - /// Waker to wake if the connection should be polled again. - poll_connection_waker: Option, -} - -impl Inner { - fn unchecked_substream_state(&mut self, id: quinn_proto::StreamId) -> &mut SubstreamState { - self.substreams - .get_mut(&id) - .expect("Substream should be known.") - } -} - impl Muxer { /// Crate-internal function that builds a [`Muxer`] from a raw connection. pub(crate) fn from_connection(connection: Connection) -> Self { @@ -214,6 +191,29 @@ impl StreamMuxer for Muxer { } } +/// Mutex-protected fields of [`Muxer`]. +#[derive(Debug)] +struct Inner { + /// Inner connection object that yields events. + connection: Connection, + /// State of all the substreams that the muxer reports as open. + substreams: HashMap, + /// Waker to wake if a new outbound substream is opened. + poll_outbound_waker: Option, + /// Waker to wake if a new inbound substream was happened. + poll_inbound_waker: Option, + /// Waker to wake if the connection should be polled again. + poll_connection_waker: Option, +} + +impl Inner { + fn unchecked_substream_state(&mut self, id: quinn_proto::StreamId) -> &mut SubstreamState { + self.substreams + .get_mut(&id) + .expect("Substream should be known.") + } +} + /// State of a single substream. #[derive(Debug, Default, Clone)] struct SubstreamState { From 5da85e7df0c439b8fedd82a8b817079e742902d3 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 13:46:15 +0200 Subject: [PATCH 173/218] transports/quic: handle `Inner` in `Connecting` --- transports/quic/src/muxer.rs | 24 +++++++++--------------- transports/quic/src/transport.rs | 19 +++++++++++++++++-- transports/quic/src/upgrade.rs | 19 ++++++++----------- 3 files changed, 34 insertions(+), 28 deletions(-) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index f55f378e632..de4d8f3db71 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -42,15 +42,9 @@ pub struct Muxer { impl Muxer { /// Crate-internal function that builds a [`Muxer`] from a raw connection. - pub(crate) fn from_connection(connection: Connection) -> Self { + pub(crate) fn new(inner: Inner) -> Self { Muxer { - inner: Arc::new(Mutex::new(Inner { - connection, - substreams: Default::default(), - poll_outbound_waker: None, - poll_inbound_waker: None, - poll_connection_waker: None, - })), + inner: Arc::new(Mutex::new(inner)), } } } @@ -193,17 +187,17 @@ impl StreamMuxer for Muxer { /// Mutex-protected fields of [`Muxer`]. #[derive(Debug)] -struct Inner { +pub struct Inner { /// Inner connection object that yields events. - connection: Connection, + pub connection: Connection, /// State of all the substreams that the muxer reports as open. - substreams: HashMap, + pub substreams: HashMap, /// Waker to wake if a new outbound substream is opened. - poll_outbound_waker: Option, + pub poll_outbound_waker: Option, /// Waker to wake if a new inbound substream was happened. - poll_inbound_waker: Option, + pub poll_inbound_waker: Option, /// Waker to wake if the connection should be polled again. - poll_connection_waker: Option, + pub poll_connection_waker: Option, } impl Inner { @@ -216,7 +210,7 @@ impl Inner { /// State of a single substream. #[derive(Debug, Default, Clone)] -struct SubstreamState { +pub struct SubstreamState { /// Waker to wake if the substream becomes readable or stopped. read_waker: Option, /// Waker to wake if the substream becomes writable or stopped. diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 0f0d9d51813..4e6244522d6 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -24,6 +24,7 @@ use crate::connection::Connection; use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; +use crate::muxer::Inner; use crate::provider::Provider; use crate::{endpoint, muxer::Muxer, upgrade::Connecting, Error}; @@ -266,7 +267,14 @@ impl DialerState { async move { // Our oneshot getting dropped means the message didn't make it to the endpoint driver. let connection = tx.await.map_err(|_| Error::EndpointDriverCrashed)??; - let (peer, muxer) = Connecting::from_connection(connection, timeout).await?; + let inner = Inner { + connection, + substreams: Default::default(), + poll_outbound_waker: None, + poll_inbound_waker: None, + poll_connection_waker: None, + }; + let (peer, muxer) = Connecting::new(inner, timeout).await?; Ok((peer, muxer)) } @@ -442,8 +450,15 @@ impl Stream for Listener { Poll::Ready(Some(connection)) => { let local_addr = socketaddr_to_multiaddr(connection.local_addr()); let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); + let inner = Inner { + connection, + substreams: Default::default(), + poll_outbound_waker: None, + poll_inbound_waker: None, + poll_connection_waker: None, + }; let event = TransportEvent::Incoming { - upgrade: Connecting::from_connection(connection, self.handshake_timeout), + upgrade: Connecting::new(inner, self.handshake_timeout), local_addr, send_back_addr, listener_id: self.listener_id, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 68fa4d1f91f..05ae0ba7c3d 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -20,10 +20,7 @@ //! Future that drives a QUIC connection until is has performed its TLS handshake. -use crate::{ - connection::{Connection, ConnectionEvent}, - Error, Muxer, -}; +use crate::{connection::ConnectionEvent, muxer::Inner, Error, Muxer}; use futures::prelude::*; use futures_timer::Delay; @@ -37,15 +34,15 @@ use std::{ /// A QUIC connection currently being negotiated. #[derive(Debug)] pub struct Connecting { - connection: Option, + inner: Option, timeout: Delay, } impl Connecting { /// Builds an [`Connecting`] that wraps around a [`Connection`]. - pub(crate) fn from_connection(connection: Connection, timeout: Duration) -> Self { + pub(crate) fn new(inner: Inner, timeout: Duration) -> Self { Connecting { - connection: Some(connection), + inner: Some(inner), timeout: Delay::new(timeout), } } @@ -55,15 +52,15 @@ impl Future for Connecting { type Output = Result<(PeerId, Muxer), Error>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let connection = self - .connection + let inner = self + .inner .as_mut() .expect("Future polled after it has completed"); loop { - match connection.poll_event(cx) { + match inner.connection.poll_event(cx) { Poll::Ready(ConnectionEvent::Connected(peer_id)) => { - let muxer = Muxer::from_connection(self.connection.take().unwrap()); + let muxer = Muxer::new(self.inner.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } Poll::Ready(ConnectionEvent::ConnectionLost(err)) => return Poll::Ready(Err(err)), From 6307b8b0f24dd976dc6bfea393e110c84a2b352b Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 14:12:07 +0200 Subject: [PATCH 174/218] quic: return `quinn_proto::Event` in `Connection::poll_event` --- transports/quic/src/connection.rs | 105 +++--------------------------- transports/quic/src/muxer.rs | 63 +++++++++++------- transports/quic/src/upgrade.rs | 51 ++++++++++----- 3 files changed, 84 insertions(+), 135 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 4e9baf8bf92..181e2902dbe 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -26,13 +26,9 @@ //! All interactions with a QUIC connection should be done through this struct. // TODO: docs -use crate::{ - endpoint::{self, ToEndpoint}, - ConnectionError, Error, -}; +use crate::endpoint::{self, ToEndpoint}; use futures::{channel::mpsc, prelude::*}; use futures_timer::Delay; -use libp2p_core::PeerId; use std::{ net::SocketAddr, task::{Context, Poll}, @@ -172,8 +168,12 @@ impl Connection { self.connection.send_stream(id).finish() } + pub fn crypto_session(&self) -> &dyn quinn_proto::crypto::Session { + self.connection.crypto_session() + } + /// Polls the connection for an event that happened on it. - pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll { + pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { loop { match self.from_endpoint.poll_next_unpin(cx) { Poll::Ready(Some(event)) => { @@ -181,9 +181,7 @@ impl Connection { continue; } Poll::Ready(None) => { - return Poll::Ready(ConnectionEvent::ConnectionLost( - Error::EndpointDriverCrashed, - )); + return Poll::Ready(None); } Poll::Pending => {} } @@ -204,9 +202,7 @@ impl Connection { return Poll::Pending; } Err(endpoint::Disconnected {}) => { - return Poll::Ready(ConnectionEvent::ConnectionLost( - Error::EndpointDriverCrashed, - )); + return Poll::Ready(None); } } } @@ -258,65 +254,12 @@ impl Connection { // The final step consists in handling the events related to the various substreams. if let Some(ev) = self.connection.poll() { - let event = self.parse_connection_event(ev); - return Poll::Ready(event); + return Poll::Ready(Some(ev)); } return Poll::Pending; } } - - fn parse_connection_event(&self, event: quinn_proto::Event) -> ConnectionEvent { - match event { - quinn_proto::Event::Connected => { - let session = self.connection.crypto_session(); - let identity = session - .peer_identity() - .expect("connection got identity because it passed TLS handshake; qed"); - let certificates: Box> = - identity.downcast().expect("we rely on rustls feature; qed"); - let end_entity = certificates - .get(0) - .expect("there should be exactly one certificate; qed"); - let end_entity_der = end_entity.as_ref(); - let p2p_cert = crate::tls::certificate::parse_certificate(end_entity_der) - .expect("the certificate was validated during TLS handshake; qed"); - let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); - ConnectionEvent::Connected(peer_id) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { - ConnectionEvent::StreamReadable(id) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Writable { id }) => { - ConnectionEvent::StreamWritable(id) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { id, .. }) => { - ConnectionEvent::StreamStopped(id) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { - dir: quinn_proto::Dir::Bi, - }) => ConnectionEvent::StreamAvailable, - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { - dir: quinn_proto::Dir::Bi, - }) => ConnectionEvent::StreamOpened, - quinn_proto::Event::ConnectionLost { reason } => { - ConnectionEvent::ConnectionLost(ConnectionError::from(reason).into()) - } - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { - ConnectionEvent::StreamFinished(id) - } - quinn_proto::Event::HandshakeDataReady => ConnectionEvent::HandshakeDataReady, - quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { - dir: quinn_proto::Dir::Uni, - }) - | quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { - dir: quinn_proto::Dir::Uni, - }) - | quinn_proto::Event::DatagramReceived => { - unreachable!("We don't use datagrams or unidirectional streams.") - } - } - } } impl Drop for Connection { @@ -328,33 +271,3 @@ impl Drop for Connection { self.endpoint_channel.send_on_drop(to_endpoint); } } - -/// Event generated by the [`Connection`]. -#[derive(Debug)] -pub enum ConnectionEvent { - /// Now connected to the remote and certificates are available. - Connected(PeerId), - - /// Connection has been closed and can no longer be used. - ConnectionLost(Error), - - /// Generated after [`Connection::accept_substream`] has been called and has returned - /// `None`. After this event has been generated, this method is guaranteed to return `Some`. - StreamAvailable, - /// Generated after [`Connection::open_substream`] has been called and has returned - /// `None`. After this event has been generated, this method is guaranteed to return `Some`. - StreamOpened, - - /// Generated after `read_substream` has returned a `Blocked` error. - StreamReadable(quinn_proto::StreamId), - /// Generated after `write_substream` has returned a `Blocked` error. - StreamWritable(quinn_proto::StreamId), - - /// Generated after [`Connection::finish_substream`] has been called. - StreamFinished(quinn_proto::StreamId), - /// A substream has been stopped. This concept is similar to the concept of a substream being - /// "reset", as in a TCP socket being reset for example. - StreamStopped(quinn_proto::StreamId), - - HandshakeDataReady, -} diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index de4d8f3db71..21d1bd357b9 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -18,10 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::{ - connection::{Connection, ConnectionEvent}, - Error, -}; +use crate::{connection::Connection, Error}; use futures::{ready, AsyncRead, AsyncWrite}; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; @@ -61,53 +58,73 @@ impl StreamMuxer for Muxer { // Poll the inner [`quinn_proto::Connection`] for events and wake // the wakers of related poll-based methods. while let Poll::Ready(event) = inner.connection.poll_event(cx) { + let event = match event { + Some(event) => event, + None => return Poll::Ready(Err(Error::EndpointDriverCrashed)), + }; match event { - ConnectionEvent::Connected(_) | ConnectionEvent::HandshakeDataReady => { + quinn_proto::Event::Connected | quinn_proto::Event::HandshakeDataReady => { debug_assert!( false, "Unexpected event {:?} on established QUIC connection", event ); } - ConnectionEvent::ConnectionLost(err) => { + quinn_proto::Event::ConnectionLost { reason } => { inner.connection.close(); inner.substreams.values_mut().for_each(|s| s.wake_all()); - return Poll::Ready(Err(err)); + return Poll::Ready(Err(Error::Connection(reason.into()))); } - ConnectionEvent::StreamOpened => { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Bi, + }) => { if let Some(waker) = inner.poll_outbound_waker.take() { waker.wake(); } } - ConnectionEvent::StreamReadable(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Bi, + }) => { + if let Some(waker) = inner.poll_inbound_waker.take() { + waker.wake(); + } + } + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Readable { id }) => { + if let Some(substream) = inner.substreams.get_mut(&id) { if let Some(waker) = substream.read_waker.take() { waker.wake(); } } } - ConnectionEvent::StreamWritable(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Writable { id }) => { + if let Some(substream) = inner.substreams.get_mut(&id) { if let Some(waker) = substream.write_waker.take() { waker.wake(); } } } - ConnectionEvent::StreamFinished(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { + if let Some(substream) = inner.substreams.get_mut(&id) { substream.wake_all(); substream.is_write_closed = true; } } - ConnectionEvent::StreamStopped(substream) => { - if let Some(substream) = inner.substreams.get_mut(&substream) { + quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { + id, + error_code: _, + }) => { + if let Some(substream) = inner.substreams.get_mut(&id) { substream.wake_all(); } } - ConnectionEvent::StreamAvailable => { - if let Some(waker) = inner.poll_inbound_waker.take() { - waker.wake(); - } + quinn_proto::Event::DatagramReceived + | quinn_proto::Event::Stream(quinn_proto::StreamEvent::Available { + dir: quinn_proto::Dir::Uni, + }) + | quinn_proto::Event::Stream(quinn_proto::StreamEvent::Opened { + dir: quinn_proto::Dir::Uni, + }) => { + unreachable!("We don't use datagrams or unidirectional streams.") } } } @@ -178,8 +195,10 @@ impl StreamMuxer for Muxer { if connection.send_stream_count() == 0 && !connection.is_closed() { connection.close() } - if let ConnectionEvent::ConnectionLost(_) = ready!(connection.poll_event(cx)) { - return Poll::Ready(Ok(())); + match ready!(connection.poll_event(cx)) { + Some(quinn_proto::Event::ConnectionLost { .. }) => return Poll::Ready(Ok(())), + None => return Poll::Ready(Err(Error::EndpointDriverCrashed)), + _ => {} } } } diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 05ae0ba7c3d..dc7d9222676 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -20,7 +20,7 @@ //! Future that drives a QUIC connection until is has performed its TLS handshake. -use crate::{connection::ConnectionEvent, muxer::Inner, Error, Muxer}; +use crate::{muxer::Inner, Error, Muxer}; use futures::prelude::*; use futures_timer::Delay; @@ -58,26 +58,43 @@ impl Future for Connecting { .expect("Future polled after it has completed"); loop { - match inner.connection.poll_event(cx) { - Poll::Ready(ConnectionEvent::Connected(peer_id)) => { + let event = match inner.connection.poll_event(cx) { + Poll::Ready(Some(event)) => event, + Poll::Ready(None) => return Poll::Ready(Err(Error::EndpointDriverCrashed)), + Poll::Pending => { + return self + .timeout + .poll_unpin(cx) + .map(|()| Err(Error::HandshakeTimedOut)); + } + }; + match event { + quinn_proto::Event::Connected => { + let session = inner.connection.crypto_session(); + let identity = session + .peer_identity() + .expect("connection got identity because it passed TLS handshake; qed"); + let certificates: Box> = + identity.downcast().expect("we rely on rustls feature; qed"); + let end_entity = certificates + .get(0) + .expect("there should be exactly one certificate; qed"); + let end_entity_der = end_entity.as_ref(); + let p2p_cert = crate::tls::certificate::parse_certificate(end_entity_der) + .expect("the certificate was validated during TLS handshake; qed"); + let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); + let muxer = Muxer::new(self.inner.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } - Poll::Ready(ConnectionEvent::ConnectionLost(err)) => return Poll::Ready(Err(err)), - Poll::Ready(ConnectionEvent::HandshakeDataReady) - | Poll::Ready(ConnectionEvent::StreamAvailable) - | Poll::Ready(ConnectionEvent::StreamOpened) - | Poll::Ready(ConnectionEvent::StreamReadable(_)) - | Poll::Ready(ConnectionEvent::StreamWritable(_)) - | Poll::Ready(ConnectionEvent::StreamFinished(_)) - | Poll::Ready(ConnectionEvent::StreamStopped(_)) => continue, - Poll::Pending => {} - } - match self.timeout.poll_unpin(cx) { - Poll::Ready(()) => return Poll::Ready(Err(Error::HandshakeTimedOut)), - Poll::Pending => {} + quinn_proto::Event::ConnectionLost { reason } => { + return Poll::Ready(Err(Error::Connection(reason.into()))) + } + quinn_proto::Event::HandshakeDataReady | quinn_proto::Event::Stream(_) => {} + quinn_proto::Event::DatagramReceived => { + debug_assert!(false, "Datagrams are not supported") + } } - return Poll::Pending; } } } From 0271539ae00e465203fd73bf2fc1d07d7d148874 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 15:00:13 +0200 Subject: [PATCH 175/218] quic: inline `Connection` into `muxer::Inner` --- transports/quic/src/connection.rs | 273 ------------------------------ transports/quic/src/endpoint.rs | 18 +- transports/quic/src/lib.rs | 1 - transports/quic/src/muxer.rs | 262 +++++++++++++++++++++++++--- transports/quic/src/transport.rs | 21 +-- transports/quic/src/upgrade.rs | 16 +- 6 files changed, 261 insertions(+), 330 deletions(-) delete mode 100644 transports/quic/src/connection.rs diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs deleted file mode 100644 index 181e2902dbe..00000000000 --- a/transports/quic/src/connection.rs +++ /dev/null @@ -1,273 +0,0 @@ -// 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. - -//! A single QUIC connection. -//! -//! The [`Connection`] struct of this module contains, amongst other things, a -//! [`quinn_proto::Connection`] state machine and an `Arc`. This struct is responsible -//! for communication between quinn_proto's connection and its associated endpoint. -//! All interactions with a QUIC connection should be done through this struct. -// TODO: docs - -use crate::endpoint::{self, ToEndpoint}; -use futures::{channel::mpsc, prelude::*}; -use futures_timer::Delay; -use std::{ - net::SocketAddr, - task::{Context, Poll}, - time::Instant, -}; - -/// Underlying structure for both [`crate::Muxer`] and [`crate::Connecting`]. -/// -/// Contains everything needed to process a connection with a remote. -/// Tied to a specific endpoint. -#[derive(Debug)] -pub struct Connection { - /// Channel to the endpoint this connection belongs to. - endpoint_channel: endpoint::Channel, - /// Pending message to be sent to the background task that is driving the endpoint. - pending_to_endpoint: Option, - /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. - /// Passed at initialization. - from_endpoint: mpsc::Receiver, - - /// The QUIC state machine for this specific connection. - connection: quinn_proto::Connection, - /// Identifier for this connection according to the endpoint. Used when sending messages to - /// the endpoint. - connection_id: quinn_proto::ConnectionHandle, - /// `Future` that triggers at the [`Instant`] that `self.connection.poll_timeout()` indicates. - next_timeout: Option<(Delay, Instant)>, -} - -impl Connection { - /// Crate-internal function that builds a [`Connection`] from raw components. - /// - /// This function assumes that there exists a [`EndpointDriver`](super::endpoint::EndpointDriver) - /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages - /// on `from_endpoint`. - /// - /// `connection_id` is used to identify the local connection in the messages sent to - /// `to_endpoint`. - /// - /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of - /// its methods has ever been called. Failure to comply might lead to logic errors and panics. - pub fn from_quinn_connection( - endpoint_channel: endpoint::Channel, - connection: quinn_proto::Connection, - connection_id: quinn_proto::ConnectionHandle, - from_endpoint: mpsc::Receiver, - ) -> Self { - debug_assert!(!connection.is_closed()); - Connection { - endpoint_channel, - pending_to_endpoint: None, - connection, - next_timeout: None, - from_endpoint, - connection_id, - } - } - - /// The address that the local socket is bound to. - pub fn local_addr(&self) -> &SocketAddr { - self.endpoint_channel.socket_addr() - } - - /// Returns the address of the node we're connected to. - pub fn remote_addr(&self) -> SocketAddr { - self.connection.remote_address() - } - - /// Start closing the connection. A [`ConnectionEvent::ConnectionLost`] event will be - /// produced in the future. - pub fn close(&mut self) { - // We send a dummy `0` error code with no message, as the API of StreamMuxer doesn't - // support this. - self.connection - .close(Instant::now(), From::from(0u32), Default::default()); - } - - /// Whether the connection is closed. - /// A [`ConnectionEvent::ConnectionLost`] event is emitted with details when the - /// connection becomes closed. - pub fn is_closed(&self) -> bool { - self.connection.is_closed() - } - - /// Whether there is no longer any need to keep the connection around. - /// All drained connections have been closed. - pub fn is_drained(&self) -> bool { - self.connection.is_drained() - } - - /// Pops a new substream opened by the remote. - /// - /// If `None` is returned, then a [`ConnectionEvent::StreamAvailable`] event will later be - /// produced when a substream is available. - pub fn accept_substream(&mut self) -> Option { - self.connection.streams().accept(quinn_proto::Dir::Bi) - } - - /// Pops a new substream opened locally. - /// - /// The API can be thought as if outgoing substreams were automatically opened by the local - /// QUIC connection and were added to a queue for availability. - /// - /// If `None` is returned, then a [`ConnectionEvent::StreamOpened`] event will later be - /// produced when a substream is available. - pub fn open_substream(&mut self) -> Option { - self.connection.streams().open(quinn_proto::Dir::Bi) - } - - /// Control over the stream for reading. - pub fn recv_stream(&mut self, id: quinn_proto::StreamId) -> quinn_proto::RecvStream<'_> { - self.connection.recv_stream(id) - } - - /// Control over the stream for writing. - pub fn send_stream(&mut self, id: quinn_proto::StreamId) -> quinn_proto::SendStream<'_> { - self.connection.send_stream(id) - } - - /// Number of streams that may have unacknowledged data. - pub fn send_stream_count(&mut self) -> usize { - self.connection.streams().send_streams() - } - - /// Closes the given substream. - /// - /// `write_substream` must no longer be called. The substream is however still - /// readable. - /// - /// On success, a [`quinn_proto::StreamEvent::Finished`] event will later be produced when the - /// substream has been effectively closed. A [`ConnectionEvent::StreamStopped`] event can also - /// be emitted. - pub fn finish_substream( - &mut self, - id: quinn_proto::StreamId, - ) -> Result<(), quinn_proto::FinishError> { - self.connection.send_stream(id).finish() - } - - pub fn crypto_session(&self) -> &dyn quinn_proto::crypto::Session { - self.connection.crypto_session() - } - - /// Polls the connection for an event that happened on it. - pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { - loop { - match self.from_endpoint.poll_next_unpin(cx) { - Poll::Ready(Some(event)) => { - self.connection.handle_event(event); - continue; - } - Poll::Ready(None) => { - return Poll::Ready(None); - } - Poll::Pending => {} - } - - // Sending the pending event to the endpoint. If the endpoint is too busy, we just - // stop the processing here. - // We need to be careful to avoid a potential deadlock if both `from_endpoint` and - // `to_endpoint` are full. As such, we continue to transfer data from `from_endpoint` - // to the `quinn_proto::Connection` (see above). - // However we don't deliver substream-related events to the user as long as - // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` - // being full to the user. - if let Some(to_endpoint) = self.pending_to_endpoint.take() { - match self.endpoint_channel.try_send(to_endpoint, cx) { - Ok(Ok(())) => continue, // The endpoint may send back an event. - Ok(Err(to_endpoint)) => { - self.pending_to_endpoint = Some(to_endpoint); - return Poll::Pending; - } - Err(endpoint::Disconnected {}) => { - return Poll::Ready(None); - } - } - } - - // The maximum amount of segments which can be transmitted in a single Transmit - // if a platform supports Generic Send Offload (GSO). - // Set to 1 for now since not all platforms support GSO. - // TODO: Fix for platforms that support GSO. - let max_datagrams = 1; - // Poll the connection for packets to send on the UDP socket and try to send them on - // `to_endpoint`. - if let Some(transmit) = self.connection.poll_transmit(Instant::now(), max_datagrams) { - // TODO: ECN bits not handled - self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket(transmit)); - continue; - } - - match self.connection.poll_timeout() { - Some(timeout) => match self.next_timeout { - Some((_, when)) if when == timeout => {} - _ => { - let now = Instant::now(); - // 0ns if now > when - let duration = timeout.duration_since(now); - let next_timeout = Delay::new(duration); - self.next_timeout = Some((next_timeout, timeout)) - } - }, - None => self.next_timeout = None, - } - - if let Some((timeout, when)) = self.next_timeout.as_mut() { - if timeout.poll_unpin(cx).is_ready() { - self.connection.handle_timeout(*when); - continue; - } - } - - // The connection also needs to be able to send control messages to the endpoint. This is - // handled here, and we try to send them on `to_endpoint` as well. - if let Some(event) = self.connection.poll_endpoint_events() { - let connection_id = self.connection_id; - self.pending_to_endpoint = Some(ToEndpoint::ProcessConnectionEvent { - connection_id, - event, - }); - continue; - } - - // The final step consists in handling the events related to the various substreams. - if let Some(ev) = self.connection.poll() { - return Poll::Ready(Some(ev)); - } - - return Poll::Pending; - } - } -} - -impl Drop for Connection { - fn drop(&mut self) { - let to_endpoint = ToEndpoint::ProcessConnectionEvent { - connection_id: self.connection_id, - event: quinn_proto::EndpointEvent::drained(), - }; - self.endpoint_channel.send_on_drop(to_endpoint); - } -} diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b2e7b4e6c76..7b5d4d4fd6d 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,9 +28,7 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`EndpointDriver`] for a thorough description. -use crate::{ - connection::Connection, provider::Provider, tls, transport::SocketFamily, ConnectError, Error, -}; +use crate::{muxer::Inner, provider::Provider, tls, transport::SocketFamily, ConnectError, Error}; use bytes::BytesMut; use futures::{ @@ -148,7 +146,7 @@ impl Channel { pub fn new_bidirectional( quinn_config: QuinnConfig, socket_addr: SocketAddr, - ) -> Result<(Self, mpsc::Receiver), Error> { + ) -> Result<(Self, mpsc::Receiver), Error> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Self::new::

(quinn_config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) @@ -169,7 +167,7 @@ impl Channel { fn new( quinn_config: QuinnConfig, socket_addr: SocketAddr, - new_connections: Option>, + new_connections: Option>, ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; @@ -260,7 +258,7 @@ pub enum ToEndpoint { /// UDP address to connect to. addr: SocketAddr, /// Channel to return the result of the dialing to. - result: oneshot::Sender>, + result: 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. Only `quinn_proto` knows what is in @@ -375,7 +373,7 @@ pub struct EndpointDriver { HashMap>, // Channel to forward new inbound connections to the transport. // `None` if server capabilities are disabled, i.e. the endpoint is only used for dialing. - new_connection_tx: Option>, + new_connection_tx: Option>, // Whether the transport dropped its handle for this endpoint. is_decoupled: bool, } @@ -384,7 +382,7 @@ impl EndpointDriver

{ fn new( endpoint_config: Arc, client_config: quinn_proto::ClientConfig, - new_connection_tx: Option>, + new_connection_tx: Option>, server_config: Option>, channel: Channel, socket: P, @@ -421,7 +419,7 @@ impl EndpointDriver

{ debug_assert_eq!(connection.side(), quinn_proto::Side::Client); let (tx, rx) = mpsc::channel(16); - let connection = Connection::from_quinn_connection( + let connection = Inner::from_quinn_connection( self.channel.clone(), connection, connection_id, @@ -530,7 +528,7 @@ impl EndpointDriver

{ let (tx, rx) = mpsc::channel(16); let connection = - Connection::from_quinn_connection(self.channel.clone(), connec, connec_id, rx); + Inner::from_quinn_connection(self.channel.clone(), connec, connec_id, rx); match connection_tx.try_send(connection) { Ok(()) => { self.alive_connections.insert(connec_id, tx); diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 9508274276e..ef623a3a443 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -57,7 +57,6 @@ #![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))] -mod connection; mod endpoint; mod error; mod muxer; diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/muxer.rs index 21d1bd357b9..fbf2fcdd9fd 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/muxer.rs @@ -18,17 +18,23 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::{connection::Connection, Error}; +use crate::{ + endpoint::{self, ToEndpoint}, + Error, +}; -use futures::{ready, AsyncRead, AsyncWrite}; +use futures::{channel::mpsc, ready, AsyncRead, AsyncWrite, FutureExt, StreamExt}; +use futures_timer::Delay; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ collections::HashMap, io::{self, Write}, + net::SocketAddr, pin::Pin, sync::Arc, task::{Context, Poll, Waker}, + time::Instant, }; /// State for a single opened QUIC connection. @@ -57,7 +63,7 @@ impl StreamMuxer for Muxer { let mut inner = self.inner.lock(); // Poll the inner [`quinn_proto::Connection`] for events and wake // the wakers of related poll-based methods. - while let Poll::Ready(event) = inner.connection.poll_event(cx) { + while let Poll::Ready(event) = inner.poll_event(cx) { let event = match event { Some(event) => event, None => return Poll::Ready(Err(Error::EndpointDriverCrashed)), @@ -71,7 +77,7 @@ impl StreamMuxer for Muxer { ); } quinn_proto::Event::ConnectionLost { reason } => { - inner.connection.close(); + inner.close(); inner.substreams.values_mut().for_each(|s| s.wake_all()); return Poll::Ready(Err(Error::Connection(reason.into()))); } @@ -141,7 +147,7 @@ impl StreamMuxer for Muxer { ) -> Poll> { let mut inner = self.inner.lock(); - let substream_id = match inner.connection.accept_substream() { + let substream_id = match inner.accept_substream() { Some(id) => { inner.poll_outbound_waker = None; id @@ -162,7 +168,7 @@ impl StreamMuxer for Muxer { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); - let substream_id = match inner.connection.open_substream() { + let substream_id = match inner.open_substream() { Some(id) => { inner.poll_outbound_waker = None; id @@ -178,24 +184,20 @@ impl StreamMuxer for Muxer { } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let Inner { - substreams, - connection, - .. - } = &mut *self.inner.lock(); - if connection.is_drained() { + let inner = &mut *self.inner.lock(); + if inner.is_drained() { return Poll::Ready(Ok(())); } - for substream in substreams.keys() { - let _ = connection.finish_substream(*substream); + for substream in inner.substreams.keys().cloned().collect::>() { + let _ = inner.finish_substream(substream); } loop { - if connection.send_stream_count() == 0 && !connection.is_closed() { - connection.close() + if inner.send_stream_count() == 0 && !inner.is_closed() { + inner.close() } - match ready!(connection.poll_event(cx)) { + match ready!(inner.poll_event(cx)) { Some(quinn_proto::Event::ConnectionLost { .. }) => return Poll::Ready(Ok(())), None => return Poll::Ready(Err(Error::EndpointDriverCrashed)), _ => {} @@ -207,8 +209,22 @@ impl StreamMuxer for Muxer { /// Mutex-protected fields of [`Muxer`]. #[derive(Debug)] pub struct Inner { - /// Inner connection object that yields events. - pub connection: Connection, + /// Channel to the endpoint this connection belongs to. + endpoint_channel: endpoint::Channel, + /// Pending message to be sent to the background task that is driving the endpoint. + pending_to_endpoint: Option, + /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. + /// Passed at initialization. + from_endpoint: mpsc::Receiver, + + /// The QUIC state machine for this specific connection. + connection: quinn_proto::Connection, + /// Identifier for this connection according to the endpoint. Used when sending messages to + /// the endpoint. + connection_id: quinn_proto::ConnectionHandle, + /// `Future` that triggers at the [`Instant`] that `self.connection.poll_timeout()` indicates. + next_timeout: Option<(Delay, Instant)>, + /// State of all the substreams that the muxer reports as open. pub substreams: HashMap, /// Waker to wake if a new outbound substream is opened. @@ -225,6 +241,212 @@ impl Inner { .get_mut(&id) .expect("Substream should be known.") } + + /// Crate-internal function that builds [`Inner`] from raw components. + /// + /// This function assumes that there exists a [`EndpointDriver`](super::endpoint::EndpointDriver) + /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages + /// on `from_endpoint`. + /// + /// `connection_id` is used to identify the local connection in the messages sent to + /// `to_endpoint`. + /// + /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of + /// its methods has ever been called. Failure to comply might lead to logic errors and panics. + pub fn from_quinn_connection( + endpoint_channel: endpoint::Channel, + connection: quinn_proto::Connection, + connection_id: quinn_proto::ConnectionHandle, + from_endpoint: mpsc::Receiver, + ) -> Self { + debug_assert!(!connection.is_closed()); + Inner { + endpoint_channel, + pending_to_endpoint: None, + connection, + next_timeout: None, + from_endpoint, + connection_id, + substreams: HashMap::new(), + poll_connection_waker: None, + poll_inbound_waker: None, + poll_outbound_waker: None, + } + } + + /// The address that the local socket is bound to. + pub fn local_addr(&self) -> &SocketAddr { + self.endpoint_channel.socket_addr() + } + + /// Returns the address of the node we're connected to. + pub fn remote_addr(&self) -> SocketAddr { + self.connection.remote_address() + } + + /// Start closing the connection. A [`quinn_proto::Event::ConnectionLost`] event will be + /// produced in the future. + pub fn close(&mut self) { + // We send a dummy `0` error code with no message, as the API of StreamMuxer doesn't + // support this. + self.connection + .close(Instant::now(), From::from(0u32), Default::default()); + } + + /// Whether the connection is closed. + /// A [`quinn_proto::Event::ConnectionLost`] event is emitted with details when the + /// connection becomes closed. + pub fn is_closed(&self) -> bool { + self.connection.is_closed() + } + + /// Whether there is no longer any need to keep the connection around. + /// All drained connections have been closed. + pub fn is_drained(&self) -> bool { + self.connection.is_drained() + } + + /// Pops a new substream opened by the remote. + /// + /// If `None` is returned, then a [`quinn_proto::StreamEvent::Available`] event will later be + /// produced when a substream is available. + pub fn accept_substream(&mut self) -> Option { + self.connection.streams().accept(quinn_proto::Dir::Bi) + } + + /// Pops a new substream opened locally. + /// + /// The API can be thought as if outgoing substreams were automatically opened by the local + /// QUIC connection and were added to a queue for availability. + /// + /// If `None` is returned, then a [`quinn_proto::StreamEvent::Opened`] event will later be + /// produced when a substream is available. + pub fn open_substream(&mut self) -> Option { + self.connection.streams().open(quinn_proto::Dir::Bi) + } + + /// Number of streams that may have unacknowledged data. + pub fn send_stream_count(&mut self) -> usize { + self.connection.streams().send_streams() + } + + /// Closes the given substream. + /// + /// `write_substream` must no longer be called. The substream is however still + /// readable. + /// + /// On success, a [`quinn_proto::StreamEvent::Finished`] event will later be produced when the + /// substream has been effectively closed. A [`quinn_proto::StreamEvent::Stopped`] event can also + /// be emitted. + pub fn finish_substream( + &mut self, + id: quinn_proto::StreamId, + ) -> Result<(), quinn_proto::FinishError> { + self.connection.send_stream(id).finish() + } + + pub fn crypto_session(&self) -> &dyn quinn_proto::crypto::Session { + self.connection.crypto_session() + } + + /// Polls the connection for an event that happened on it. + pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { + loop { + match self.from_endpoint.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => { + self.connection.handle_event(event); + continue; + } + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => {} + } + + // Sending the pending event to the endpoint. If the endpoint is too busy, we just + // stop the processing here. + // We need to be careful to avoid a potential deadlock if both `from_endpoint` and + // `to_endpoint` are full. As such, we continue to transfer data from `from_endpoint` + // to the `quinn_proto::Connection` (see above). + // However we don't deliver substream-related events to the user as long as + // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` + // being full to the user. + if let Some(to_endpoint) = self.pending_to_endpoint.take() { + match self.endpoint_channel.try_send(to_endpoint, cx) { + Ok(Ok(())) => continue, // The endpoint may send back an event. + Ok(Err(to_endpoint)) => { + self.pending_to_endpoint = Some(to_endpoint); + return Poll::Pending; + } + Err(endpoint::Disconnected {}) => { + return Poll::Ready(None); + } + } + } + + // The maximum amount of segments which can be transmitted in a single Transmit + // if a platform supports Generic Send Offload (GSO). + // Set to 1 for now since not all platforms support GSO. + // TODO: Fix for platforms that support GSO. + let max_datagrams = 1; + // Poll the connection for packets to send on the UDP socket and try to send them on + // `to_endpoint`. + if let Some(transmit) = self.connection.poll_transmit(Instant::now(), max_datagrams) { + // TODO: ECN bits not handled + self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket(transmit)); + continue; + } + + match self.connection.poll_timeout() { + Some(timeout) => match self.next_timeout { + Some((_, when)) if when == timeout => {} + _ => { + let now = Instant::now(); + // 0ns if now > when + let duration = timeout.duration_since(now); + let next_timeout = Delay::new(duration); + self.next_timeout = Some((next_timeout, timeout)) + } + }, + None => self.next_timeout = None, + } + + if let Some((timeout, when)) = self.next_timeout.as_mut() { + if timeout.poll_unpin(cx).is_ready() { + self.connection.handle_timeout(*when); + continue; + } + } + + // The connection also needs to be able to send control messages to the endpoint. This is + // handled here, and we try to send them on `to_endpoint` as well. + if let Some(event) = self.connection.poll_endpoint_events() { + let connection_id = self.connection_id; + self.pending_to_endpoint = Some(ToEndpoint::ProcessConnectionEvent { + connection_id, + event, + }); + continue; + } + + // The final step consists in handling the events related to the various substreams. + if let Some(ev) = self.connection.poll() { + return Poll::Ready(Some(ev)); + } + + return Poll::Pending; + } + } +} + +impl Drop for Inner { + fn drop(&mut self) { + let to_endpoint = ToEndpoint::ProcessConnectionEvent { + connection_id: self.connection_id, + event: quinn_proto::EndpointEvent::drained(), + }; + self.endpoint_channel.send_on_drop(to_endpoint); + } } /// State of a single substream. @@ -360,7 +582,7 @@ impl AsyncWrite for Substream { return Poll::Ready(Ok(())); } - match muxer.connection.finish_substream(self.id) { + match muxer.finish_substream(self.id) { Ok(()) => { let substream_state = muxer.unchecked_substream_state(self.id); substream_state.finished_waker = Some(cx.waker().clone()); diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 4e6244522d6..b7fe480d0e8 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -22,7 +22,6 @@ //! //! Combines all the objects in the other modules to implement the trait. -use crate::connection::Connection; use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; use crate::muxer::Inner; use crate::provider::Provider; @@ -267,14 +266,7 @@ impl DialerState { async move { // Our oneshot getting dropped means the message didn't make it to the endpoint driver. let connection = tx.await.map_err(|_| Error::EndpointDriverCrashed)??; - let inner = Inner { - connection, - substreams: Default::default(), - poll_outbound_waker: None, - poll_inbound_waker: None, - poll_connection_waker: None, - }; - let (peer, muxer) = Connecting::new(inner, timeout).await?; + let (peer, muxer) = Connecting::new(connection, timeout).await?; Ok((peer, muxer)) } @@ -307,7 +299,7 @@ struct Listener { listener_id: ListenerId, /// Channel where new connections are being sent. - new_connections_rx: mpsc::Receiver, + new_connections_rx: mpsc::Receiver, handshake_timeout: Duration, if_watcher: Option, @@ -450,15 +442,8 @@ impl Stream for Listener { Poll::Ready(Some(connection)) => { let local_addr = socketaddr_to_multiaddr(connection.local_addr()); let send_back_addr = socketaddr_to_multiaddr(&connection.remote_addr()); - let inner = Inner { - connection, - substreams: Default::default(), - poll_outbound_waker: None, - poll_inbound_waker: None, - poll_connection_waker: None, - }; let event = TransportEvent::Incoming { - upgrade: Connecting::new(inner, self.handshake_timeout), + upgrade: Connecting::new(connection, self.handshake_timeout), local_addr, send_back_addr, listener_id: self.listener_id, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index dc7d9222676..3aeae5e7330 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -34,15 +34,15 @@ use std::{ /// A QUIC connection currently being negotiated. #[derive(Debug)] pub struct Connecting { - inner: Option, + connection: Option, timeout: Delay, } impl Connecting { - /// Builds an [`Connecting`] that wraps around a [`Connection`]. + /// Builds an [`Connecting`] that wraps around an [`Inner`] connection. pub(crate) fn new(inner: Inner, timeout: Duration) -> Self { Connecting { - inner: Some(inner), + connection: Some(inner), timeout: Delay::new(timeout), } } @@ -52,13 +52,13 @@ impl Future for Connecting { type Output = Result<(PeerId, Muxer), Error>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - let inner = self - .inner + let connection = self + .connection .as_mut() .expect("Future polled after it has completed"); loop { - let event = match inner.connection.poll_event(cx) { + let event = match connection.poll_event(cx) { Poll::Ready(Some(event)) => event, Poll::Ready(None) => return Poll::Ready(Err(Error::EndpointDriverCrashed)), Poll::Pending => { @@ -70,7 +70,7 @@ impl Future for Connecting { }; match event { quinn_proto::Event::Connected => { - let session = inner.connection.crypto_session(); + let session = connection.crypto_session(); let identity = session .peer_identity() .expect("connection got identity because it passed TLS handshake; qed"); @@ -84,7 +84,7 @@ impl Future for Connecting { .expect("the certificate was validated during TLS handshake; qed"); let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); - let muxer = Muxer::new(self.inner.take().unwrap()); + let muxer = Muxer::new(self.connection.take().unwrap()); return Poll::Ready(Ok((peer_id, muxer))); } quinn_proto::Event::ConnectionLost { reason } => { From d2e24360b7c982fe0985d30b87b25a673905bace Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 15:12:31 +0200 Subject: [PATCH 176/218] quic: rename Muxer -> Connection --- .../quic/src/{muxer.rs => connection.rs} | 53 ++++++++++--------- transports/quic/src/endpoint.rs | 16 +++--- transports/quic/src/lib.rs | 4 +- transports/quic/src/transport.rs | 12 ++--- transports/quic/src/upgrade.rs | 8 +-- transports/quic/tests/smoke.rs | 4 +- 6 files changed, 51 insertions(+), 46 deletions(-) rename transports/quic/src/{muxer.rs => connection.rs} (93%) diff --git a/transports/quic/src/muxer.rs b/transports/quic/src/connection.rs similarity index 93% rename from transports/quic/src/muxer.rs rename to transports/quic/src/connection.rs index fbf2fcdd9fd..0806b8a3c28 100644 --- a/transports/quic/src/muxer.rs +++ b/transports/quic/src/connection.rs @@ -39,20 +39,20 @@ use std::{ /// State for a single opened QUIC connection. #[derive(Debug)] -pub struct Muxer { +pub struct Connection { inner: Arc>, } -impl Muxer { - /// Crate-internal function that builds a [`Muxer`] from a raw connection. +impl Connection { + /// Crate-internal function that builds a [`Connection`] from a raw connection. pub(crate) fn new(inner: Inner) -> Self { - Muxer { + Connection { inner: Arc::new(Mutex::new(inner)), } } } -impl StreamMuxer for Muxer { +impl StreamMuxer for Connection { type Substream = Substream; type Error = Error; @@ -206,7 +206,7 @@ impl StreamMuxer for Muxer { } } -/// Mutex-protected fields of [`Muxer`]. +/// Mutex-protected fields of [`Connection`]. #[derive(Debug)] pub struct Inner { /// Channel to the endpoint this connection belongs to. @@ -479,12 +479,12 @@ impl SubstreamState { #[derive(Debug)] pub struct Substream { id: quinn_proto::StreamId, - muxer: Arc>, + connection: Arc>, } impl Substream { - fn new(id: quinn_proto::StreamId, muxer: Arc>) -> Self { - Self { id, muxer } + fn new(id: quinn_proto::StreamId, connection: Arc>) -> Self { + Self { id, connection } } } @@ -494,9 +494,9 @@ impl AsyncRead for Substream { cx: &mut Context<'_>, mut buf: &mut [u8], ) -> Poll> { - let mut muxer = self.muxer.lock(); + let mut connection = self.connection.lock(); - let mut stream = muxer.connection.recv_stream(self.id); + let mut stream = connection.connection.recv_stream(self.id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(quinn_proto::ReadableError::UnknownStream) => { @@ -527,12 +527,12 @@ impl AsyncRead for Substream { bytes += chunk.bytes.len(); } if chunks.finalize().should_transmit() { - if let Some(waker) = muxer.poll_connection_waker.take() { + if let Some(waker) = connection.poll_connection_waker.take() { waker.wake(); } } if pending && bytes == 0 { - let substream_state = muxer.unchecked_substream_state(self.id); + let substream_state = connection.unchecked_substream_state(self.id); substream_state.read_waker = Some(cx.waker().clone()); Poll::Pending } else { @@ -547,17 +547,17 @@ impl AsyncWrite for Substream { cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - let mut muxer = self.muxer.lock(); + let mut connection = self.connection.lock(); - match muxer.connection.send_stream(self.id).write(buf) { + match connection.connection.send_stream(self.id).write(buf) { Ok(bytes) => { - if let Some(waker) = muxer.poll_connection_waker.take() { + if let Some(waker) = connection.poll_connection_waker.take() { waker.wake(); } Poll::Ready(Ok(bytes)) } Err(quinn_proto::WriteError::Blocked) => { - let substream_state = muxer.unchecked_substream_state(self.id); + let substream_state = connection.unchecked_substream_state(self.id); substream_state.write_waker = Some(cx.waker().clone()); Poll::Pending } @@ -576,15 +576,18 @@ impl AsyncWrite for Substream { } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let mut muxer = self.muxer.lock(); + let mut connection = self.connection.lock(); - if muxer.unchecked_substream_state(self.id).is_write_closed { + if connection + .unchecked_substream_state(self.id) + .is_write_closed + { return Poll::Ready(Ok(())); } - match muxer.finish_substream(self.id) { + match connection.finish_substream(self.id) { Ok(()) => { - let substream_state = muxer.unchecked_substream_state(self.id); + let substream_state = connection.unchecked_substream_state(self.id); substream_state.finished_waker = Some(cx.waker().clone()); Poll::Pending } @@ -600,10 +603,10 @@ impl AsyncWrite for Substream { impl Drop for Substream { fn drop(&mut self) { - let mut muxer = self.muxer.lock(); - muxer.substreams.remove(&self.id); - let _ = muxer.connection.recv_stream(self.id).stop(0u32.into()); - let mut send_stream = muxer.connection.send_stream(self.id); + let mut connection = self.connection.lock(); + connection.substreams.remove(&self.id); + let _ = connection.connection.recv_stream(self.id).stop(0u32.into()); + let mut send_stream = connection.connection.send_stream(self.id); match send_stream.finish() { Ok(()) => {} // Already finished or reset, which is fine. diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 7b5d4d4fd6d..5dba28779cf 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,7 +28,9 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`EndpointDriver`] for a thorough description. -use crate::{muxer::Inner, provider::Provider, tls, transport::SocketFamily, ConnectError, Error}; +use crate::{ + connection::Inner, provider::Provider, tls, transport::SocketFamily, ConnectError, Error, +}; use bytes::BytesMut; use futures::{ @@ -116,7 +118,7 @@ impl From for QuinnConfig { server_config.transport = Arc::clone(&transport); // Disables connection migration. // Long-term this should be enabled, however we then need to handle address change - // on connections in the `Muxer`. + // on connections in the `Connection`. server_config.migration(false); let mut client_config = quinn_proto::ClientConfig::new(client_tls_config); @@ -288,9 +290,9 @@ pub enum ToEndpoint { /// - Receiving packets from the UDP socket and feed them to the [`quinn_proto::Endpoint`] state /// machine. /// - Transmitting events generated by the [`quinn_proto::Endpoint`] to the corresponding -/// [`Connection`]. +/// [`crate::Connection`]. /// - Receiving messages from the `receiver` and processing the requested actions. This includes -/// UDP packets to send and events emitted by the [`Connection`] objects. +/// UDP packets to send and events emitted by the [`crate::Connection`] objects. /// - Sending new connections on `new_connections`. /// /// When it comes to channels, there exists three main multi-producer-single-consumer channels @@ -299,7 +301,7 @@ pub enum ToEndpoint { /// - One channel, represented by `EndpointChannel::to_endpoint` and `receiver`, that communicates /// messages from [`Channel`] to the [`EndpointDriver`]. /// - One channel per each existing connection that communicates messages from the [`EndpointDriver`] -/// to that [`Connection`]. +/// to that [`crate::Connection`]. /// - One channel for the [`EndpointDriver`] to send newly-opened connections to. The receiving /// side is processed by the [`GenTransport`][crate::GenTransport]. /// @@ -310,7 +312,7 @@ pub enum ToEndpoint { /// /// Apart from freezing when the network interface is too busy, the background task should sleep /// as little as possible. It is in particular important for the `receiver` to be drained as -/// quickly as possible in order to avoid unnecessary back-pressure on the [`Connection`] objects. +/// quickly as possible in order to avoid unnecessary back-pressure on the [`crate::Connection`] objects. /// /// ## Back-pressure on `new_connections` /// @@ -401,7 +403,7 @@ impl EndpointDriver

{ } } - /// Handle a message sent from either the [`GenTransport`](super::GenTransport) or a [`Connection`]. + /// Handle a message sent from either the [`GenTransport`](super::GenTransport) or a [`crate::Connection`]. fn handle_message(&mut self, to_endpoint: ToEndpoint) -> ControlFlow<()> { match to_endpoint { ToEndpoint::Dial { addr, result } => { diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index ef623a3a443..fc990effe8d 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -57,17 +57,17 @@ #![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg))] +mod connection; mod endpoint; mod error; -mod muxer; mod provider; mod tls; mod transport; mod upgrade; +pub use connection::Connection; pub use endpoint::Config; pub use error::{ConnectError, ConnectionError, Error}; -pub use muxer::Muxer; #[cfg(feature = "async-std")] pub use provider::async_std; #[cfg(feature = "tokio")] diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index b7fe480d0e8..24d2134db7c 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -22,10 +22,10 @@ //! //! Combines all the objects in the other modules to implement the trait. +use crate::connection::Inner; use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; -use crate::muxer::Inner; use crate::provider::Provider; -use crate::{endpoint, muxer::Muxer, upgrade::Connecting, Error}; +use crate::{connection::Connection, endpoint, upgrade::Connecting, Error}; use futures::channel::{mpsc, oneshot}; use futures::future::BoxFuture; @@ -80,7 +80,7 @@ impl

GenTransport

{ } impl Transport for GenTransport

{ - type Output = (PeerId, Muxer); + type Output = (PeerId, Connection); type Error = Error; type ListenerUpgrade = Connecting; type Dial = BoxFuture<'static, Result>; @@ -249,7 +249,7 @@ impl DialerState { &mut self, address: SocketAddr, timeout: Duration, - ) -> BoxFuture<'static, Result<(PeerId, Muxer), Error>> { + ) -> BoxFuture<'static, Result<(PeerId, Connection), Error>> { let (rx, tx) = oneshot::channel(); let message = ToEndpoint::Dial { @@ -266,9 +266,9 @@ impl DialerState { async move { // Our oneshot getting dropped means the message didn't make it to the endpoint driver. let connection = tx.await.map_err(|_| Error::EndpointDriverCrashed)??; - let (peer, muxer) = Connecting::new(connection, timeout).await?; + let (peer, connection) = Connecting::new(connection, timeout).await?; - Ok((peer, muxer)) + Ok((peer, connection)) } .boxed() } diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index 3aeae5e7330..c34d3031064 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -20,7 +20,7 @@ //! Future that drives a QUIC connection until is has performed its TLS handshake. -use crate::{muxer::Inner, Error, Muxer}; +use crate::{connection::Inner, Connection, Error}; use futures::prelude::*; use futures_timer::Delay; @@ -49,7 +49,7 @@ impl Connecting { } impl Future for Connecting { - type Output = Result<(PeerId, Muxer), Error>; + type Output = Result<(PeerId, Connection), Error>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let connection = self @@ -84,8 +84,8 @@ impl Future for Connecting { .expect("the certificate was validated during TLS handshake; qed"); let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); - let muxer = Muxer::new(self.connection.take().unwrap()); - return Poll::Ready(Ok((peer_id, muxer))); + let connection = Connection::new(self.connection.take().unwrap()); + return Poll::Ready(Ok((peer_id, connection))); } quinn_proto::Event::ConnectionLost { reason } => { return Poll::Ready(Err(Error::Connection(reason.into()))) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 9f876c25a91..60da17e4add 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -40,8 +40,8 @@ async fn create_swarm() -> Swarm> { config.handshake_timeout = Duration::from_secs(1); let transport = quic::GenTransport::

::new(config); - let transport = Transport::map(transport, |(peer, muxer), _| { - (peer, StreamMuxerBox::new(muxer)) + let transport = Transport::map(transport, |(peer, connection), _| { + (peer, StreamMuxerBox::new(connection)) }) .boxed(); From c075dad600c7f677e58d3471e84a81f4f610eccf Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 19:04:04 +0200 Subject: [PATCH 177/218] quic: move polling logic from Inner to Connection Only share in `Inner` the properties that are used by both, Connection and Substreams. Move all logic for communicating with the EndpointDriver into Connection. --- transports/quic/src/connection.rs | 438 +++++++++++++----------------- transports/quic/src/endpoint.rs | 18 +- transports/quic/src/transport.rs | 3 +- transports/quic/src/upgrade.rs | 16 +- 4 files changed, 207 insertions(+), 268 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 0806b8a3c28..7fa6c6a15a9 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -28,6 +28,7 @@ use futures_timer::Delay; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ + any::Any, collections::HashMap, io::{self, Write}, net::SocketAddr, @@ -41,15 +42,166 @@ use std::{ #[derive(Debug)] pub struct Connection { inner: Arc>, + /// Channel to the endpoint this connection belongs to. + endpoint_channel: endpoint::Channel, + /// Pending message to be sent to the background task that is driving the endpoint. + pending_to_endpoint: Option, + /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. + /// Passed at initialization. + from_endpoint: mpsc::Receiver, + /// Identifier for this connection according to the endpoint. Used when sending messages to + /// the endpoint. + connection_id: quinn_proto::ConnectionHandle, + /// `Future` that triggers at the [`Instant`] that `self.connection.poll_timeout()` indicates. + next_timeout: Option<(Delay, Instant)>, } impl Connection { - /// Crate-internal function that builds a [`Connection`] from a raw connection. - pub(crate) fn new(inner: Inner) -> Self { - Connection { + /// Crate-internal function that builds a [`Connection`] from raw components. + /// + /// This function assumes that there exists a [`EndpointDriver`](super::endpoint::EndpointDriver) + /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages + /// on `from_endpoint`. + /// + /// `connection_id` is used to identify the local connection in the messages sent to + /// `to_endpoint`. + /// + /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of + /// its methods has ever been called. Failure to comply might lead to logic errors and panics. + pub fn from_quinn_connection( + endpoint_channel: endpoint::Channel, + connection: quinn_proto::Connection, + connection_id: quinn_proto::ConnectionHandle, + from_endpoint: mpsc::Receiver, + ) -> Self { + debug_assert!(!connection.is_closed()); + let inner = Inner { + connection, + substreams: HashMap::new(), + poll_connection_waker: None, + poll_inbound_waker: None, + poll_outbound_waker: None, + }; + Self { + endpoint_channel, + pending_to_endpoint: None, + next_timeout: None, + from_endpoint, + connection_id, inner: Arc::new(Mutex::new(inner)), } } + + /// The address that the local socket is bound to. + pub fn local_addr(&self) -> &SocketAddr { + self.endpoint_channel.socket_addr() + } + + /// Returns the address of the node we're connected to. + pub fn remote_addr(&self) -> SocketAddr { + self.inner.lock().connection.remote_address() + } + + pub fn peer_identity(&self) -> Option> { + self.inner + .lock() + .connection + .crypto_session() + .peer_identity() + } + + /// Polls the connection for an event that happened on it. + pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { + let mut inner = self.inner.lock(); + loop { + match self.from_endpoint.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => { + inner.connection.handle_event(event); + continue; + } + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => {} + } + + // Sending the pending event to the endpoint. If the endpoint is too busy, we just + // stop the processing here. + // We need to be careful to avoid a potential deadlock if both `from_endpoint` and + // `to_endpoint` are full. As such, we continue to transfer data from `from_endpoint` + // to the `quinn_proto::Connection` (see above). + // However we don't deliver substream-related events to the user as long as + // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` + // being full to the user. + if let Some(to_endpoint) = self.pending_to_endpoint.take() { + match self.endpoint_channel.try_send(to_endpoint, cx) { + Ok(Ok(())) => continue, // The endpoint may send back an event. + Ok(Err(to_endpoint)) => { + self.pending_to_endpoint = Some(to_endpoint); + return Poll::Pending; + } + Err(endpoint::Disconnected {}) => { + return Poll::Ready(None); + } + } + } + + // The maximum amount of segments which can be transmitted in a single Transmit + // if a platform supports Generic Send Offload (GSO). + // Set to 1 for now since not all platforms support GSO. + // TODO: Fix for platforms that support GSO. + let max_datagrams = 1; + // Poll the connection for packets to send on the UDP socket and try to send them on + // `to_endpoint`. + if let Some(transmit) = inner + .connection + .poll_transmit(Instant::now(), max_datagrams) + { + // TODO: ECN bits not handled + self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket(transmit)); + continue; + } + + match inner.connection.poll_timeout() { + Some(timeout) => match self.next_timeout { + Some((_, when)) if when == timeout => {} + _ => { + let now = Instant::now(); + // 0ns if now > when + let duration = timeout.duration_since(now); + let next_timeout = Delay::new(duration); + self.next_timeout = Some((next_timeout, timeout)) + } + }, + None => self.next_timeout = None, + } + + if let Some((timeout, when)) = self.next_timeout.as_mut() { + if timeout.poll_unpin(cx).is_ready() { + inner.connection.handle_timeout(*when); + continue; + } + } + + // The connection also needs to be able to send control messages to the endpoint. This is + // handled here, and we try to send them on `to_endpoint` as well. + if let Some(event) = inner.connection.poll_endpoint_events() { + let connection_id = self.connection_id; + self.pending_to_endpoint = Some(ToEndpoint::ProcessConnectionEvent { + connection_id, + event, + }); + continue; + } + + // The final step consists in handling the events related to the various substreams. + if let Some(ev) = inner.connection.poll() { + return Poll::Ready(Some(ev)); + } + + return Poll::Pending; + } + } } impl StreamMuxer for Connection { @@ -57,13 +209,11 @@ impl StreamMuxer for Connection { type Error = Error; fn poll( - self: Pin<&mut Self>, + mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - let mut inner = self.inner.lock(); - // Poll the inner [`quinn_proto::Connection`] for events and wake - // the wakers of related poll-based methods. - while let Poll::Ready(event) = inner.poll_event(cx) { + while let Poll::Ready(event) = self.poll_event(cx) { + let mut inner = self.inner.lock(); let event = match event { Some(event) => event, None => return Poll::Ready(Err(Error::EndpointDriverCrashed)), @@ -77,7 +227,9 @@ impl StreamMuxer for Connection { ); } quinn_proto::Event::ConnectionLost { reason } => { - inner.close(); + inner + .connection + .close(Instant::now(), From::from(0u32), Default::default()); inner.substreams.values_mut().for_each(|s| s.wake_all()); return Poll::Ready(Err(Error::Connection(reason.into()))); } @@ -137,7 +289,7 @@ impl StreamMuxer for Connection { // TODO: If connection migration is enabled (currently disabled) address // change on the connection needs to be handled. - inner.poll_connection_waker = Some(cx.waker().clone()); + self.inner.lock().poll_connection_waker = Some(cx.waker().clone()); Poll::Pending } @@ -147,7 +299,7 @@ impl StreamMuxer for Connection { ) -> Poll> { let mut inner = self.inner.lock(); - let substream_id = match inner.accept_substream() { + let substream_id = match inner.connection.streams().accept(quinn_proto::Dir::Bi) { Some(id) => { inner.poll_outbound_waker = None; id @@ -168,7 +320,7 @@ impl StreamMuxer for Connection { cx: &mut Context<'_>, ) -> Poll> { let mut inner = self.inner.lock(); - let substream_id = match inner.open_substream() { + let substream_id = match inner.connection.streams().open(quinn_proto::Dir::Bi) { Some(id) => { inner.poll_outbound_waker = None; id @@ -183,21 +335,25 @@ impl StreamMuxer for Connection { Poll::Ready(Ok(substream)) } - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let inner = &mut *self.inner.lock(); - if inner.is_drained() { + fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let mut inner = self.inner.lock(); + if inner.connection.is_drained() { return Poll::Ready(Ok(())); } for substream in inner.substreams.keys().cloned().collect::>() { - let _ = inner.finish_substream(substream); + let _ = inner.connection.send_stream(substream).finish(); + } + + if inner.connection.streams().send_streams() == 0 && !inner.connection.is_closed() { + inner + .connection + .close(Instant::now(), From::from(0u32), Default::default()) } + drop(inner); loop { - if inner.send_stream_count() == 0 && !inner.is_closed() { - inner.close() - } - match ready!(inner.poll_event(cx)) { + match ready!(self.poll_event(cx)) { Some(quinn_proto::Event::ConnectionLost { .. }) => return Poll::Ready(Ok(())), None => return Poll::Ready(Err(Error::EndpointDriverCrashed)), _ => {} @@ -206,24 +362,21 @@ impl StreamMuxer for Connection { } } +impl Drop for Connection { + fn drop(&mut self) { + let to_endpoint = ToEndpoint::ProcessConnectionEvent { + connection_id: self.connection_id, + event: quinn_proto::EndpointEvent::drained(), + }; + self.endpoint_channel.send_on_drop(to_endpoint); + } +} + /// Mutex-protected fields of [`Connection`]. #[derive(Debug)] pub struct Inner { - /// Channel to the endpoint this connection belongs to. - endpoint_channel: endpoint::Channel, - /// Pending message to be sent to the background task that is driving the endpoint. - pending_to_endpoint: Option, - /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. - /// Passed at initialization. - from_endpoint: mpsc::Receiver, - - /// The QUIC state machine for this specific connection. + /// The QUIC inner machine for this specific connection. connection: quinn_proto::Connection, - /// Identifier for this connection according to the endpoint. Used when sending messages to - /// the endpoint. - connection_id: quinn_proto::ConnectionHandle, - /// `Future` that triggers at the [`Instant`] that `self.connection.poll_timeout()` indicates. - next_timeout: Option<(Delay, Instant)>, /// State of all the substreams that the muxer reports as open. pub substreams: HashMap, @@ -241,212 +394,6 @@ impl Inner { .get_mut(&id) .expect("Substream should be known.") } - - /// Crate-internal function that builds [`Inner`] from raw components. - /// - /// This function assumes that there exists a [`EndpointDriver`](super::endpoint::EndpointDriver) - /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages - /// on `from_endpoint`. - /// - /// `connection_id` is used to identify the local connection in the messages sent to - /// `to_endpoint`. - /// - /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of - /// its methods has ever been called. Failure to comply might lead to logic errors and panics. - pub fn from_quinn_connection( - endpoint_channel: endpoint::Channel, - connection: quinn_proto::Connection, - connection_id: quinn_proto::ConnectionHandle, - from_endpoint: mpsc::Receiver, - ) -> Self { - debug_assert!(!connection.is_closed()); - Inner { - endpoint_channel, - pending_to_endpoint: None, - connection, - next_timeout: None, - from_endpoint, - connection_id, - substreams: HashMap::new(), - poll_connection_waker: None, - poll_inbound_waker: None, - poll_outbound_waker: None, - } - } - - /// The address that the local socket is bound to. - pub fn local_addr(&self) -> &SocketAddr { - self.endpoint_channel.socket_addr() - } - - /// Returns the address of the node we're connected to. - pub fn remote_addr(&self) -> SocketAddr { - self.connection.remote_address() - } - - /// Start closing the connection. A [`quinn_proto::Event::ConnectionLost`] event will be - /// produced in the future. - pub fn close(&mut self) { - // We send a dummy `0` error code with no message, as the API of StreamMuxer doesn't - // support this. - self.connection - .close(Instant::now(), From::from(0u32), Default::default()); - } - - /// Whether the connection is closed. - /// A [`quinn_proto::Event::ConnectionLost`] event is emitted with details when the - /// connection becomes closed. - pub fn is_closed(&self) -> bool { - self.connection.is_closed() - } - - /// Whether there is no longer any need to keep the connection around. - /// All drained connections have been closed. - pub fn is_drained(&self) -> bool { - self.connection.is_drained() - } - - /// Pops a new substream opened by the remote. - /// - /// If `None` is returned, then a [`quinn_proto::StreamEvent::Available`] event will later be - /// produced when a substream is available. - pub fn accept_substream(&mut self) -> Option { - self.connection.streams().accept(quinn_proto::Dir::Bi) - } - - /// Pops a new substream opened locally. - /// - /// The API can be thought as if outgoing substreams were automatically opened by the local - /// QUIC connection and were added to a queue for availability. - /// - /// If `None` is returned, then a [`quinn_proto::StreamEvent::Opened`] event will later be - /// produced when a substream is available. - pub fn open_substream(&mut self) -> Option { - self.connection.streams().open(quinn_proto::Dir::Bi) - } - - /// Number of streams that may have unacknowledged data. - pub fn send_stream_count(&mut self) -> usize { - self.connection.streams().send_streams() - } - - /// Closes the given substream. - /// - /// `write_substream` must no longer be called. The substream is however still - /// readable. - /// - /// On success, a [`quinn_proto::StreamEvent::Finished`] event will later be produced when the - /// substream has been effectively closed. A [`quinn_proto::StreamEvent::Stopped`] event can also - /// be emitted. - pub fn finish_substream( - &mut self, - id: quinn_proto::StreamId, - ) -> Result<(), quinn_proto::FinishError> { - self.connection.send_stream(id).finish() - } - - pub fn crypto_session(&self) -> &dyn quinn_proto::crypto::Session { - self.connection.crypto_session() - } - - /// Polls the connection for an event that happened on it. - pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { - loop { - match self.from_endpoint.poll_next_unpin(cx) { - Poll::Ready(Some(event)) => { - self.connection.handle_event(event); - continue; - } - Poll::Ready(None) => { - return Poll::Ready(None); - } - Poll::Pending => {} - } - - // Sending the pending event to the endpoint. If the endpoint is too busy, we just - // stop the processing here. - // We need to be careful to avoid a potential deadlock if both `from_endpoint` and - // `to_endpoint` are full. As such, we continue to transfer data from `from_endpoint` - // to the `quinn_proto::Connection` (see above). - // However we don't deliver substream-related events to the user as long as - // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` - // being full to the user. - if let Some(to_endpoint) = self.pending_to_endpoint.take() { - match self.endpoint_channel.try_send(to_endpoint, cx) { - Ok(Ok(())) => continue, // The endpoint may send back an event. - Ok(Err(to_endpoint)) => { - self.pending_to_endpoint = Some(to_endpoint); - return Poll::Pending; - } - Err(endpoint::Disconnected {}) => { - return Poll::Ready(None); - } - } - } - - // The maximum amount of segments which can be transmitted in a single Transmit - // if a platform supports Generic Send Offload (GSO). - // Set to 1 for now since not all platforms support GSO. - // TODO: Fix for platforms that support GSO. - let max_datagrams = 1; - // Poll the connection for packets to send on the UDP socket and try to send them on - // `to_endpoint`. - if let Some(transmit) = self.connection.poll_transmit(Instant::now(), max_datagrams) { - // TODO: ECN bits not handled - self.pending_to_endpoint = Some(ToEndpoint::SendUdpPacket(transmit)); - continue; - } - - match self.connection.poll_timeout() { - Some(timeout) => match self.next_timeout { - Some((_, when)) if when == timeout => {} - _ => { - let now = Instant::now(); - // 0ns if now > when - let duration = timeout.duration_since(now); - let next_timeout = Delay::new(duration); - self.next_timeout = Some((next_timeout, timeout)) - } - }, - None => self.next_timeout = None, - } - - if let Some((timeout, when)) = self.next_timeout.as_mut() { - if timeout.poll_unpin(cx).is_ready() { - self.connection.handle_timeout(*when); - continue; - } - } - - // The connection also needs to be able to send control messages to the endpoint. This is - // handled here, and we try to send them on `to_endpoint` as well. - if let Some(event) = self.connection.poll_endpoint_events() { - let connection_id = self.connection_id; - self.pending_to_endpoint = Some(ToEndpoint::ProcessConnectionEvent { - connection_id, - event, - }); - continue; - } - - // The final step consists in handling the events related to the various substreams. - if let Some(ev) = self.connection.poll() { - return Poll::Ready(Some(ev)); - } - - return Poll::Pending; - } - } -} - -impl Drop for Inner { - fn drop(&mut self) { - let to_endpoint = ToEndpoint::ProcessConnectionEvent { - connection_id: self.connection_id, - event: quinn_proto::EndpointEvent::drained(), - }; - self.endpoint_channel.send_on_drop(to_endpoint); - } } /// State of a single substream. @@ -576,18 +523,15 @@ impl AsyncWrite for Substream { } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let mut connection = self.connection.lock(); + let mut inner = self.connection.lock(); - if connection - .unchecked_substream_state(self.id) - .is_write_closed - { + if inner.unchecked_substream_state(self.id).is_write_closed { return Poll::Ready(Ok(())); } - match connection.finish_substream(self.id) { + match inner.connection.send_stream(self.id).finish() { Ok(()) => { - let substream_state = connection.unchecked_substream_state(self.id); + let substream_state = inner.unchecked_substream_state(self.id); substream_state.finished_waker = Some(cx.waker().clone()); Poll::Pending } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 5dba28779cf..6ae0c1f44d0 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -28,9 +28,7 @@ //! the rest of the code only happens through channels. See the documentation of the //! [`EndpointDriver`] for a thorough description. -use crate::{ - connection::Inner, provider::Provider, tls, transport::SocketFamily, ConnectError, Error, -}; +use crate::{provider::Provider, tls, transport::SocketFamily, ConnectError, Connection, Error}; use bytes::BytesMut; use futures::{ @@ -148,7 +146,7 @@ impl Channel { pub fn new_bidirectional( quinn_config: QuinnConfig, socket_addr: SocketAddr, - ) -> Result<(Self, mpsc::Receiver), Error> { + ) -> Result<(Self, mpsc::Receiver), Error> { let (new_connections_tx, new_connections_rx) = mpsc::channel(1); let endpoint = Self::new::

(quinn_config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) @@ -169,7 +167,7 @@ impl Channel { fn new( quinn_config: QuinnConfig, socket_addr: SocketAddr, - new_connections: Option>, + new_connections: Option>, ) -> Result { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; @@ -260,7 +258,7 @@ pub enum ToEndpoint { /// UDP address to connect to. addr: SocketAddr, /// Channel to return the result of the dialing to. - result: oneshot::Sender>, + result: 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. Only `quinn_proto` knows what is in @@ -375,7 +373,7 @@ pub struct EndpointDriver { HashMap>, // Channel to forward new inbound connections to the transport. // `None` if server capabilities are disabled, i.e. the endpoint is only used for dialing. - new_connection_tx: Option>, + new_connection_tx: Option>, // Whether the transport dropped its handle for this endpoint. is_decoupled: bool, } @@ -384,7 +382,7 @@ impl EndpointDriver

{ fn new( endpoint_config: Arc, client_config: quinn_proto::ClientConfig, - new_connection_tx: Option>, + new_connection_tx: Option>, server_config: Option>, channel: Channel, socket: P, @@ -421,7 +419,7 @@ impl EndpointDriver

{ debug_assert_eq!(connection.side(), quinn_proto::Side::Client); let (tx, rx) = mpsc::channel(16); - let connection = Inner::from_quinn_connection( + let connection = Connection::from_quinn_connection( self.channel.clone(), connection, connection_id, @@ -530,7 +528,7 @@ impl EndpointDriver

{ let (tx, rx) = mpsc::channel(16); let connection = - Inner::from_quinn_connection(self.channel.clone(), connec, connec_id, rx); + Connection::from_quinn_connection(self.channel.clone(), connec, connec_id, rx); match connection_tx.try_send(connection) { Ok(()) => { self.alive_connections.insert(connec_id, tx); diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 24d2134db7c..31a0ba793fc 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -22,7 +22,6 @@ //! //! Combines all the objects in the other modules to implement the trait. -use crate::connection::Inner; use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; use crate::provider::Provider; use crate::{connection::Connection, endpoint, upgrade::Connecting, Error}; @@ -299,7 +298,7 @@ struct Listener { listener_id: ListenerId, /// Channel where new connections are being sent. - new_connections_rx: mpsc::Receiver, + new_connections_rx: mpsc::Receiver, handshake_timeout: Duration, if_watcher: Option, diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/upgrade.rs index c34d3031064..ba067b92456 100644 --- a/transports/quic/src/upgrade.rs +++ b/transports/quic/src/upgrade.rs @@ -20,7 +20,7 @@ //! Future that drives a QUIC connection until is has performed its TLS handshake. -use crate::{connection::Inner, Connection, Error}; +use crate::{Connection, Error}; use futures::prelude::*; use futures_timer::Delay; @@ -34,15 +34,15 @@ use std::{ /// A QUIC connection currently being negotiated. #[derive(Debug)] pub struct Connecting { - connection: Option, + connection: Option, timeout: Delay, } impl Connecting { - /// Builds an [`Connecting`] that wraps around an [`Inner`] connection. - pub(crate) fn new(inner: Inner, timeout: Duration) -> Self { + /// Builds an [`Connecting`] that wraps around an [`Connection`]. + pub(crate) fn new(connection: Connection, timeout: Duration) -> Self { Connecting { - connection: Some(inner), + connection: Some(connection), timeout: Delay::new(timeout), } } @@ -70,8 +70,7 @@ impl Future for Connecting { }; match event { quinn_proto::Event::Connected => { - let session = connection.crypto_session(); - let identity = session + let identity = connection .peer_identity() .expect("connection got identity because it passed TLS handshake; qed"); let certificates: Box> = @@ -84,8 +83,7 @@ impl Future for Connecting { .expect("the certificate was validated during TLS handshake; qed"); let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); - let connection = Connection::new(self.connection.take().unwrap()); - return Poll::Ready(Ok((peer_id, connection))); + return Poll::Ready(Ok((peer_id, self.connection.take().unwrap()))); } quinn_proto::Event::ConnectionLost { reason } => { return Poll::Ready(Err(Error::Connection(reason.into()))) From 954908b690b52aa477bf6d8dfb19bdc5931b03d4 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 19:14:01 +0200 Subject: [PATCH 178/218] quic: rename Inner -> State --- transports/quic/src/connection.rs | 64 +++++++++++++++---------------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 7fa6c6a15a9..0aea9bca763 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -41,7 +41,7 @@ use std::{ /// State for a single opened QUIC connection. #[derive(Debug)] pub struct Connection { - inner: Arc>, + state: Arc>, /// Channel to the endpoint this connection belongs to. endpoint_channel: endpoint::Channel, /// Pending message to be sent to the background task that is driving the endpoint. @@ -75,7 +75,7 @@ impl Connection { from_endpoint: mpsc::Receiver, ) -> Self { debug_assert!(!connection.is_closed()); - let inner = Inner { + let state = State { connection, substreams: HashMap::new(), poll_connection_waker: None, @@ -88,7 +88,7 @@ impl Connection { next_timeout: None, from_endpoint, connection_id, - inner: Arc::new(Mutex::new(inner)), + state: Arc::new(Mutex::new(state)), } } @@ -99,11 +99,11 @@ impl Connection { /// Returns the address of the node we're connected to. pub fn remote_addr(&self) -> SocketAddr { - self.inner.lock().connection.remote_address() + self.state.lock().connection.remote_address() } pub fn peer_identity(&self) -> Option> { - self.inner + self.state .lock() .connection .crypto_session() @@ -112,7 +112,7 @@ impl Connection { /// Polls the connection for an event that happened on it. pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { - let mut inner = self.inner.lock(); + let mut inner = self.state.lock(); loop { match self.from_endpoint.poll_next_unpin(cx) { Poll::Ready(Some(event)) => { @@ -213,7 +213,7 @@ impl StreamMuxer for Connection { cx: &mut Context<'_>, ) -> Poll> { while let Poll::Ready(event) = self.poll_event(cx) { - let mut inner = self.inner.lock(); + let mut inner = self.state.lock(); let event = match event { Some(event) => event, None => return Poll::Ready(Err(Error::EndpointDriverCrashed)), @@ -289,7 +289,7 @@ impl StreamMuxer for Connection { // TODO: If connection migration is enabled (currently disabled) address // change on the connection needs to be handled. - self.inner.lock().poll_connection_waker = Some(cx.waker().clone()); + self.state.lock().poll_connection_waker = Some(cx.waker().clone()); Poll::Pending } @@ -297,7 +297,7 @@ impl StreamMuxer for Connection { self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - let mut inner = self.inner.lock(); + let mut inner = self.state.lock(); let substream_id = match inner.connection.streams().accept(quinn_proto::Dir::Bi) { Some(id) => { @@ -310,7 +310,7 @@ impl StreamMuxer for Connection { } }; inner.substreams.insert(substream_id, Default::default()); - let substream = Substream::new(substream_id, self.inner.clone()); + let substream = Substream::new(substream_id, self.state.clone()); Poll::Ready(Ok(substream)) } @@ -319,7 +319,7 @@ impl StreamMuxer for Connection { self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - let mut inner = self.inner.lock(); + let mut inner = self.state.lock(); let substream_id = match inner.connection.streams().open(quinn_proto::Dir::Bi) { Some(id) => { inner.poll_outbound_waker = None; @@ -331,12 +331,12 @@ impl StreamMuxer for Connection { } }; inner.substreams.insert(substream_id, Default::default()); - let substream = Substream::new(substream_id, self.inner.clone()); + let substream = Substream::new(substream_id, self.state.clone()); Poll::Ready(Ok(substream)) } fn poll_close(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let mut inner = self.inner.lock(); + let mut inner = self.state.lock(); if inner.connection.is_drained() { return Poll::Ready(Ok(())); } @@ -372,9 +372,9 @@ impl Drop for Connection { } } -/// Mutex-protected fields of [`Connection`]. +/// Mutex-protected state of [`Connection`]. #[derive(Debug)] -pub struct Inner { +pub struct State { /// The QUIC inner machine for this specific connection. connection: quinn_proto::Connection, @@ -388,7 +388,7 @@ pub struct Inner { pub poll_connection_waker: Option, } -impl Inner { +impl State { fn unchecked_substream_state(&mut self, id: quinn_proto::StreamId) -> &mut SubstreamState { self.substreams .get_mut(&id) @@ -426,12 +426,12 @@ impl SubstreamState { #[derive(Debug)] pub struct Substream { id: quinn_proto::StreamId, - connection: Arc>, + state: Arc>, } impl Substream { - fn new(id: quinn_proto::StreamId, connection: Arc>) -> Self { - Self { id, connection } + fn new(id: quinn_proto::StreamId, state: Arc>) -> Self { + Self { id, state } } } @@ -441,9 +441,9 @@ impl AsyncRead for Substream { cx: &mut Context<'_>, mut buf: &mut [u8], ) -> Poll> { - let mut connection = self.connection.lock(); + let mut state = self.state.lock(); - let mut stream = connection.connection.recv_stream(self.id); + let mut stream = state.connection.recv_stream(self.id); let mut chunks = match stream.read(true) { Ok(chunks) => chunks, Err(quinn_proto::ReadableError::UnknownStream) => { @@ -474,12 +474,12 @@ impl AsyncRead for Substream { bytes += chunk.bytes.len(); } if chunks.finalize().should_transmit() { - if let Some(waker) = connection.poll_connection_waker.take() { + if let Some(waker) = state.poll_connection_waker.take() { waker.wake(); } } if pending && bytes == 0 { - let substream_state = connection.unchecked_substream_state(self.id); + let substream_state = state.unchecked_substream_state(self.id); substream_state.read_waker = Some(cx.waker().clone()); Poll::Pending } else { @@ -494,17 +494,17 @@ impl AsyncWrite for Substream { cx: &mut Context<'_>, buf: &[u8], ) -> Poll> { - let mut connection = self.connection.lock(); + let mut state = self.state.lock(); - match connection.connection.send_stream(self.id).write(buf) { + match state.connection.send_stream(self.id).write(buf) { Ok(bytes) => { - if let Some(waker) = connection.poll_connection_waker.take() { + if let Some(waker) = state.poll_connection_waker.take() { waker.wake(); } Poll::Ready(Ok(bytes)) } Err(quinn_proto::WriteError::Blocked) => { - let substream_state = connection.unchecked_substream_state(self.id); + let substream_state = state.unchecked_substream_state(self.id); substream_state.write_waker = Some(cx.waker().clone()); Poll::Pending } @@ -523,7 +523,7 @@ impl AsyncWrite for Substream { } fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let mut inner = self.connection.lock(); + let mut inner = self.state.lock(); if inner.unchecked_substream_state(self.id).is_write_closed { return Poll::Ready(Ok(())); @@ -547,10 +547,10 @@ impl AsyncWrite for Substream { impl Drop for Substream { fn drop(&mut self) { - let mut connection = self.connection.lock(); - connection.substreams.remove(&self.id); - let _ = connection.connection.recv_stream(self.id).stop(0u32.into()); - let mut send_stream = connection.connection.send_stream(self.id); + let mut state = self.state.lock(); + state.substreams.remove(&self.id); + let _ = state.connection.recv_stream(self.id).stop(0u32.into()); + let mut send_stream = state.connection.send_stream(self.id); match send_stream.finish() { Ok(()) => {} // Already finished or reset, which is fine. From 80348505e1a66cab55a0f59a87818347c11481c8 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 16 Oct 2022 19:25:54 +0200 Subject: [PATCH 179/218] quic: submodules for connection - Move substream-related logic into crate::connection::substream - Rename/ move crate::upgrade -> crate::connection::connecting --- transports/quic/src/connection.rs | 176 +--------------- .../{upgrade.rs => connection/connecting.rs} | 0 transports/quic/src/connection/substream.rs | 197 ++++++++++++++++++ transports/quic/src/lib.rs | 4 +- transports/quic/src/transport.rs | 2 +- 5 files changed, 207 insertions(+), 172 deletions(-) rename transports/quic/src/{upgrade.rs => connection/connecting.rs} (100%) create mode 100644 transports/quic/src/connection/substream.rs diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 0aea9bca763..98855c44859 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -18,19 +18,24 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +mod connecting; +mod substream; + use crate::{ endpoint::{self, ToEndpoint}, Error, }; +pub use connecting::Connecting; +pub use substream::Substream; +use substream::SubstreamState; -use futures::{channel::mpsc, ready, AsyncRead, AsyncWrite, FutureExt, StreamExt}; +use futures::{channel::mpsc, ready, FutureExt, StreamExt}; use futures_timer::Delay; use libp2p_core::muxing::{StreamMuxer, StreamMuxerEvent}; use parking_lot::Mutex; use std::{ any::Any, collections::HashMap, - io::{self, Write}, net::SocketAddr, pin::Pin, sync::Arc, @@ -380,6 +385,7 @@ pub struct State { /// State of all the substreams that the muxer reports as open. pub substreams: HashMap, + /// Waker to wake if a new outbound substream is opened. pub poll_outbound_waker: Option, /// Waker to wake if a new inbound substream was happened. @@ -395,169 +401,3 @@ impl State { .expect("Substream should be known.") } } - -/// State of a single substream. -#[derive(Debug, Default, Clone)] -pub struct SubstreamState { - /// Waker to wake if the substream becomes readable or stopped. - read_waker: Option, - /// Waker to wake if the substream becomes writable or stopped. - write_waker: Option, - /// Waker to wake if the substream becomes closed or stopped. - finished_waker: Option, - - is_write_closed: bool, -} - -impl SubstreamState { - fn wake_all(&mut self) { - if let Some(waker) = self.read_waker.take() { - waker.wake(); - } - if let Some(waker) = self.write_waker.take() { - waker.wake(); - } - if let Some(waker) = self.finished_waker.take() { - waker.wake(); - } - } -} - -#[derive(Debug)] -pub struct Substream { - id: quinn_proto::StreamId, - state: Arc>, -} - -impl Substream { - fn new(id: quinn_proto::StreamId, state: Arc>) -> Self { - Self { id, state } - } -} - -impl AsyncRead for Substream { - fn poll_read( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - mut buf: &mut [u8], - ) -> Poll> { - let mut state = self.state.lock(); - - let mut stream = state.connection.recv_stream(self.id); - let mut chunks = match stream.read(true) { - Ok(chunks) => chunks, - Err(quinn_proto::ReadableError::UnknownStream) => { - return Poll::Ready(Ok(0)); - } - Err(quinn_proto::ReadableError::IllegalOrderedRead) => { - unreachable!( - "Illegal ordered read can only happen if `stream.read(false)` is used." - ); - } - }; - let mut bytes = 0; - let mut pending = false; - loop { - let chunk = match chunks.next(buf.len()) { - Ok(Some(chunk)) if !chunk.bytes.is_empty() => chunk, - Ok(_) => break, - Err(err @ quinn_proto::ReadError::Reset(_)) => { - return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) - } - Err(quinn_proto::ReadError::Blocked) => { - pending = true; - break; - } - }; - - buf.write_all(&chunk.bytes).expect("enough buffer space"); - bytes += chunk.bytes.len(); - } - if chunks.finalize().should_transmit() { - if let Some(waker) = state.poll_connection_waker.take() { - waker.wake(); - } - } - if pending && bytes == 0 { - let substream_state = state.unchecked_substream_state(self.id); - substream_state.read_waker = Some(cx.waker().clone()); - Poll::Pending - } else { - Poll::Ready(Ok(bytes)) - } - } -} - -impl AsyncWrite for Substream { - fn poll_write( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - let mut state = self.state.lock(); - - match state.connection.send_stream(self.id).write(buf) { - Ok(bytes) => { - if let Some(waker) = state.poll_connection_waker.take() { - waker.wake(); - } - Poll::Ready(Ok(bytes)) - } - Err(quinn_proto::WriteError::Blocked) => { - let substream_state = state.unchecked_substream_state(self.id); - substream_state.write_waker = Some(cx.waker().clone()); - Poll::Pending - } - Err(err @ quinn_proto::WriteError::Stopped(_)) => { - Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) - } - Err(quinn_proto::WriteError::UnknownStream) => { - Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())) - } - } - } - - fn poll_flush(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { - // quinn doesn't support flushing, calling close will flush all substreams. - Poll::Ready(Ok(())) - } - - fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let mut inner = self.state.lock(); - - if inner.unchecked_substream_state(self.id).is_write_closed { - return Poll::Ready(Ok(())); - } - - match inner.connection.send_stream(self.id).finish() { - Ok(()) => { - let substream_state = inner.unchecked_substream_state(self.id); - substream_state.finished_waker = Some(cx.waker().clone()); - Poll::Pending - } - Err(err @ quinn_proto::FinishError::Stopped(_)) => { - Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) - } - Err(quinn_proto::FinishError::UnknownStream) => { - Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())) - } - } - } -} - -impl Drop for Substream { - fn drop(&mut self) { - let mut state = self.state.lock(); - state.substreams.remove(&self.id); - let _ = state.connection.recv_stream(self.id).stop(0u32.into()); - let mut send_stream = state.connection.send_stream(self.id); - match send_stream.finish() { - Ok(()) => {} - // Already finished or reset, which is fine. - Err(quinn_proto::FinishError::UnknownStream) => {} - Err(quinn_proto::FinishError::Stopped(reason)) => { - let _ = send_stream.reset(reason); - } - } - } -} diff --git a/transports/quic/src/upgrade.rs b/transports/quic/src/connection/connecting.rs similarity index 100% rename from transports/quic/src/upgrade.rs rename to transports/quic/src/connection/connecting.rs diff --git a/transports/quic/src/connection/substream.rs b/transports/quic/src/connection/substream.rs new file mode 100644 index 00000000000..9fac101b4e8 --- /dev/null +++ b/transports/quic/src/connection/substream.rs @@ -0,0 +1,197 @@ +// Copyright 2022 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 std::{ + io::{self, Write}, + pin::Pin, + sync::Arc, + task::{Context, Poll, Waker}, +}; + +use futures::{AsyncRead, AsyncWrite}; +use parking_lot::Mutex; + +use super::State; + +/// State of a single substream. +#[derive(Debug, Default, Clone)] +pub struct SubstreamState { + /// Waker to wake if the substream becomes readable or stopped. + pub read_waker: Option, + /// Waker to wake if the substream becomes writable or stopped. + pub write_waker: Option, + /// Waker to wake if the substream becomes closed or stopped. + pub finished_waker: Option, + + pub is_write_closed: bool, +} + +impl SubstreamState { + pub fn wake_all(&mut self) { + if let Some(waker) = self.read_waker.take() { + waker.wake(); + } + if let Some(waker) = self.write_waker.take() { + waker.wake(); + } + if let Some(waker) = self.finished_waker.take() { + waker.wake(); + } + } +} + +#[derive(Debug)] +pub struct Substream { + id: quinn_proto::StreamId, + state: Arc>, +} + +impl Substream { + pub fn new(id: quinn_proto::StreamId, state: Arc>) -> Self { + Self { id, state } + } +} + +impl AsyncRead for Substream { + fn poll_read( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + mut buf: &mut [u8], + ) -> Poll> { + let mut state = self.state.lock(); + + let mut stream = state.connection.recv_stream(self.id); + let mut chunks = match stream.read(true) { + Ok(chunks) => chunks, + Err(quinn_proto::ReadableError::UnknownStream) => { + return Poll::Ready(Ok(0)); + } + Err(quinn_proto::ReadableError::IllegalOrderedRead) => { + unreachable!( + "Illegal ordered read can only happen if `stream.read(false)` is used." + ); + } + }; + let mut bytes = 0; + let mut pending = false; + loop { + let chunk = match chunks.next(buf.len()) { + Ok(Some(chunk)) if !chunk.bytes.is_empty() => chunk, + Ok(_) => break, + Err(err @ quinn_proto::ReadError::Reset(_)) => { + return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) + } + Err(quinn_proto::ReadError::Blocked) => { + pending = true; + break; + } + }; + + buf.write_all(&chunk.bytes).expect("enough buffer space"); + bytes += chunk.bytes.len(); + } + if chunks.finalize().should_transmit() { + if let Some(waker) = state.poll_connection_waker.take() { + waker.wake(); + } + } + if pending && bytes == 0 { + let substream_state = state.unchecked_substream_state(self.id); + substream_state.read_waker = Some(cx.waker().clone()); + Poll::Pending + } else { + Poll::Ready(Ok(bytes)) + } + } +} + +impl AsyncWrite for Substream { + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + let mut state = self.state.lock(); + + match state.connection.send_stream(self.id).write(buf) { + Ok(bytes) => { + if let Some(waker) = state.poll_connection_waker.take() { + waker.wake(); + } + Poll::Ready(Ok(bytes)) + } + Err(quinn_proto::WriteError::Blocked) => { + let substream_state = state.unchecked_substream_state(self.id); + substream_state.write_waker = Some(cx.waker().clone()); + Poll::Pending + } + Err(err @ quinn_proto::WriteError::Stopped(_)) => { + Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) + } + Err(quinn_proto::WriteError::UnknownStream) => { + Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())) + } + } + } + + fn poll_flush(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + // quinn doesn't support flushing, calling close will flush all substreams. + Poll::Ready(Ok(())) + } + + fn poll_close(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let mut inner = self.state.lock(); + + if inner.unchecked_substream_state(self.id).is_write_closed { + return Poll::Ready(Ok(())); + } + + match inner.connection.send_stream(self.id).finish() { + Ok(()) => { + let substream_state = inner.unchecked_substream_state(self.id); + substream_state.finished_waker = Some(cx.waker().clone()); + Poll::Pending + } + Err(err @ quinn_proto::FinishError::Stopped(_)) => { + Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) + } + Err(quinn_proto::FinishError::UnknownStream) => { + Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())) + } + } + } +} + +impl Drop for Substream { + fn drop(&mut self) { + let mut state = self.state.lock(); + state.substreams.remove(&self.id); + let _ = state.connection.recv_stream(self.id).stop(0u32.into()); + let mut send_stream = state.connection.send_stream(self.id); + match send_stream.finish() { + Ok(()) => {} + // Already finished or reset, which is fine. + Err(quinn_proto::FinishError::UnknownStream) => {} + Err(quinn_proto::FinishError::Stopped(reason)) => { + let _ = send_stream.reset(reason); + } + } + } +} diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index fc990effe8d..6e85f2235b2 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -63,9 +63,8 @@ mod error; mod provider; mod tls; mod transport; -mod upgrade; -pub use connection::Connection; +pub use connection::{Connecting, Connection, Substream}; pub use endpoint::Config; pub use error::{ConnectError, ConnectionError, Error}; #[cfg(feature = "async-std")] @@ -74,4 +73,3 @@ pub use provider::async_std; pub use provider::tokio; pub use provider::Provider; pub use transport::GenTransport; -pub use upgrade::Connecting; diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 31a0ba793fc..cf4f2e41d4a 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -24,7 +24,7 @@ use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; use crate::provider::Provider; -use crate::{connection::Connection, endpoint, upgrade::Connecting, Error}; +use crate::{endpoint, Connecting, Connection, Error}; use futures::channel::{mpsc, oneshot}; use futures::future::BoxFuture; From f146834b8f29190d3078659875f19d723ec3c1ce Mon Sep 17 00:00:00 2001 From: Max Inden Date: Fri, 14 Oct 2022 15:30:16 +0100 Subject: [PATCH 180/218] *: Prepare v0.49.0 (#2931) --- CHANGELOG.md | 2 +- core/CHANGELOG.md | 4 ++-- misc/metrics/CHANGELOG.md | 2 +- misc/multistream-select/CHANGELOG.md | 2 +- muxers/mplex/CHANGELOG.md | 2 +- muxers/yamux/CHANGELOG.md | 2 +- protocols/autonat/CHANGELOG.md | 4 ++-- protocols/dcutr/CHANGELOG.md | 2 +- protocols/floodsub/CHANGELOG.md | 2 +- protocols/gossipsub/CHANGELOG.md | 2 +- protocols/identify/CHANGELOG.md | 2 +- protocols/kad/CHANGELOG.md | 2 +- protocols/mdns/CHANGELOG.md | 2 +- protocols/ping/CHANGELOG.md | 2 +- protocols/relay/CHANGELOG.md | 2 +- protocols/rendezvous/CHANGELOG.md | 2 +- protocols/request-response/CHANGELOG.md | 2 +- swarm-derive/CHANGELOG.md | 2 +- swarm/CHANGELOG.md | 4 ++-- transports/deflate/CHANGELOG.md | 2 +- transports/dns/CHANGELOG.md | 2 +- transports/noise/CHANGELOG.md | 2 +- transports/plaintext/CHANGELOG.md | 2 +- transports/pnet/CHANGELOG.md | 2 +- transports/tcp/CHANGELOG.md | 2 +- transports/uds/CHANGELOG.md | 2 +- transports/wasm-ext/CHANGELOG.md | 2 +- transports/websocket/CHANGELOG.md | 2 +- 28 files changed, 31 insertions(+), 31 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 03b343314ff..65999fbb193 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -43,7 +43,7 @@ # `libp2p` facade crate -# 0.49.0 - [unreleased] +# 0.49.0 - Remove default features. You need to enable required features explicitly now. As a quick workaround, you may want to use the new `full` feature which activates all features. See [PR 2918]. diff --git a/core/CHANGELOG.md b/core/CHANGELOG.md index a04b92435ad..e996824209c 100644 --- a/core/CHANGELOG.md +++ b/core/CHANGELOG.md @@ -1,8 +1,8 @@ -# 0.37.0 [unreleased] +# 0.37.0 - Implement `Hash` and `Ord` for `PublicKey`. See [PR 2915]. -- Remove default features. If you previously depended on `secp256k1` or `ecdsa` you need to enable these explicitly +- Remove default features. If you previously depended on `secp256k1` or `ecdsa` you need to enable these explicitly now. See [PR 2918]. - Deprecate `StreamMuxerExt::next_{inbound,outbound}`. See [PR 3002]. diff --git a/misc/metrics/CHANGELOG.md b/misc/metrics/CHANGELOG.md index fe428745d70..d76ef307e99 100644 --- a/misc/metrics/CHANGELOG.md +++ b/misc/metrics/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.10.0 [unreleased] +# 0.10.0 - Update to `libp2p-swarm` `v0.40.0`. diff --git a/misc/multistream-select/CHANGELOG.md b/misc/multistream-select/CHANGELOG.md index 4b15cc51ba1..820d18449e5 100644 --- a/misc/multistream-select/CHANGELOG.md +++ b/misc/multistream-select/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.12.0 [unreleased] +# 0.12.0 - Remove parallel dialing optimization, to avoid requiring the use of the `ls` command. See [PR 2934]. diff --git a/muxers/mplex/CHANGELOG.md b/muxers/mplex/CHANGELOG.md index 161cdbc9a64..ed1c5c7382a 100644 --- a/muxers/mplex/CHANGELOG.md +++ b/muxers/mplex/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.37.0 [unreleased] +# 0.37.0 - Bump rand to 0.8 and quickcheck to 1. See [PR 2857]. diff --git a/muxers/yamux/CHANGELOG.md b/muxers/yamux/CHANGELOG.md index 41eff104500..b3aa8dac0e1 100644 --- a/muxers/yamux/CHANGELOG.md +++ b/muxers/yamux/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.41.0 [unreleased] +# 0.41.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/protocols/autonat/CHANGELOG.md b/protocols/autonat/CHANGELOG.md index 06809d78c29..6f55afa4c06 100644 --- a/protocols/autonat/CHANGELOG.md +++ b/protocols/autonat/CHANGELOG.md @@ -1,5 +1,5 @@ -# 0.8.0 [unreleased] - +# 0.8.0 + - Update to `libp2p-core` `v0.37.0`. - Update to `libp2p-swarm` `v0.40.0`. diff --git a/protocols/dcutr/CHANGELOG.md b/protocols/dcutr/CHANGELOG.md index dabd0ffbd49..0d49f90d008 100644 --- a/protocols/dcutr/CHANGELOG.md +++ b/protocols/dcutr/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.7.0 [unreleased] +# 0.7.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/protocols/floodsub/CHANGELOG.md b/protocols/floodsub/CHANGELOG.md index e7c1392c642..dd42387eda2 100644 --- a/protocols/floodsub/CHANGELOG.md +++ b/protocols/floodsub/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.40.0 [unreleased] +# 0.40.0 - Bump rand to 0.8 and quickcheck to 1. See [PR 2857]. diff --git a/protocols/gossipsub/CHANGELOG.md b/protocols/gossipsub/CHANGELOG.md index 99d2ad3a1c3..555c146af11 100644 --- a/protocols/gossipsub/CHANGELOG.md +++ b/protocols/gossipsub/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.42.0 [unreleased] +# 0.42.0 - Bump rand to 0.8 and quickcheck to 1. See [PR 2857]. diff --git a/protocols/identify/CHANGELOG.md b/protocols/identify/CHANGELOG.md index 00ed1e138e2..f60ace8ace0 100644 --- a/protocols/identify/CHANGELOG.md +++ b/protocols/identify/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.40.0 [unreleased] +# 0.40.0 - Update dependencies. diff --git a/protocols/kad/CHANGELOG.md b/protocols/kad/CHANGELOG.md index dcf4eedf2fb..9a76bb9c9df 100644 --- a/protocols/kad/CHANGELOG.md +++ b/protocols/kad/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.41.0 [unreleased] +# 0.41.0 - Remove deprecated `set_protocol_name()` from `KademliaConfig` & `KademliaProtocolConfig`. Use `set_protocol_names()` instead. See [PR 2866]. diff --git a/protocols/mdns/CHANGELOG.md b/protocols/mdns/CHANGELOG.md index 915564341ae..66a2ee57394 100644 --- a/protocols/mdns/CHANGELOG.md +++ b/protocols/mdns/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.41.0 [unreleased] +# 0.41.0 - Remove default features. If you previously depended on `async-io` you need to enable this explicitly now. See [PR 2918]. diff --git a/protocols/ping/CHANGELOG.md b/protocols/ping/CHANGELOG.md index 923dfa48c84..bcdcdf9ab0e 100644 --- a/protocols/ping/CHANGELOG.md +++ b/protocols/ping/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.40.0 [unreleased] +# 0.40.0 - Bump rand to 0.8 and quickcheck to 1. See [PR 2857]. - Deprecate types with `Ping` prefix. Prefer importing them via the `ping` namespace, i.e. `libp2p::ping::Event` instead diff --git a/protocols/relay/CHANGELOG.md b/protocols/relay/CHANGELOG.md index 9c4df7c9c3e..b21d4123b08 100644 --- a/protocols/relay/CHANGELOG.md +++ b/protocols/relay/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.13.0 [unreleased] +# 0.13.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/protocols/rendezvous/CHANGELOG.md b/protocols/rendezvous/CHANGELOG.md index 3e04334ad50..1b858855108 100644 --- a/protocols/rendezvous/CHANGELOG.md +++ b/protocols/rendezvous/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.10.0 [unreleased] +# 0.10.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/protocols/request-response/CHANGELOG.md b/protocols/request-response/CHANGELOG.md index 7e4ba48044d..a48d74ee82f 100644 --- a/protocols/request-response/CHANGELOG.md +++ b/protocols/request-response/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.22.0 [unreleased] +# 0.22.0 - Bump rand to 0.8 and quickcheck to 1. See [PR 2857]. diff --git a/swarm-derive/CHANGELOG.md b/swarm-derive/CHANGELOG.md index 0c64fd352da..722537e6b59 100644 --- a/swarm-derive/CHANGELOG.md +++ b/swarm-derive/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.30.1 [unreleased] +# 0.30.1 - Fix an issue where the derive would generate bad code if the type parameters between the behaviour and a custom out event differed. See [PR 2907]. diff --git a/swarm/CHANGELOG.md b/swarm/CHANGELOG.md index ec7b8e83d4f..1758579dc21 100644 --- a/swarm/CHANGELOG.md +++ b/swarm/CHANGELOG.md @@ -1,10 +1,10 @@ -# 0.40.0 [unreleased] +# 0.40.0 - Bump rand to 0.8 and quickcheck to 1. See [PR 2857]. - Update to `libp2p-core` `v0.37.0`. -- Introduce `libp2p_swarm::keep_alive::ConnectionHandler` in favor of removing `keep_alive` from +- Introduce `libp2p_swarm::keep_alive::ConnectionHandler` in favor of removing `keep_alive` from `libp2p_swarm::dummy::ConnectionHandler`. `dummy::ConnectionHandler` now literally does not do anything. In the same spirit, introduce `libp2p_swarm::keep_alive::Behaviour` and `libp2p_swarm::dummy::Behaviour`. See [PR 2859]. diff --git a/transports/deflate/CHANGELOG.md b/transports/deflate/CHANGELOG.md index d7c58a6e4a9..941efc421f8 100644 --- a/transports/deflate/CHANGELOG.md +++ b/transports/deflate/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.37.0 [unreleased] +# 0.37.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/transports/dns/CHANGELOG.md b/transports/dns/CHANGELOG.md index e20c92f66e0..6d03b237a79 100644 --- a/transports/dns/CHANGELOG.md +++ b/transports/dns/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.37.0 [unreleased] +# 0.37.0 - Remove default features. If you previously depended on `async-std` you need to enable this explicitly now. See [PR 2918]. diff --git a/transports/noise/CHANGELOG.md b/transports/noise/CHANGELOG.md index 6c7b94c5876..ec28bb3ff4d 100644 --- a/transports/noise/CHANGELOG.md +++ b/transports/noise/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.40.0 [unreleased] +# 0.40.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/transports/plaintext/CHANGELOG.md b/transports/plaintext/CHANGELOG.md index a0d21b8d1f3..93162dfa26d 100644 --- a/transports/plaintext/CHANGELOG.md +++ b/transports/plaintext/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.37.0 [unreleased] +# 0.37.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/transports/pnet/CHANGELOG.md b/transports/pnet/CHANGELOG.md index f2e40c03d8f..e63f196c869 100644 --- a/transports/pnet/CHANGELOG.md +++ b/transports/pnet/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.22.1 [unreleased] +# 0.22.1 - Bump rand to 0.8 and quickcheck to 1. See [PR 2857]. diff --git a/transports/tcp/CHANGELOG.md b/transports/tcp/CHANGELOG.md index 83ad2d3c1a3..54cbed563e1 100644 --- a/transports/tcp/CHANGELOG.md +++ b/transports/tcp/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.37.0 - [unreleased] +# 0.37.0 - Update to `if-watch` `v2.0.0`. Simplify `IfWatcher` integration. Use `if_watch::IfWatcher` for all runtimes. See [PR 2813]. diff --git a/transports/uds/CHANGELOG.md b/transports/uds/CHANGELOG.md index 6d7581890c9..97ae47216ac 100644 --- a/transports/uds/CHANGELOG.md +++ b/transports/uds/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.36.0 [unreleased] +# 0.36.0 - Remove default features. If you previously depended on `async-std` you need to enable this explicitly now. See [PR 2918]. diff --git a/transports/wasm-ext/CHANGELOG.md b/transports/wasm-ext/CHANGELOG.md index b778fb9a7ca..ea8ef101851 100644 --- a/transports/wasm-ext/CHANGELOG.md +++ b/transports/wasm-ext/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.37.0 [unreleased] +# 0.37.0 - Update to `libp2p-core` `v0.37.0`. diff --git a/transports/websocket/CHANGELOG.md b/transports/websocket/CHANGELOG.md index 10fcf02e183..dde9105483c 100644 --- a/transports/websocket/CHANGELOG.md +++ b/transports/websocket/CHANGELOG.md @@ -1,4 +1,4 @@ -# 0.39.0 [unreleased] +# 0.39.0 - Update to `libp2p-core` `v0.37.0`. From 6732482237696070db152dabc3fc2d925be23109 Mon Sep 17 00:00:00 2001 From: Max Inden Date: Fri, 14 Oct 2022 16:55:50 +0100 Subject: [PATCH 181/218] transports/dns/: Don't feature flag std::io import (#3027) Functions like `parse_dnsaddr_txt` depend on the `std::io` import. Given that the function is not feature flagged, compilation without features fails. --- transports/dns/src/lib.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/transports/dns/src/lib.rs b/transports/dns/src/lib.rs index 7f76a378990..1b9bb199200 100644 --- a/transports/dns/src/lib.rs +++ b/transports/dns/src/lib.rs @@ -65,7 +65,6 @@ use libp2p_core::{ }; use parking_lot::Mutex; use smallvec::SmallVec; -#[cfg(any(feature = "async-std", feature = "tokio"))] use std::io; use std::{ convert::TryFrom, From 1da27326ce04eb7ddffa835815ea8409396c7c61 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Sat, 15 Oct 2022 15:38:53 +1100 Subject: [PATCH 182/218] .github/workflows: Don't allow concurrent workflow runs (#3000) --- .github/workflows/cargo-deny-pr.yml | 6 ++++ .github/workflows/ci.yml | 46 +++-------------------------- .github/workflows/interop-test.yml | 9 ++++-- 3 files changed, 17 insertions(+), 44 deletions(-) diff --git a/.github/workflows/cargo-deny-pr.yml b/.github/workflows/cargo-deny-pr.yml index 16b16d16a65..c9c0e7d447b 100644 --- a/.github/workflows/cargo-deny-pr.yml +++ b/.github/workflows/cargo-deny-pr.yml @@ -1,4 +1,5 @@ name: cargo deny + on: push: paths: @@ -6,6 +7,11 @@ on: pull_request: paths: - '**/Cargo.toml' + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true + jobs: cargo-deny: runs-on: ubuntu-latest diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index dae8ffa0990..b0daf6bb863 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -6,6 +6,10 @@ on: branches: - master +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true + jobs: test-desktop: name: Build and test @@ -18,12 +22,6 @@ jobs: "--benches --all-features", ] steps: - - - name: Cancel Previous Runs - uses: styfle/cancel-workflow-action@bb6001c4ea612bf59c3abfc4756fbceee4f870c7 # 0.10.0 - with: - access_token: ${{ github.token }} - - name: Install Protoc uses: arduino/setup-protoc@v1 @@ -53,12 +51,6 @@ jobs: run: shell: bash steps: - - - name: Cancel Previous Runs - uses: styfle/cancel-workflow-action@bb6001c4ea612bf59c3abfc4756fbceee4f870c7 # 0.10.0 - with: - access_token: ${{ github.token }} - - name: Install Protoc uses: arduino/setup-protoc@v1 @@ -91,12 +83,6 @@ jobs: name: Check rustdoc intra-doc links runs-on: ubuntu-latest steps: - - - name: Cancel Previous Runs - uses: styfle/cancel-workflow-action@bb6001c4ea612bf59c3abfc4756fbceee4f870c7 # 0.10.0 - with: - access_token: ${{ github.token }} - - name: Install Protoc uses: arduino/setup-protoc@v1 @@ -116,12 +102,6 @@ jobs: check-clippy: runs-on: ubuntu-latest steps: - - - name: Cancel Previous Runs - uses: styfle/cancel-workflow-action@bb6001c4ea612bf59c3abfc4756fbceee4f870c7 # 0.10.0 - with: - access_token: ${{ github.token }} - - name: Install Protoc uses: arduino/setup-protoc@v1 @@ -145,12 +125,6 @@ jobs: name: Integration tests runs-on: ubuntu-latest steps: - - - name: Cancel Previous Runs - uses: styfle/cancel-workflow-action@bb6001c4ea612bf59c3abfc4756fbceee4f870c7 # 0.10.0 - with: - access_token: ${{ github.token }} - - name: Install Protoc uses: arduino/setup-protoc@v1 @@ -170,12 +144,6 @@ jobs: rustfmt: runs-on: ubuntu-latest steps: - - - name: Cancel Previous Runs - uses: styfle/cancel-workflow-action@bb6001c4ea612bf59c3abfc4756fbceee4f870c7 # 0.10.0 - with: - access_token: ${{ github.token }} - - uses: actions/checkout@v3 - uses: actions-rs/toolchain@16499b5e05bf2e26879000db0c1d13f7e13fa3af # v1.0.7 @@ -191,12 +159,6 @@ jobs: manifest_lint: runs-on: ubuntu-latest steps: - - - name: Cancel Previous Runs - uses: styfle/cancel-workflow-action@bb6001c4ea612bf59c3abfc4756fbceee4f870c7 # 0.10.0 - with: - access_token: ${{ github.token }} - - uses: actions/checkout@v3 - uses: actions-rs/toolchain@16499b5e05bf2e26879000db0c1d13f7e13fa3af # v1.0.7 diff --git a/.github/workflows/interop-test.yml b/.github/workflows/interop-test.yml index 9413cce7e95..de5949bd2b7 100644 --- a/.github/workflows/interop-test.yml +++ b/.github/workflows/interop-test.yml @@ -1,9 +1,14 @@ +name: Interoperability Testing + on: pull_request: push: branches: - master -name: Interoperability Testing + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true jobs: # NOTE: during a pull request run, github creates a merge commit referenced in `github.sha` @@ -24,4 +29,4 @@ jobs: composition_file: "ping/_compositions/go-rust-interop-latest.toml" custom_git_target: github.com/${{ github.event.pull_request.head.repo.full_name || github.event.repository.full_name }} custom_git_reference: ${{ github.event.pull_request.head.sha || github.sha }} - custom_interop_target: rust \ No newline at end of file + custom_interop_target: rust From 0fc4e64cf4d1de2fda6c9a426f38def85d629893 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 17 Oct 2022 11:23:13 +1100 Subject: [PATCH 183/218] muxers: Add test harness for `StreamMuxer` implementations (#2952) --- Cargo.toml | 1 + muxers/mplex/Cargo.toml | 3 +- muxers/mplex/tests/async_write.rs | 91 -------- muxers/mplex/tests/compliance.rs | 28 +++ muxers/mplex/tests/two_peers.rs | 213 ------------------ muxers/test-harness/Cargo.toml | 14 ++ muxers/test-harness/src/lib.rs | 348 ++++++++++++++++++++++++++++++ muxers/yamux/Cargo.toml | 4 + muxers/yamux/tests/compliance.rs | 29 +++ 9 files changed, 426 insertions(+), 305 deletions(-) delete mode 100644 muxers/mplex/tests/async_write.rs create mode 100644 muxers/mplex/tests/compliance.rs delete mode 100644 muxers/mplex/tests/two_peers.rs create mode 100644 muxers/test-harness/Cargo.toml create mode 100644 muxers/test-harness/src/lib.rs create mode 100644 muxers/yamux/tests/compliance.rs diff --git a/Cargo.toml b/Cargo.toml index ba3d653b546..cbfca9f108b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -155,6 +155,7 @@ members = [ "misc/quickcheck-ext", "muxers/mplex", "muxers/yamux", + "muxers/test-harness", "protocols/dcutr", "protocols/autonat", "protocols/floodsub", diff --git a/muxers/mplex/Cargo.toml b/muxers/mplex/Cargo.toml index 0e38ab84018..e5fb1c305ef 100644 --- a/muxers/mplex/Cargo.toml +++ b/muxers/mplex/Cargo.toml @@ -23,11 +23,12 @@ smallvec = "1.6.1" unsigned-varint = { version = "0.7", features = ["asynchronous_codec"] } [dev-dependencies] -async-std = "1.7.0" +async-std = { version = "1.7.0", features = ["attributes"] } criterion = "0.4" env_logger = "0.9" futures = "0.3" libp2p = { path = "../..", features = ["full"] } +libp2p-muxer-test-harness = { path = "../test-harness" } quickcheck = { package = "quickcheck-ext", path = "../../misc/quickcheck-ext" } [[bench]] diff --git a/muxers/mplex/tests/async_write.rs b/muxers/mplex/tests/async_write.rs deleted file mode 100644 index d4252ad20e4..00000000000 --- a/muxers/mplex/tests/async_write.rs +++ /dev/null @@ -1,91 +0,0 @@ -// Copyright 2019 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 futures::future::poll_fn; -use futures::{channel::oneshot, prelude::*}; -use libp2p::core::muxing::StreamMuxerExt; -use libp2p::core::{upgrade, Transport}; -use libp2p::tcp::TcpTransport; - -#[test] -fn async_write() { - // Tests that `AsyncWrite::close` implies flush. - - let (tx, rx) = oneshot::channel(); - - let bg_thread = async_std::task::spawn(async move { - let mplex = libp2p::mplex::MplexConfig::new(); - - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)) - .boxed(); - - transport - .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) - .unwrap(); - - let addr = transport - .next() - .await - .expect("some event") - .into_new_address() - .expect("listen address"); - - tx.send(addr).unwrap(); - - let mut client = transport - .next() - .await - .expect("some event") - .into_incoming() - .unwrap() - .0 - .await - .unwrap(); - - // Just calling `poll_outbound` without `poll` is fine here because mplex makes progress through all `poll_` functions. It is hacky though. - let mut outbound = poll_fn(|cx| client.poll_outbound_unpin(cx)) - .await - .expect("unexpected error"); - - let mut buf = Vec::new(); - outbound.read_to_end(&mut buf).await.unwrap(); - assert_eq!(buf, b"hello world"); - }); - - async_std::task::block_on(async { - let mplex = libp2p::mplex::MplexConfig::new(); - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)); - - let mut client = transport.dial(rx.await.unwrap()).unwrap().await.unwrap(); - - // Just calling `poll_inbound` without `poll` is fine here because mplex makes progress through all `poll_` functions. It is hacky though. - let mut inbound = poll_fn(|cx| client.poll_inbound_unpin(cx)) - .await - .expect("unexpected error"); - inbound.write_all(b"hello world").await.unwrap(); - - // The test consists in making sure that this flushes the substream. - inbound.close().await.unwrap(); - - bg_thread.await; - }); -} diff --git a/muxers/mplex/tests/compliance.rs b/muxers/mplex/tests/compliance.rs new file mode 100644 index 00000000000..849ff9e0c20 --- /dev/null +++ b/muxers/mplex/tests/compliance.rs @@ -0,0 +1,28 @@ +use libp2p_mplex::MplexConfig; + +#[async_std::test] +async fn close_implies_flush() { + let (alice, bob) = + libp2p_muxer_test_harness::connected_muxers_on_memory_transport::() + .await; + + libp2p_muxer_test_harness::close_implies_flush(alice, bob).await; +} + +#[async_std::test] +async fn dialer_can_receive() { + let (alice, bob) = + libp2p_muxer_test_harness::connected_muxers_on_memory_transport::() + .await; + + libp2p_muxer_test_harness::dialer_can_receive(alice, bob).await; +} + +#[async_std::test] +async fn read_after_close() { + let (alice, bob) = + libp2p_muxer_test_harness::connected_muxers_on_memory_transport::() + .await; + + libp2p_muxer_test_harness::read_after_close(alice, bob).await; +} diff --git a/muxers/mplex/tests/two_peers.rs b/muxers/mplex/tests/two_peers.rs deleted file mode 100644 index 70a10e8a0f7..00000000000 --- a/muxers/mplex/tests/two_peers.rs +++ /dev/null @@ -1,213 +0,0 @@ -// Copyright 2018 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 futures::future::poll_fn; -use futures::{channel::oneshot, prelude::*}; -use libp2p::core::muxing::StreamMuxerExt; -use libp2p::core::{upgrade, Transport}; -use libp2p::tcp::TcpTransport; - -#[test] -fn client_to_server_outbound() { - // Simulate a client sending a message to a server through a multiplex upgrade. - - let (tx, rx) = oneshot::channel(); - - let bg_thread = async_std::task::spawn(async move { - let mplex = libp2p_mplex::MplexConfig::new(); - - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)) - .boxed(); - - transport - .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) - .unwrap(); - - let addr = transport - .next() - .await - .expect("some event") - .into_new_address() - .expect("listen address"); - - tx.send(addr).unwrap(); - - let mut client = transport - .next() - .await - .expect("some event") - .into_incoming() - .unwrap() - .0 - .await - .unwrap(); - - // Just calling `poll_outbound` without `poll` is fine here because mplex makes progress through all `poll_` functions. It is hacky though. - let mut outbound = poll_fn(|cx| client.poll_outbound_unpin(cx)) - .await - .expect("unexpected error"); - - let mut buf = Vec::new(); - outbound.read_to_end(&mut buf).await.unwrap(); - assert_eq!(buf, b"hello world"); - }); - - async_std::task::block_on(async { - let mplex = libp2p_mplex::MplexConfig::new(); - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)) - .boxed(); - - let mut client = transport.dial(rx.await.unwrap()).unwrap().await.unwrap(); - // Just calling `poll_inbound` without `poll` is fine here because mplex makes progress through all `poll_` functions. It is hacky though. - let mut inbound = poll_fn(|cx| client.poll_inbound_unpin(cx)) - .await - .expect("unexpected error"); - inbound.write_all(b"hello world").await.unwrap(); - inbound.close().await.unwrap(); - - bg_thread.await; - }); -} - -#[test] -fn client_to_server_inbound() { - // Simulate a client sending a message to a server through a multiplex upgrade. - - let (tx, rx) = oneshot::channel(); - - let bg_thread = async_std::task::spawn(async move { - let mplex = libp2p_mplex::MplexConfig::new(); - - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)) - .boxed(); - - transport - .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) - .unwrap(); - - let addr = transport - .next() - .await - .expect("some event") - .into_new_address() - .expect("listen address"); - - tx.send(addr).unwrap(); - - let mut client = transport - .next() - .await - .expect("some event") - .into_incoming() - .unwrap() - .0 - .await - .unwrap(); - - // Just calling `poll_inbound` without `poll` is fine here because mplex makes progress through all `poll_` functions. It is hacky though. - let mut inbound = poll_fn(|cx| client.poll_inbound_unpin(cx)) - .await - .expect("unexpected error"); - - let mut buf = Vec::new(); - inbound.read_to_end(&mut buf).await.unwrap(); - assert_eq!(buf, b"hello world"); - }); - - async_std::task::block_on(async { - let mplex = libp2p_mplex::MplexConfig::new(); - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)) - .boxed(); - - let mut client = transport.dial(rx.await.unwrap()).unwrap().await.unwrap(); - - // Just calling `poll_outbound` without `poll` is fine here because mplex makes progress through all `poll_` functions. It is hacky though. - let mut outbound = poll_fn(|cx| client.poll_outbound_unpin(cx)) - .await - .expect("unexpected error"); - outbound.write_all(b"hello world").await.unwrap(); - outbound.close().await.unwrap(); - - bg_thread.await; - }); -} - -#[test] -fn protocol_not_match() { - let (tx, rx) = oneshot::channel(); - - let _bg_thread = async_std::task::spawn(async move { - let mplex = libp2p_mplex::MplexConfig::new(); - - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)) - .boxed(); - - transport - .listen_on("/ip4/127.0.0.1/tcp/0".parse().unwrap()) - .unwrap(); - - let addr = transport - .next() - .await - .expect("some event") - .into_new_address() - .expect("listen address"); - - tx.send(addr).unwrap(); - - let mut client = transport - .next() - .await - .expect("some event") - .into_incoming() - .unwrap() - .0 - .await - .unwrap(); - - // Just calling `poll_outbound` without `poll` is fine here because mplex makes progress through all `poll_` functions. It is hacky though. - let mut outbound = poll_fn(|cx| client.poll_outbound_unpin(cx)) - .await - .expect("unexpected error"); - - let mut buf = Vec::new(); - outbound.read_to_end(&mut buf).await.unwrap(); - assert_eq!(buf, b"hello world"); - }); - - async_std::task::block_on(async { - // Make sure they do not connect when protocols do not match - let mut mplex = libp2p_mplex::MplexConfig::new(); - mplex.set_protocol_name(b"/mplextest/1.0.0"); - let mut transport = TcpTransport::default() - .and_then(move |c, e| upgrade::apply(c, mplex, e, upgrade::Version::V1)) - .boxed(); - - assert!( - transport.dial(rx.await.unwrap()).unwrap().await.is_err(), - "Dialing should fail here as protocols do not match" - ); - }); -} diff --git a/muxers/test-harness/Cargo.toml b/muxers/test-harness/Cargo.toml new file mode 100644 index 00000000000..43b75132169 --- /dev/null +++ b/muxers/test-harness/Cargo.toml @@ -0,0 +1,14 @@ +[package] +name = "libp2p-muxer-test-harness" +version = "0.1.0" +edition = "2021" +publish = false +license = "MIT" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +libp2p-core = { path = "../../core" } +futures = "0.3.24" +log = "0.4" +futures-timer = "3.0.2" diff --git a/muxers/test-harness/src/lib.rs b/muxers/test-harness/src/lib.rs new file mode 100644 index 00000000000..65dedf581ba --- /dev/null +++ b/muxers/test-harness/src/lib.rs @@ -0,0 +1,348 @@ +use crate::future::{BoxFuture, Either, FutureExt}; +use futures::{future, AsyncRead, AsyncWrite}; +use futures::{AsyncReadExt, Stream}; +use futures::{AsyncWriteExt, StreamExt}; +use libp2p_core::multiaddr::Protocol; +use libp2p_core::muxing::StreamMuxerExt; +use libp2p_core::transport::memory::Channel; +use libp2p_core::transport::MemoryTransport; +use libp2p_core::{ + upgrade, InboundUpgrade, Negotiated, OutboundUpgrade, StreamMuxer, Transport, UpgradeInfo, +}; +use std::future::Future; +use std::pin::Pin; +use std::task::{Context, Poll}; +use std::time::Duration; +use std::{fmt, mem}; + +pub async fn connected_muxers_on_memory_transport() -> (M, M) +where + MC: InboundUpgrade>>, Error = E, Output = M> + + OutboundUpgrade>>, Error = E, Output = M> + + Send + + 'static + + Default, + ::Info: Send, + <::InfoIter as IntoIterator>::IntoIter: Send, + >>>>::Future: Send, + >>>>::Future: Send, + E: std::error::Error + Send + Sync + 'static, +{ + let mut alice = MemoryTransport::default() + .and_then(move |c, e| upgrade::apply(c, MC::default(), e, upgrade::Version::V1)) + .boxed(); + let mut bob = MemoryTransport::default() + .and_then(move |c, e| upgrade::apply(c, MC::default(), e, upgrade::Version::V1)) + .boxed(); + + alice.listen_on(Protocol::Memory(0).into()).unwrap(); + let listen_address = alice.next().await.unwrap().into_new_address().unwrap(); + + futures::future::join( + async { + alice + .next() + .await + .unwrap() + .into_incoming() + .unwrap() + .0 + .await + .unwrap() + }, + async { bob.dial(listen_address).unwrap().await.unwrap() }, + ) + .await +} + +/// Verifies that Alice can send a message and immediately close the stream afterwards and Bob can use `read_to_end` to read the entire message. +pub async fn close_implies_flush(alice: A, bob: B) +where + A: StreamMuxer + Unpin, + B: StreamMuxer + Unpin, + S: AsyncRead + AsyncWrite + Send + Unpin + 'static, + E: fmt::Debug, +{ + run_commutative( + alice, + bob, + |mut stream| async move { + stream.write_all(b"PING").await.unwrap(); + stream.close().await.unwrap(); + }, + |mut stream| async move { + let mut buf = Vec::new(); + stream.read_to_end(&mut buf).await.unwrap(); + + assert_eq!(buf, b"PING"); + }, + ) + .await; +} + +/// Verifies that the dialer of a substream can receive a message. +pub async fn dialer_can_receive(alice: A, bob: B) +where + A: StreamMuxer + Unpin, + B: StreamMuxer + Unpin, + S: AsyncRead + AsyncWrite + Send + Unpin + 'static, + E: fmt::Debug, +{ + run_commutative( + alice, + bob, + |mut stream| async move { + let mut buf = Vec::new(); + stream.read_to_end(&mut buf).await.unwrap(); + + assert_eq!(buf, b"PING"); + }, + |mut stream| async move { + stream.write_all(b"PING").await.unwrap(); + stream.close().await.unwrap(); + }, + ) + .await; +} + +/// Verifies that we can "half-close" a substream. +pub async fn read_after_close(alice: A, bob: B) +where + A: StreamMuxer + Unpin, + B: StreamMuxer + Unpin, + S: AsyncRead + AsyncWrite + Send + Unpin + 'static, + E: fmt::Debug, +{ + run_commutative( + alice, + bob, + |mut stream| async move { + stream.write_all(b"PING").await.unwrap(); + stream.close().await.unwrap(); + + let mut buf = Vec::new(); + stream.read_to_end(&mut buf).await.unwrap(); + + assert_eq!(buf, b"PONG"); + }, + |mut stream| async move { + let mut buf = [0u8; 4]; + stream.read_exact(&mut buf).await.unwrap(); + + assert_eq!(&buf, b"PING"); + + stream.write_all(b"PONG").await.unwrap(); + stream.close().await.unwrap(); + }, + ) + .await; +} + +/// Runs the given protocol between the two parties, ensuring commutativity, i.e. either party can be the dialer and listener. +async fn run_commutative( + mut alice: A, + mut bob: B, + alice_proto: impl Fn(S) -> F1 + Clone + 'static, + bob_proto: impl Fn(S) -> F2 + Clone + 'static, +) where + A: StreamMuxer + Unpin, + B: StreamMuxer + Unpin, + S: AsyncRead + AsyncWrite + Send + Unpin + 'static, + E: fmt::Debug, + F1: Future + Send + 'static, + F2: Future + Send + 'static, +{ + run(&mut alice, &mut bob, alice_proto.clone(), bob_proto.clone()).await; + run(&mut bob, &mut alice, alice_proto, bob_proto).await; +} + +/// Runs a given protocol between the two parties. +/// +/// The first party will open a new substream and the second party will wait for this. +/// The [`StreamMuxer`] is polled until both parties have completed the protocol to ensure that the underlying connection can make progress at all times. +async fn run( + dialer: &mut A, + listener: &mut B, + alice_proto: impl Fn(S) -> F1 + 'static, + bob_proto: impl Fn(S) -> F2 + 'static, +) where + A: StreamMuxer + Unpin, + B: StreamMuxer + Unpin, + S: AsyncRead + AsyncWrite + Send + Unpin + 'static, + E: fmt::Debug, + F1: Future + Send + 'static, + F2: Future + Send + 'static, +{ + let mut dialer = Harness::OutboundSetup { + muxer: dialer, + proto_fn: Box::new(move |s| alice_proto(s).boxed()), + }; + let mut listener = Harness::InboundSetup { + muxer: listener, + proto_fn: Box::new(move |s| bob_proto(s).boxed()), + }; + + let mut dialer_complete = false; + let mut listener_complete = false; + + loop { + match futures::future::select(dialer.next(), listener.next()).await { + Either::Left((Some(Event::SetupComplete), _)) => { + log::info!("Dialer opened outbound stream"); + } + Either::Left((Some(Event::ProtocolComplete), _)) => { + log::info!("Dialer completed protocol"); + dialer_complete = true + } + Either::Left((Some(Event::Timeout), _)) => { + panic!("Dialer protocol timed out"); + } + Either::Right((Some(Event::SetupComplete), _)) => { + log::info!("Listener received inbound stream"); + } + Either::Right((Some(Event::ProtocolComplete), _)) => { + log::info!("Listener completed protocol"); + listener_complete = true + } + Either::Right((Some(Event::Timeout), _)) => { + panic!("Listener protocol timed out"); + } + _ => unreachable!(), + } + + if dialer_complete && listener_complete { + break; + } + } +} + +enum Harness<'m, M> +where + M: StreamMuxer, +{ + InboundSetup { + muxer: &'m mut M, + proto_fn: Box BoxFuture<'static, ()>>, + }, + OutboundSetup { + muxer: &'m mut M, + proto_fn: Box BoxFuture<'static, ()>>, + }, + Running { + muxer: &'m mut M, + timeout: futures_timer::Delay, + proto: BoxFuture<'static, ()>, + }, + Complete { + muxer: &'m mut M, + }, + Poisoned, +} + +enum Event { + SetupComplete, + Timeout, + ProtocolComplete, +} + +impl<'m, M> Stream for Harness<'m, M> +where + M: StreamMuxer + Unpin, +{ + type Item = Event; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + + loop { + match mem::replace(this, Self::Poisoned) { + Harness::InboundSetup { muxer, proto_fn } => { + if let Poll::Ready(stream) = muxer.poll_inbound_unpin(cx) { + *this = Harness::Running { + muxer, + timeout: futures_timer::Delay::new(Duration::from_secs(10)), + proto: proto_fn(stream.unwrap()), + }; + return Poll::Ready(Some(Event::SetupComplete)); + } + + if let Poll::Ready(event) = muxer.poll_unpin(cx) { + event.unwrap(); + + *this = Harness::InboundSetup { muxer, proto_fn }; + continue; + } + + *this = Harness::InboundSetup { muxer, proto_fn }; + return Poll::Pending; + } + Harness::OutboundSetup { muxer, proto_fn } => { + if let Poll::Ready(stream) = muxer.poll_outbound_unpin(cx) { + *this = Harness::Running { + muxer, + timeout: futures_timer::Delay::new(Duration::from_secs(10)), + proto: proto_fn(stream.unwrap()), + }; + return Poll::Ready(Some(Event::SetupComplete)); + } + + if let Poll::Ready(event) = muxer.poll_unpin(cx) { + event.unwrap(); + + *this = Harness::OutboundSetup { muxer, proto_fn }; + continue; + } + + *this = Harness::OutboundSetup { muxer, proto_fn }; + return Poll::Pending; + } + Harness::Running { + muxer, + mut proto, + mut timeout, + } => { + if let Poll::Ready(event) = muxer.poll_unpin(cx) { + event.unwrap(); + + *this = Harness::Running { + muxer, + proto, + timeout, + }; + continue; + } + + if let Poll::Ready(()) = proto.poll_unpin(cx) { + *this = Harness::Complete { muxer }; + return Poll::Ready(Some(Event::ProtocolComplete)); + } + + if let Poll::Ready(()) = timeout.poll_unpin(cx) { + return Poll::Ready(Some(Event::Timeout)); + } + + *this = Harness::Running { + muxer, + proto, + timeout, + }; + return Poll::Pending; + } + Harness::Complete { muxer } => { + if let Poll::Ready(event) = muxer.poll_unpin(cx) { + event.unwrap(); + + *this = Harness::Complete { muxer }; + continue; + } + + *this = Harness::Complete { muxer }; + return Poll::Pending; + } + Harness::Poisoned => { + unreachable!() + } + } + } + } +} diff --git a/muxers/yamux/Cargo.toml b/muxers/yamux/Cargo.toml index 10800a78c3c..35992e29efc 100644 --- a/muxers/yamux/Cargo.toml +++ b/muxers/yamux/Cargo.toml @@ -17,3 +17,7 @@ parking_lot = "0.12" thiserror = "1.0" yamux = "0.10.0" log = "0.4" + +[dev-dependencies] +async-std = { version = "1.7.0", features = ["attributes"] } +libp2p-muxer-test-harness = { path = "../test-harness" } diff --git a/muxers/yamux/tests/compliance.rs b/muxers/yamux/tests/compliance.rs new file mode 100644 index 00000000000..51cbea387d2 --- /dev/null +++ b/muxers/yamux/tests/compliance.rs @@ -0,0 +1,29 @@ +use libp2p_yamux::YamuxConfig; + +#[async_std::test] +async fn close_implies_flush() { + let (alice, bob) = + libp2p_muxer_test_harness::connected_muxers_on_memory_transport::() + .await; + + libp2p_muxer_test_harness::close_implies_flush(alice, bob).await; +} + +#[async_std::test] +#[ignore] // Hangs forever, is this a harness bug? It passes if we try to write to the stream. +async fn dialer_can_receive() { + let (alice, bob) = + libp2p_muxer_test_harness::connected_muxers_on_memory_transport::() + .await; + + libp2p_muxer_test_harness::dialer_can_receive(alice, bob).await; +} + +#[async_std::test] +async fn read_after_close() { + let (alice, bob) = + libp2p_muxer_test_harness::connected_muxers_on_memory_transport::() + .await; + + libp2p_muxer_test_harness::read_after_close(alice, bob).await; +} From b51f28055dc1f358967b4733596221dbdbfc6f94 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 17 Oct 2022 11:53:14 +1100 Subject: [PATCH 184/218] Don't rename crates in manifest With the new `dep:` syntax, this is no longer necessary. --- transports/quic/Cargo.toml | 12 ++++++------ transports/quic/src/provider/async_std.rs | 2 +- transports/quic/src/provider/tokio.rs | 4 ++-- transports/quic/src/transport.rs | 4 ---- transports/quic/tests/smoke.rs | 5 ----- 5 files changed, 9 insertions(+), 18 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index f4d4624b460..662f08f5555 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -8,7 +8,7 @@ repository = "https://github.com/libp2p/rust-libp2p" license = "MIT" [dependencies] -async-std-crate = { package = "async-std", version = "1.12.0", default-features = false, optional = true } +async-std = { version = "1.12.0", default-features = false, optional = true } async-trait = "0.1.50" bytes = "1.2.1" futures = "0.3.15" @@ -23,14 +23,14 @@ rcgen = "0.9.2" ring = "0.16.20" rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } thiserror = "1.0.26" -tokio-crate = { package = "tokio", version = "1.21.1", default-features = false, features = ["net", "rt"], optional = true } +tokio = { version = "1.21.1", default-features = false, features = ["net", "rt"], optional = true } webpki = "0.22.0" x509-parser = "0.13.0" yasna = "0.5.0" [features] -tokio = ["dep:tokio-crate"] -async-std = ["dep:async-std-crate"] +tokio = ["dep:tokio"] +async-std = ["dep:async-std"] # Passing arguments to the docsrs builder in order to properly document cfg's. # More information: https://docs.rs/about/builds#cross-compiling @@ -40,9 +40,9 @@ rustdoc-args = ["--cfg", "docsrs"] rustc-args = ["--cfg", "docsrs"] [dev-dependencies] -async-std-crate = { package = "async-std", version = "1.12.0", features = ["attributes"] } +async-std = { version = "1.12.0", features = ["attributes"] } libp2p = { path = "../..", features = ["request-response"] } env_logger = "0.9.0" rand = "0.8.4" -tokio-crate = { package = "tokio", version = "1.21.1", features = ["macros", "rt-multi-thread"] } +tokio = { version = "1.21.1", features = ["macros", "rt-multi-thread"] } quickcheck = "1" diff --git a/transports/quic/src/provider/async_std.rs b/transports/quic/src/provider/async_std.rs index 34bfd22ac0e..eef4f5dc740 100644 --- a/transports/quic/src/provider/async_std.rs +++ b/transports/quic/src/provider/async_std.rs @@ -26,7 +26,7 @@ use std::{ task::{Context, Poll}, }; -use async_std_crate::{net::UdpSocket, task::spawn}; +use async_std::{net::UdpSocket, task::spawn}; use futures::{future::BoxFuture, ready, Future, FutureExt, Stream, StreamExt}; use crate::GenTransport; diff --git a/transports/quic/src/provider/tokio.rs b/transports/quic/src/provider/tokio.rs index 09bda1e9881..b7374e8c9d6 100644 --- a/transports/quic/src/provider/tokio.rs +++ b/transports/quic/src/provider/tokio.rs @@ -25,7 +25,7 @@ use std::{ }; use futures::{ready, Future}; -use tokio_crate::{io::ReadBuf, net::UdpSocket}; +use tokio::{io::ReadBuf, net::UdpSocket}; use x509_parser::nom::AsBytes; use crate::GenTransport; @@ -80,6 +80,6 @@ impl ProviderTrait for Provider { } fn spawn(future: impl Future + Send + 'static) { - tokio_crate::spawn(future); + tokio::spawn(future); } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index cf4f2e41d4a..681281ea418 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -565,13 +565,9 @@ fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { #[cfg(test)] #[cfg(any(feature = "async-std", feature = "tokio"))] mod test { - #[cfg(feature = "async-std")] - use async_std_crate as async_std; use futures::future::poll_fn; use futures_timer::Delay; use std::net::{IpAddr, Ipv4Addr, Ipv6Addr}; - #[cfg(feature = "tokio")] - use tokio_crate as tokio; use super::*; diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 60da17e4add..c0bfa8e88ce 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -24,11 +24,6 @@ use std::num::NonZeroU8; use std::time::Duration; use std::{io, iter}; -#[cfg(feature = "async-std")] -use async_std_crate as async_std; -#[cfg(feature = "tokio")] -use tokio_crate as tokio; - fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } From 4cad0803d270bf3e93104ef723e5b94016c475f9 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 20 Oct 2022 15:55:11 +0200 Subject: [PATCH 185/218] quic: add config max_data and max_stream_data --- transports/quic/src/endpoint.rs | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 6ae0c1f44d0..64081a87211 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -36,6 +36,7 @@ use futures::{ prelude::*, ready, }; +use quinn_proto::VarInt; use std::{ collections::HashMap, net::{Ipv4Addr, Ipv6Addr, SocketAddr}, @@ -52,8 +53,8 @@ pub struct Config { /// Timeout for the initial handshake when establishing a connection. /// The actual timeout is the minimum of this an the [`Config::max_idle_timeout`]. pub handshake_timeout: Duration, - /// Maximum duration of inactivity to accept before timing out the connection. - pub max_idle_timeout: Duration, + /// Maximum duration of inactivity in ms to accept before timing out the connection. + pub max_idle_timeout: u32, /// Period of inactivity before sending a keep-alive packet. /// Must be set lower than the idle_timeout of both /// peers to be effective. @@ -65,6 +66,13 @@ pub struct Config { /// concurrently by the remote peer. pub max_concurrent_stream_limit: u32, + /// Max unacknowledged data in bytes that may be send on a single stream. + pub max_stream_data: u32, + + /// Max unacknowledged data in bytes that may be send in total on all streams + /// of a connection. + pub max_connection_data: u32, + client_tls_config: Arc, server_tls_config: Arc, } @@ -78,9 +86,13 @@ impl Config { client_tls_config, server_tls_config, handshake_timeout: Duration::from_secs(5), - max_idle_timeout: Duration::from_secs(30), + max_idle_timeout: 30 * 1000, max_concurrent_stream_limit: 256, keep_alive_interval: Duration::from_secs(15), + max_connection_data: 15_000_000, + + // Ensure that one stream is not consuming the whole connection. + max_stream_data: 10_000_000, } } } @@ -101,6 +113,8 @@ impl From for QuinnConfig { max_idle_timeout, max_concurrent_stream_limit, keep_alive_interval, + max_connection_data, + max_stream_data, handshake_timeout: _, } = config; let mut transport = quinn_proto::TransportConfig::default(); @@ -108,8 +122,10 @@ impl From for QuinnConfig { transport.max_concurrent_bidi_streams(max_concurrent_stream_limit.into()); transport.datagram_receive_buffer_size(None); transport.keep_alive_interval(Some(keep_alive_interval)); - transport.max_idle_timeout(Some(max_idle_timeout.try_into().expect("is < 2^62"))); + transport.max_idle_timeout(Some(VarInt::from_u32(max_idle_timeout).into())); transport.allow_spin(false); + transport.stream_receive_window(max_stream_data.into()); + transport.receive_window(max_connection_data.into()); let transport = Arc::new(transport); let mut server_config = quinn_proto::ServerConfig::with_crypto(server_tls_config); From 445a2a4102f38b80916d0d5667305668d282e419 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 20 Oct 2022 16:27:29 +0200 Subject: [PATCH 186/218] quic: assert that we only drop datagram events --- transports/quic/src/endpoint.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 64081a87211..6282f319455 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -506,6 +506,11 @@ impl EndpointDriver

{ }; match event { quinn_proto::DatagramEvent::ConnectionEvent(event) => { + // `event` has type `quinn_proto::ConnectionEvent`, which has multiple + // variants. However, `quinn_proto::Endpoint::handle` only ever returns + // `ConnectionEvent::Datagram`. + debug_assert!(format!("{:?}", event).contains("Datagram")); + // Redirect the datagram to its connection. if let Some(sender) = self.alive_connections.get_mut(&connec_id) { match sender.try_send(event) { From 70cc694b4c9b6701527e7dbc78c623073161c353 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 20 Oct 2022 16:29:43 +0200 Subject: [PATCH 187/218] quic/connection: make internal fns private --- transports/quic/src/connection.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 98855c44859..bcd38ac50fe 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -73,7 +73,7 @@ impl Connection { /// /// This function assumes that the [`quinn_proto::Connection`] is completely fresh and none of /// its methods has ever been called. Failure to comply might lead to logic errors and panics. - pub fn from_quinn_connection( + pub(crate) fn from_quinn_connection( endpoint_channel: endpoint::Channel, connection: quinn_proto::Connection, connection_id: quinn_proto::ConnectionHandle, @@ -98,16 +98,16 @@ impl Connection { } /// The address that the local socket is bound to. - pub fn local_addr(&self) -> &SocketAddr { + pub(crate) fn local_addr(&self) -> &SocketAddr { self.endpoint_channel.socket_addr() } /// Returns the address of the node we're connected to. - pub fn remote_addr(&self) -> SocketAddr { + pub(crate) fn remote_addr(&self) -> SocketAddr { self.state.lock().connection.remote_address() } - pub fn peer_identity(&self) -> Option> { + fn peer_identity(&self) -> Option> { self.state .lock() .connection @@ -116,7 +116,7 @@ impl Connection { } /// Polls the connection for an event that happened on it. - pub fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { + fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { let mut inner = self.state.lock(); loop { match self.from_endpoint.poll_next_unpin(cx) { From a38f0e95afc3edc0e9f1ba4003dac52024911289 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 20 Oct 2022 17:36:26 +0200 Subject: [PATCH 188/218] quic/substream: break read early if buf is empty --- transports/quic/src/connection/substream.rs | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/connection/substream.rs b/transports/quic/src/connection/substream.rs index 9fac101b4e8..d7f2a67bbb6 100644 --- a/transports/quic/src/connection/substream.rs +++ b/transports/quic/src/connection/substream.rs @@ -92,9 +92,15 @@ impl AsyncRead for Substream { let mut bytes = 0; let mut pending = false; loop { + if buf.is_empty() { + // Chunks::next will continue returning `Ok(Some(_))` with an + // empty chunk if there are no bytes left to read, so we break + // early here. + break; + } let chunk = match chunks.next(buf.len()) { - Ok(Some(chunk)) if !chunk.bytes.is_empty() => chunk, - Ok(_) => break, + Ok(Some(chunk)) => chunk, + Ok(None) => break, Err(err @ quinn_proto::ReadError::Reset(_)) => { return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } From 3725f5fbfeda813f90fc746ac624f46d7063b64e Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Thu, 20 Oct 2022 17:48:43 +0200 Subject: [PATCH 189/218] quic/substream: fix comment --- transports/quic/src/connection/substream.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/src/connection/substream.rs b/transports/quic/src/connection/substream.rs index d7f2a67bbb6..1c1e960b86e 100644 --- a/transports/quic/src/connection/substream.rs +++ b/transports/quic/src/connection/substream.rs @@ -94,8 +94,8 @@ impl AsyncRead for Substream { loop { if buf.is_empty() { // Chunks::next will continue returning `Ok(Some(_))` with an - // empty chunk if there are no bytes left to read, so we break - // early here. + // empty chunk if there is no space left in the buffer, so we + // break early here. break; } let chunk = match chunks.next(buf.len()) { From 4b3d35ec25d529b7425bb773ad3a887f720c4c69 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 23 Oct 2022 01:38:44 +0200 Subject: [PATCH 190/218] quic: continue read from socket if write is busy Don't block the endpoint driver when writing to the udp socket is blocked. Instead continue reading from the socket. However, while writing is blocked don't poll the other parts of the application (transport, connections) for events, to backpressure them. --- transports/quic/src/endpoint.rs | 82 ++++++++++++++++++--------------- 1 file changed, 46 insertions(+), 36 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 6282f319455..feca5c46b57 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -34,7 +34,6 @@ use bytes::BytesMut; use futures::{ channel::{mpsc, oneshot}, prelude::*, - ready, }; use quinn_proto::VarInt; use std::{ @@ -418,7 +417,10 @@ impl EndpointDriver

{ } /// Handle a message sent from either the [`GenTransport`](super::GenTransport) or a [`crate::Connection`]. - fn handle_message(&mut self, to_endpoint: ToEndpoint) -> ControlFlow<()> { + fn handle_message( + &mut self, + to_endpoint: ToEndpoint, + ) -> ControlFlow<(), Option> { match to_endpoint { ToEndpoint::Dial { addr, result } => { // This `"l"` seems necessary because an empty string is an invalid domain @@ -429,7 +431,7 @@ impl EndpointDriver

{ Ok(c) => c, Err(err) => { let _ = result.send(Err(ConnectError::from(err).into())); - return ControlFlow::Continue(()); + return ControlFlow::Continue(None); } }; @@ -452,7 +454,7 @@ impl EndpointDriver

{ } => { let has_key = self.alive_connections.contains_key(&connection_id); if !has_key { - return ControlFlow::Continue(()); + return ControlFlow::Continue(None); } // We "drained" event indicates that the connection no longer exists and // its ID can be reclaimed. @@ -485,10 +487,10 @@ impl EndpointDriver

{ } // Data needs to be sent on the UDP socket. - ToEndpoint::SendUdpPacket(transmit) => self.next_packet_out = Some(transmit), + ToEndpoint::SendUdpPacket(transmit) => return ControlFlow::Continue(Some(transmit)), ToEndpoint::Decoupled => self.handle_decoupling()?, } - ControlFlow::Continue(()) + ControlFlow::Continue(None) } /// Handle datagram received on the socket. @@ -550,15 +552,16 @@ impl EndpointDriver

{ let (tx, rx) = mpsc::channel(16); let connection = Connection::from_quinn_connection(self.channel.clone(), connec, connec_id, rx); - match connection_tx.try_send(connection) { + match connection_tx.start_send(connection) { Ok(()) => { self.alive_connections.insert(connec_id, tx); } Err(e) if e.is_disconnected() => self.handle_decoupling()?, - Err(_) => log::warn!( + Err(e) if e.is_full() => log::warn!( "Dropping new incoming connection {:?} because the channel to the listener is full", connec_id - ) + ), + Err(_) => unreachable!("Error is either `Full` or `Disconnected`."), } } } @@ -581,37 +584,43 @@ impl Future for EndpointDriver

{ type Output = (); fn poll(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { - match ready!(self.provider_socket.poll_send_flush(cx)) { - Ok(_) => {} + match self.provider_socket.poll_send_flush(cx) { + Poll::Ready(Ok(_)) => { + if let Some(transmit) = self.next_packet_out.take() { + self.provider_socket + .start_send(transmit.contents, transmit.destination); + continue; + } + + // The endpoint might request packets to be sent out. This is handled in priority to avoid + // buffering up packets. + if let Some(transmit) = self.endpoint.poll_transmit() { + self.next_packet_out = Some(transmit); + continue; + } + + match self.rx.poll_next_unpin(cx) { + Poll::Ready(Some(to_endpoint)) => match self.handle_message(to_endpoint) { + ControlFlow::Continue(Some(transmit)) => { + self.next_packet_out = Some(transmit); + continue; + } + ControlFlow::Continue(None) => continue, + ControlFlow::Break(()) => break, + }, + Poll::Ready(None) => { + unreachable!("Sender side is never dropped or closed.") + } + Poll::Pending => {} + } + } // Errors on the socket are expected to never happen, and we handle them by simply // printing a log message. The packet gets discarded in case of error, but we are // robust to packet losses and it is consequently not a logic error to proceed with // normal operations. - Err(err) => { - log::error!("Error while sending on QUIC UDP socket: {:?}", err) - } - } - - if let Some(transmit) = self.next_packet_out.take() { - self.provider_socket - .start_send(transmit.contents, transmit.destination); - continue; - } - - // The endpoint might request packets to be sent out. This is handled in priority to avoid - // buffering up packets. - if let Some(transmit) = self.endpoint.poll_transmit() { - self.next_packet_out = Some(transmit); - continue; - } - - match self.rx.poll_next_unpin(cx) { - Poll::Ready(Some(to_endpoint)) => match self.handle_message(to_endpoint) { - ControlFlow::Continue(()) => continue, - ControlFlow::Break(()) => break, - }, - Poll::Ready(None) => { - unreachable!("Sender side is never dropped or closed.") + Poll::Ready(Err(err)) => { + log::error!("Error while sending on QUIC UDP socket: {:?}", err); + continue; } Poll::Pending => {} } @@ -632,6 +641,7 @@ impl Future for EndpointDriver

{ } Poll::Pending => {} } + return Poll::Pending; } From bbe736fe92be5561c7e0e184133671eb5c7b5aca Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sun, 23 Oct 2022 19:53:37 +0200 Subject: [PATCH 191/218] quic/endpoint: unify channel capacity - **Channel where endpoint is sender:** The `EndpointDriver` drops packets if the channel to the connection or transport is full. Set capacity 10 to avoid unnecessary packet drops if the receiver is only very briefly busy but not buffer a large amount of packets if it is blocked longer. - **Channel to endpoint:** Capacity 0 (=1 slot per sender) to back-pressure the rest of the application if the udp socket is busy. --- transports/quic/src/endpoint.rs | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index feca5c46b57..7565138cda8 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -46,6 +46,13 @@ use std::{ }; use x509_parser::nom::AsBytes; +// The `EndpointDriver` drops packets if the channel to the connection +// or transport is full. +// Set capacity 10 to avoid unnecessary packet drops if the receiver +// is only very briefly busy, but not buffer a large amount of packets +// if it is blocked longer. +const CHANNEL_CAPACITY: usize = 10; + /// Config for the transport. #[derive(Clone)] pub struct Config { @@ -162,7 +169,7 @@ impl Channel { quinn_config: QuinnConfig, socket_addr: SocketAddr, ) -> Result<(Self, mpsc::Receiver), Error> { - let (new_connections_tx, new_connections_rx) = mpsc::channel(1); + let (new_connections_tx, new_connections_rx) = mpsc::channel(CHANNEL_CAPACITY); let endpoint = Self::new::

(quinn_config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) } @@ -187,7 +194,9 @@ impl Channel { // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; socket.set_nonblocking(true)?; - let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(32); + // Capacity 0 to back-pressure the rest of the application if + // the udp socket is busy. + let (to_endpoint_tx, to_endpoint_rx) = mpsc::channel(0); let channel = Self { to_endpoint: to_endpoint_tx, @@ -436,7 +445,7 @@ impl EndpointDriver

{ }; debug_assert_eq!(connection.side(), quinn_proto::Side::Client); - let (tx, rx) = mpsc::channel(16); + let (tx, rx) = mpsc::channel(CHANNEL_CAPACITY); let connection = Connection::from_quinn_connection( self.channel.clone(), connection, @@ -549,7 +558,7 @@ impl EndpointDriver

{ } }; - let (tx, rx) = mpsc::channel(16); + let (tx, rx) = mpsc::channel(CHANNEL_CAPACITY); let connection = Connection::from_quinn_connection(self.channel.clone(), connec, connec_id, rx); match connection_tx.start_send(connection) { From c8b794a91531d46db5f13ba53bf735a44b8bdaf4 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 24 Oct 2022 01:37:49 +0200 Subject: [PATCH 192/218] quic: fix docs --- src/lib.rs | 1 - transports/quic/src/connection.rs | 50 ++++--- transports/quic/src/connection/connecting.rs | 5 +- transports/quic/src/connection/substream.rs | 12 +- transports/quic/src/endpoint.rs | 132 ++++++++----------- transports/quic/src/error.rs | 49 ------- transports/quic/src/lib.rs | 39 +++++- transports/quic/src/provider.rs | 32 +++-- transports/quic/src/provider/async_std.rs | 58 ++++---- transports/quic/src/provider/tokio.rs | 13 +- transports/quic/src/transport.rs | 47 ++++--- 11 files changed, 226 insertions(+), 212 deletions(-) delete mode 100644 transports/quic/src/error.rs diff --git a/src/lib.rs b/src/lib.rs index 3c8e488523d..0a66ba071e2 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -106,7 +106,6 @@ pub use libp2p_plaintext as plaintext; #[doc(inline)] pub use libp2p_pnet as pnet; #[cfg(feature = "quic")] -#[cfg_attr(docsrs, doc(cfg(feature = "quic")))] #[cfg(not(any(target_os = "emscripten", target_os = "wasi", target_os = "unknown")))] pub mod quic { #[doc(inline)] diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index bcd38ac50fe..45038e038eb 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -46,25 +46,28 @@ use std::{ /// State for a single opened QUIC connection. #[derive(Debug)] pub struct Connection { + /// State shared with the substreams. state: Arc>, - /// Channel to the endpoint this connection belongs to. + /// Channel to the [`endpoint::Driver`] that drives the [`quinn_proto::Endpoint`] that + /// this connection belongs to. endpoint_channel: endpoint::Channel, - /// Pending message to be sent to the background task that is driving the endpoint. + /// Pending message to be sent to the [`quinn_proto::Endpoint`] in the [`endpoint::Driver`]. pending_to_endpoint: Option, - /// Events that the endpoint will send in destination to our local [`quinn_proto::Connection`]. - /// Passed at initialization. + /// Events that the [`quinn_proto::Endpoint`] will send in destination to our local + /// [`quinn_proto::Connection`]. from_endpoint: mpsc::Receiver, - /// Identifier for this connection according to the endpoint. Used when sending messages to - /// the endpoint. + /// Identifier for this connection according to the [`quinn_proto::Endpoint`]. + /// Used when sending messages to the endpoint. connection_id: quinn_proto::ConnectionHandle, - /// `Future` that triggers at the [`Instant`] that `self.connection.poll_timeout()` indicates. + /// `Future` that triggers at the [`Instant`] that [`quinn_proto::Connection::poll_timeout`] + /// indicates. next_timeout: Option<(Delay, Instant)>, } impl Connection { - /// Crate-internal function that builds a [`Connection`] from raw components. + /// Build a [`Connection`] from raw components. /// - /// This function assumes that there exists a [`EndpointDriver`](super::endpoint::EndpointDriver) + /// This function assumes that there exists a [`Driver`](super::endpoint::Driver) /// that will process the messages sent to `EndpointChannel::to_endpoint` and send us messages /// on `from_endpoint`. /// @@ -107,6 +110,10 @@ impl Connection { self.state.lock().connection.remote_address() } + /// Identity of the remote peer inferred from the handshake. + /// + /// `None` if the handshake is not complete yet, i.e. [`Self::poll_event`] + /// has not yet reported a [`quinn_proto::Event::Connected`] fn peer_identity(&self) -> Option> { self.state .lock() @@ -116,6 +123,12 @@ impl Connection { } /// Polls the connection for an event that happened on it. + /// + /// `quinn::proto::Connection` is polled in the order instructed in their docs: + /// 1. [`quinn_proto::Connection::poll_transmit`] + /// 2. [`quinn_proto::Connection::poll_timeout`] + /// 3. [`quinn_proto::Connection::poll_endpoint_events`] + /// 4. [`quinn_proto::Connection::poll`] fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { let mut inner = self.state.lock(); loop { @@ -132,9 +145,6 @@ impl Connection { // Sending the pending event to the endpoint. If the endpoint is too busy, we just // stop the processing here. - // We need to be careful to avoid a potential deadlock if both `from_endpoint` and - // `to_endpoint` are full. As such, we continue to transfer data from `from_endpoint` - // to the `quinn_proto::Connection` (see above). // However we don't deliver substream-related events to the user as long as // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. @@ -268,8 +278,13 @@ impl StreamMuxer for Connection { } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Finished { id }) => { if let Some(substream) = inner.substreams.get_mut(&id) { - substream.wake_all(); substream.is_write_closed = true; + if let Some(waker) = substream.write_waker.take() { + waker.wake(); + } + if let Some(waker) = substream.finished_waker.take() { + waker.wake(); + } } } quinn_proto::Event::Stream(quinn_proto::StreamEvent::Stopped { @@ -277,7 +292,12 @@ impl StreamMuxer for Connection { error_code: _, }) => { if let Some(substream) = inner.substreams.get_mut(&id) { - substream.wake_all(); + if let Some(waker) = substream.write_waker.take() { + waker.wake(); + } + if let Some(waker) = substream.finished_waker.take() { + waker.wake(); + } } } quinn_proto::Event::DatagramReceived @@ -380,7 +400,7 @@ impl Drop for Connection { /// Mutex-protected state of [`Connection`]. #[derive(Debug)] pub struct State { - /// The QUIC inner machine for this specific connection. + /// The QUIC inner state machine for this specific connection. connection: quinn_proto::Connection, /// State of all the substreams that the muxer reports as open. diff --git a/transports/quic/src/connection/connecting.rs b/transports/quic/src/connection/connecting.rs index ba067b92456..3e71808bd67 100644 --- a/transports/quic/src/connection/connecting.rs +++ b/transports/quic/src/connection/connecting.rs @@ -39,7 +39,6 @@ pub struct Connecting { } impl Connecting { - /// Builds an [`Connecting`] that wraps around an [`Connection`]. pub(crate) fn new(connection: Connection, timeout: Duration) -> Self { Connecting { connection: Some(connection), @@ -70,6 +69,7 @@ impl Future for Connecting { }; match event { quinn_proto::Event::Connected => { + // Parse the remote's Id identity from the certificate. let identity = connection .peer_identity() .expect("connection got identity because it passed TLS handshake; qed"); @@ -78,8 +78,7 @@ impl Future for Connecting { let end_entity = certificates .get(0) .expect("there should be exactly one certificate; qed"); - let end_entity_der = end_entity.as_ref(); - let p2p_cert = crate::tls::certificate::parse_certificate(end_entity_der) + let p2p_cert = crate::tls::certificate::parse_certificate(end_entity.as_ref()) .expect("the certificate was validated during TLS handshake; qed"); let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); diff --git a/transports/quic/src/connection/substream.rs b/transports/quic/src/connection/substream.rs index 1c1e960b86e..5b4ed518dc3 100644 --- a/transports/quic/src/connection/substream.rs +++ b/transports/quic/src/connection/substream.rs @@ -30,20 +30,22 @@ use parking_lot::Mutex; use super::State; -/// State of a single substream. +/// Wakers for the [`AsyncRead`] and [`AsyncWrite`] on a substream. #[derive(Debug, Default, Clone)] pub struct SubstreamState { - /// Waker to wake if the substream becomes readable or stopped. + /// Waker to wake if the substream becomes readable. pub read_waker: Option, - /// Waker to wake if the substream becomes writable or stopped. + /// Waker to wake if the substream becomes writable, closed or stopped. pub write_waker: Option, /// Waker to wake if the substream becomes closed or stopped. pub finished_waker: Option, + /// `true` if the stream finished, i.e. the writing side closed. pub is_write_closed: bool, } impl SubstreamState { + /// Wake all wakers for reading, writing and closed the stream. pub fn wake_all(&mut self) { if let Some(waker) = self.read_waker.take() { waker.wake(); @@ -57,9 +59,12 @@ impl SubstreamState { } } +/// A single stream on a connection #[derive(Debug)] pub struct Substream { + /// The id of the stream. id: quinn_proto::StreamId, + /// The state of the [`super::Connection`] this stream belongs to. state: Arc>, } @@ -89,6 +94,7 @@ impl AsyncRead for Substream { ); } }; + let mut bytes = 0; let mut pending = false; loop { diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 7565138cda8..b4f1b6c0074 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -18,16 +18,6 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -//! Background task dedicated to manage the QUIC state machine. -//! -//! Considering that all QUIC communications happen over a single UDP socket, one needs to -//! maintain a unique synchronization point that holds the state of all the active connections. -//! -//! The endpoint represents this synchronization point. It is maintained in a background task -//! whose role is to interface with the UDP socket. Communication between the background task and -//! the rest of the code only happens through channels. See the documentation of the -//! [`EndpointDriver`] for a thorough description. - use crate::{provider::Provider, tls, transport::SocketFamily, ConnectError, Connection, Error}; use bytes::BytesMut; @@ -40,13 +30,14 @@ use std::{ collections::HashMap, net::{Ipv4Addr, Ipv6Addr, SocketAddr}, ops::ControlFlow, + pin::Pin, sync::Arc, task::{Context, Poll}, time::{Duration, Instant}, }; use x509_parser::nom::AsBytes; -// The `EndpointDriver` drops packets if the channel to the connection +// The `Driver` drops packets if the channel to the connection // or transport is full. // Set capacity 10 to avoid unnecessary packet drops if the receiver // is only very briefly busy, but not buffer a large amount of packets @@ -79,7 +70,9 @@ pub struct Config { /// of a connection. pub max_connection_data: u32, + /// TLS client config for the inner [`quinn_proto::ClientConfig`]. client_tls_config: Arc, + /// TLS server config for the inner [`quinn_proto::ServerConfig`]. server_tls_config: Arc, } @@ -124,8 +117,10 @@ impl From for QuinnConfig { handshake_timeout: _, } = config; let mut transport = quinn_proto::TransportConfig::default(); + // Disable uni-directional streams. transport.max_concurrent_uni_streams(0u32.into()); transport.max_concurrent_bidi_streams(max_concurrent_stream_limit.into()); + // Disable datagrams. transport.datagram_receive_buffer_size(None); transport.keep_alive_interval(Some(keep_alive_interval)); transport.max_idle_timeout(Some(VarInt::from_u32(max_idle_timeout).into())); @@ -154,6 +149,7 @@ impl From for QuinnConfig { } } +/// Channel used to send commands to the [`Driver`]. #[derive(Debug, Clone)] pub struct Channel { /// Channel to the background of the endpoint. @@ -169,6 +165,7 @@ impl Channel { quinn_config: QuinnConfig, socket_addr: SocketAddr, ) -> Result<(Self, mpsc::Receiver), Error> { + // Channel for forwarding new inbound connections to the listener. let (new_connections_tx, new_connections_rx) = mpsc::channel(CHANNEL_CAPACITY); let endpoint = Self::new::

(quinn_config, socket_addr, Some(new_connections_tx))?; Ok((endpoint, new_connections_rx)) @@ -186,13 +183,14 @@ impl Channel { Self::new::

(quinn_config, socket_addr, None) } + /// Spawn a new [`Driver`] that runs in the background. fn new( quinn_config: QuinnConfig, socket_addr: SocketAddr, new_connections: Option>, ) -> Result { - // NOT blocking, as per man:bind(2), as we pass an IP address. let socket = std::net::UdpSocket::bind(&socket_addr)?; + // NOT blocking, as per man:bind(2), as we pass an IP address. socket.set_nonblocking(true)?; // Capacity 0 to back-pressure the rest of the application if // the udp socket is busy. @@ -206,9 +204,10 @@ impl Channel { let server_config = new_connections .is_some() .then_some(quinn_config.server_config); + let provider_socket = P::from_socket(socket)?; - let driver = EndpointDriver::

::new( + let driver = Driver::

::new( quinn_config.endpoint_config, quinn_config.client_config, new_connections, @@ -218,6 +217,7 @@ impl Channel { to_endpoint_rx, ); + // Drive the endpoint future in the background. P::spawn(driver); Ok(channel) @@ -261,7 +261,7 @@ impl Channel { Ok(Ok(())) } - /// Send a message to inform the [`EndpointDriver`] about an + /// Send a message to inform the [`Driver`] about an /// event caused by the owner of this [`Channel`] dropping. /// This clones the sender to the endpoint to guarantee delivery. /// This should *not* be called for regular messages. @@ -277,14 +277,14 @@ pub struct Disconnected {} /// Message sent to the endpoint background task. #[derive(Debug)] pub enum ToEndpoint { - /// Instruct the endpoint to start connecting to the given address. + /// Instruct the [`quinn_proto::Endpoint`] to start connecting to the given address. Dial { /// UDP address to connect to. addr: SocketAddr, /// Channel to return the result of the dialing to. result: oneshot::Sender>, }, - /// Sent by a `quinn_proto` connection when the endpoint needs to process an event generated + /// Send by a [`quinn_proto::Connection`] when the endpoint needs to process an event generated /// by a connection. The event itself is opaque to us. Only `quinn_proto` knows what is in /// there. ProcessConnectionEvent { @@ -295,15 +295,13 @@ pub enum ToEndpoint { SendUdpPacket(quinn_proto::Transmit), /// The [`GenTransport`][crate::GenTransport] dialer or listener coupled to this endpoint /// was dropped. - /// Once all pending connections are closed, the [`EndpointDriver`] should shut down. + /// Once all pending connections are closed, the [`Driver`] should shut down. Decoupled, } /// Driver that runs in the background for as long as the endpoint is alive. Responsible for /// processing messages and the UDP socket. /// -/// The `receiver` parameter must be the receiving side of the `EndpointChannel::to_endpoint` sender. -/// /// # Behaviour /// /// This background task is responsible for the following: @@ -313,71 +311,45 @@ pub enum ToEndpoint { /// machine. /// - Transmitting events generated by the [`quinn_proto::Endpoint`] to the corresponding /// [`crate::Connection`]. -/// - Receiving messages from the `receiver` and processing the requested actions. This includes +/// - Receiving messages from the `rx` and processing the requested actions. This includes /// UDP packets to send and events emitted by the [`crate::Connection`] objects. -/// - Sending new connections on `new_connections`. +/// - Sending new connections on `new_connection_tx`. /// /// When it comes to channels, there exists three main multi-producer-single-consumer channels /// in play: /// -/// - One channel, represented by `EndpointChannel::to_endpoint` and `receiver`, that communicates -/// messages from [`Channel`] to the [`EndpointDriver`]. -/// - One channel per each existing connection that communicates messages from the [`EndpointDriver`] -/// to that [`crate::Connection`]. -/// - One channel for the [`EndpointDriver`] to send newly-opened connections to. The receiving +/// - One channel, represented by `EndpointChannel::to_endpoint` and `Driver::rx`, +/// that communicates messages from [`Channel`] to the [`Driver`]. +/// - One channel for each existing connection that communicates messages from the +/// [`Driver` to that [`crate::Connection`]. +/// - One channel for the [`Driver`] to send newly-opened connections to. The receiving /// side is processed by the [`GenTransport`][crate::GenTransport]. /// -/// In order to avoid an unbounded buffering of events, we prioritize sending data on the UDP -/// socket over everything else. If the network interface is too busy to process our packets, -/// everything comes to a freeze (including receiving UDP packets) until it is ready to accept -/// more. /// -/// Apart from freezing when the network interface is too busy, the background task should sleep -/// as little as possible. It is in particular important for the `receiver` to be drained as -/// quickly as possible in order to avoid unnecessary back-pressure on the [`crate::Connection`] objects. +/// ## Back-pressure /// -/// ## Back-pressure on `new_connections` +/// ### If writing to the UDP socket is blocked /// -/// The [`quinn_proto::Endpoint`] object contains an accept buffer, in other words a buffer of the -/// incoming connections waiting to be accepted. When a new connection is signalled, we send this -/// new connection on the `new_connection_tx` channel in an asynchronous way, and we only free a slot -/// in the accept buffer once the element has actually been enqueued on `new_connection_tx`. There -/// are therefore in total three buffers in play: the `new_connection_tx` channel itself, the queue -/// of elements being sent on `new_connection_tx`, and the accept buffer of the -/// [`quinn_proto::Endpoint`]. +/// In order to avoid an unbounded buffering of events, we prioritize sending data on the UDP +/// socket over everything else. Messages from the rest of the application sent through the +/// [`Channel`] are only processed if the UDP socket is ready so that we propagate back-pressure +/// in case of a busy socket. For connections, thus this eventually also back-pressures the +/// `AsyncWrite`on substreams. /// -/// ## Back-pressure on connections /// -/// Because connections are processed by the user at a rate of their choice, we cannot properly -/// handle the situation where the channel from the background task to individual connections is -/// full. Sleeping the task while waiting for the connection to be processed by the user could -/// even lead to a deadlock if this processing is also sleeping waiting for some other action that -/// itself depends on the background task (e.g. if processing the connection is waiting for a -/// message arriving on a different connection). +/// ### Back-pressuring the remote if the application is busy /// -/// In an ideal world, we would handle a background-task-to-connection channel being full by -/// dropping UDP packets destined to this connection, as a way to back-pressure the remote. -/// Unfortunately, the `quinn-proto` library doesn't provide any way for us to know which -/// connection a UDP packet is destined for before it has been turned into a `ConnectionEvent`, -/// and because these `ConnectionEvent`s are sometimes used to synchronize the states of the -/// endpoint and connection, it would be a logic error to silently drop them. +/// If the channel to a connection is full because the connection is busy, inbound datagrams +/// for that connection are dropped so that the remote is backpressured. +/// The same applies for new connections if the transport is too busy to received it. /// -/// We handle this tricky situation by simply killing connections as soon as their associated -/// channel is full. /// -// TODO: actually implement the killing of connections if channel is full, at the moment we just -// wait /// # Shutdown /// -/// The background task shuts down if `endpoint_weak`, `receiver` or `new_connections` become -/// disconnected/invalid. This corresponds to the lifetime of the associated [`quinn_proto::Endpoint`]. -/// -/// Keep in mind that we pass an `EndpointChannel` whenever we create a new connection, which -/// guarantees that the [`EndpointDriver`], is properly kept alive for as long as any QUIC -/// connection is open. -/// +/// The background task shuts down if an [`ToEndpoint::Decoupled`] event was received and the +/// last active connection has drained. #[derive(Debug)] -pub struct EndpointDriver { +pub struct Driver { // The actual QUIC state machine. endpoint: quinn_proto::Endpoint, // QuinnConfig for client connections. @@ -402,7 +374,7 @@ pub struct EndpointDriver { is_decoupled: bool, } -impl EndpointDriver

{ +impl Driver

{ fn new( endpoint_config: Arc, client_config: quinn_proto::ClientConfig, @@ -412,7 +384,7 @@ impl EndpointDriver

{ socket: P, rx: mpsc::Receiver, ) -> Self { - EndpointDriver { + Driver { endpoint: quinn_proto::Endpoint::new(endpoint_config, server_config), client_config, channel, @@ -425,7 +397,8 @@ impl EndpointDriver

{ } } - /// Handle a message sent from either the [`GenTransport`](super::GenTransport) or a [`crate::Connection`]. + /// Handle a message sent from either the [`GenTransport`](super::GenTransport) + /// or a [`crate::Connection`]. fn handle_message( &mut self, to_endpoint: ToEndpoint, @@ -502,7 +475,7 @@ impl EndpointDriver

{ ControlFlow::Continue(None) } - /// Handle datagram received on the socket. + /// Handle an UDP datagram received on the socket. /// The datagram content was written into the `socket_recv_buffer`. fn handle_datagram(&mut self, packet: BytesMut, packet_src: SocketAddr) -> ControlFlow<()> { let local_ip = self.channel.socket_addr.ip(); @@ -518,7 +491,7 @@ impl EndpointDriver

{ match event { quinn_proto::DatagramEvent::ConnectionEvent(event) => { // `event` has type `quinn_proto::ConnectionEvent`, which has multiple - // variants. However, `quinn_proto::Endpoint::handle` only ever returns + // variants. `quinn_proto::Endpoint::handle` however only ever returns // `ConnectionEvent::Datagram`. debug_assert!(format!("{:?}", event).contains("Datagram")); @@ -577,6 +550,7 @@ impl EndpointDriver

{ ControlFlow::Continue(()) } + /// The transport dropped the channel to this [`Driver`]. fn handle_decoupling(&mut self) -> ControlFlow<()> { if self.alive_connections.is_empty() { return ControlFlow::Break(()); @@ -589,25 +563,32 @@ impl EndpointDriver

{ } } -impl Future for EndpointDriver

{ +/// Future that runs until the [`Driver`] is decoupled and not active connections +/// remain +impl Future for Driver

{ type Output = (); - fn poll(mut self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { loop { + // Flush any pending pocket so that the socket is reading to write an next + // packet. match self.provider_socket.poll_send_flush(cx) { + // The pending packet was send or no packet was pending. Poll::Ready(Ok(_)) => { + // Start sending a packet on the socket. if let Some(transmit) = self.next_packet_out.take() { self.provider_socket .start_send(transmit.contents, transmit.destination); continue; } - // The endpoint might request packets to be sent out. This is handled in priority to avoid - // buffering up packets. + // The endpoint might request packets to be sent out. This is handled in + // priority to avoid buffering up packets. if let Some(transmit) = self.endpoint.poll_transmit() { self.next_packet_out = Some(transmit); continue; } + // Handle messages from transport and connections. match self.rx.poll_next_unpin(cx) { Poll::Ready(Some(to_endpoint)) => match self.handle_message(to_endpoint) { ControlFlow::Continue(Some(transmit)) => { @@ -634,6 +615,7 @@ impl Future for EndpointDriver

{ Poll::Pending => {} } + // Poll for new packets from the remote. match self.provider_socket.poll_recv_from(cx) { Poll::Ready(Ok((bytes, packet_src))) => { let bytes_mut = bytes.as_bytes().into(); diff --git a/transports/quic/src/error.rs b/transports/quic/src/error.rs deleted file mode 100644 index 9fbb074f11c..00000000000 --- a/transports/quic/src/error.rs +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright 2022 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. - -/// Error that can happen on the transport. -#[derive(Debug, thiserror::Error)] -pub enum Error { - /// Error while trying to reach a remote. - #[error(transparent)] - Reach(#[from] ConnectError), - - /// Error after the remote has been reached. - #[error(transparent)] - Connection(#[from] ConnectionError), - - #[error(transparent)] - Io(#[from] std::io::Error), - - /// The task driving the endpoint has crashed. - #[error("Endpoint driver crashed")] - EndpointDriverCrashed, - - #[error("Handshake with the remote timed out.")] - HandshakeTimedOut, -} - -#[derive(Debug, thiserror::Error)] -#[error(transparent)] -pub struct ConnectError(#[from] quinn_proto::ConnectError); - -#[derive(Debug, thiserror::Error)] -#[error(transparent)] -pub struct ConnectionError(#[from] quinn_proto::ConnectionError); diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 6e85f2235b2..60aac4d6333 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -//! Implementation of the libp2p `Transport` and `StreamMuxer` traits for QUIC. +//! Implementation of the QUIC transport protocol for libp2p. //! //! # Usage //! @@ -59,17 +59,50 @@ mod connection; mod endpoint; -mod error; mod provider; mod tls; mod transport; pub use connection::{Connecting, Connection, Substream}; pub use endpoint::Config; -pub use error::{ConnectError, ConnectionError, Error}; #[cfg(feature = "async-std")] pub use provider::async_std; #[cfg(feature = "tokio")] pub use provider::tokio; pub use provider::Provider; pub use transport::GenTransport; + +/// Errors that may happen on the [`GenTransport`] or a single [`Connection`]. +#[derive(Debug, thiserror::Error)] +pub enum Error { + /// Error while trying to reach a remote. + #[error(transparent)] + Reach(#[from] ConnectError), + + /// Error after the remote has been reached. + #[error(transparent)] + Connection(#[from] ConnectionError), + + /// I/O Error on a socket. + #[error(transparent)] + Io(#[from] std::io::Error), + + /// The task spawned in [`Provider::spawn`] to drive + /// the quic endpoint has crashed. + #[error("Endpoint driver crashed")] + EndpointDriverCrashed, + + /// The [`Connecting`] future timed out. + #[error("Handshake with the remote timed out.")] + HandshakeTimedOut, +} + +/// Dialing a remote peer failed. +#[derive(Debug, thiserror::Error)] +#[error(transparent)] +pub struct ConnectError(#[from] quinn_proto::ConnectError); + +/// Error on an established [`Connection`]. +#[derive(Debug, thiserror::Error)] +#[error(transparent)] +pub struct ConnectionError(#[from] quinn_proto::ConnectionError); diff --git a/transports/quic/src/provider.rs b/transports/quic/src/provider.rs index 6f067bd4c61..f413625aaf7 100644 --- a/transports/quic/src/provider.rs +++ b/transports/quic/src/provider.rs @@ -30,25 +30,35 @@ pub mod async_std; #[cfg(feature = "tokio")] pub mod tokio; -// Wrapped socket for non-blocking I/O operations. +/// Size of the buffer for reading data 0x10000. +const RECEIVE_BUFFER_SIZE: usize = 65536; + +/// Provider for non-blocking receiving and sending on a [`std::net::UdpSocket`] +/// and spawning tasks. pub trait Provider: Unpin + Send + Sized + 'static { - // Wrap a socket. - // Note: The socket must be set to non-blocking. + /// Create a new providing that is wrapping the socket. + /// + /// Note: The socket must be set to non-blocking. fn from_socket(socket: std::net::UdpSocket) -> io::Result; - // Receive a single datagram message. - // Returns the message and the address the message came from. + /// Receive a single packet. + /// + /// Returns the message and the address the message came from. fn poll_recv_from(&mut self, cx: &mut Context<'_>) -> Poll, SocketAddr)>>; - // Set sending the packet on the socket. - // Since only one packet may be sent at a time, this may only be called if a preceding call - // to [`Provider::poll_send_flush`] returned [`Poll::Ready`]. + /// Set sending a packet on the socket. + /// + /// Since only one packet can be sent at a time, this may only be called if a preceding + /// call to [`Provider::poll_send_flush`] returned [`Poll::Ready`]. fn start_send(&mut self, data: Vec, addr: SocketAddr); - // Flush a packet send in [`Provider::start_send`]. - // If [`Poll::Ready`] is returned the socket is ready for sending a new packet. + /// Flush a packet send in [`Provider::start_send`]. + /// + /// If [`Poll::Ready`] is returned the socket is ready for sending a new packet. fn poll_send_flush(&mut self, cx: &mut Context<'_>) -> Poll>; - // Run the given future in the background until it ends. + /// Run the given future in the background until it ends. + /// + /// This is used to spawn the task that is driving the endpoint. fn spawn(future: impl Future + Send + 'static); } diff --git a/transports/quic/src/provider/async_std.rs b/transports/quic/src/provider/async_std.rs index eef4f5dc740..a9d10b4d4f7 100644 --- a/transports/quic/src/provider/async_std.rs +++ b/transports/quic/src/provider/async_std.rs @@ -18,6 +18,8 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use async_std::{net::UdpSocket, task::spawn}; +use futures::{future::BoxFuture, ready, Future, FutureExt, Stream, StreamExt}; use std::{ io, net::SocketAddr, @@ -26,22 +28,23 @@ use std::{ task::{Context, Poll}, }; -use async_std::{net::UdpSocket, task::spawn}; -use futures::{future::BoxFuture, ready, Future, FutureExt, Stream, StreamExt}; - use crate::GenTransport; -use super::Provider as ProviderTrait; - +/// Transport with [`async-std`] runtime. pub type Transport = GenTransport; +/// Provider for reading / writing to a sockets and spawning +/// tasks using [`async-std`]. pub struct Provider { socket: Arc, + // Future for sending a packet. + // This is needed since [`async_Std::net::UdpSocket`] does not + // provide a poll-style interface for sending a packet. send_packet: Option>>, recv_stream: ReceiveStream, } -impl ProviderTrait for Provider { +impl super::Provider for Provider { fn from_socket(socket: std::net::UdpSocket) -> io::Result { let socket = Arc::new(socket.into()); let recv_stream = ReceiveStream::new(Arc::clone(&socket)); @@ -62,10 +65,9 @@ impl ProviderTrait for Provider { } fn start_send(&mut self, data: Vec, addr: SocketAddr) { - let _len = data.len(); let socket = self.socket.clone(); let send = async move { - let _send_len = socket.send_to(&data, addr).await?; + socket.send_to(&data, addr).await?; Ok(()) } .boxed(); @@ -91,44 +93,46 @@ impl ProviderTrait for Provider { } } +type ReceiveStreamItem = ( + Result<(usize, SocketAddr), io::Error>, + Arc, + Vec, +); + /// Wrapper around the socket to implement `Stream` on it. struct ReceiveStream { - fut: BoxFuture< - 'static, - ( - Result<(usize, SocketAddr), io::Error>, - Arc, - Vec, - ), - >, + /// Future for receiving a packet on the socket. + // This is needed since [`async_Std::net::UdpSocket`] does not + // provide a poll-style interface for receiving packets. + fut: BoxFuture<'static, ReceiveStreamItem>, } impl ReceiveStream { fn new(socket: Arc) -> Self { - let mut socket_recv_buffer = vec![0; 65536]; - let fut = async move { - let recv = socket.recv_from(&mut socket_recv_buffer).await; - (recv, socket, socket_recv_buffer) - }; + let fut = ReceiveStream::next(socket, vec![0; super::RECEIVE_BUFFER_SIZE]).boxed(); Self { fut: fut.boxed() } } + + async fn next(socket: Arc, mut socket_recv_buffer: Vec) -> ReceiveStreamItem { + let recv = socket.recv_from(&mut socket_recv_buffer).await; + (recv, socket, socket_recv_buffer) + } } impl Stream for ReceiveStream { type Item = Result<(Vec, SocketAddr), io::Error>; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - let (result, socket, mut buffer) = ready!(self.fut.poll_unpin(cx)); + let (result, socket, buffer) = ready!(self.fut.poll_unpin(cx)); + let result = result.map(|(packet_len, packet_src)| { debug_assert!(packet_len <= buffer.len()); // Copies the bytes from the `socket_recv_buffer` they were written into. (buffer[..packet_len].into(), packet_src) }); - self.fut = async move { - let recv = socket.recv_from(&mut buffer).await; - (recv, socket, buffer) - } - .boxed(); + // Set the future for receiving the next packet on the stream. + self.fut = ReceiveStream::next(socket, buffer).boxed(); + Poll::Ready(Some(result)) } } diff --git a/transports/quic/src/provider/tokio.rs b/transports/quic/src/provider/tokio.rs index b7374e8c9d6..e0583c3705a 100644 --- a/transports/quic/src/provider/tokio.rs +++ b/transports/quic/src/provider/tokio.rs @@ -18,33 +18,34 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. +use futures::{ready, Future}; use std::{ io, net::SocketAddr, task::{Context, Poll}, }; - -use futures::{ready, Future}; use tokio::{io::ReadBuf, net::UdpSocket}; use x509_parser::nom::AsBytes; use crate::GenTransport; -use super::Provider as ProviderTrait; - +/// Transport with [`tokio`] runtime. pub type Transport = GenTransport; + +/// Provider for reading / writing to a sockets and spawning +/// tasks using [`tokio`]. pub struct Provider { socket: UdpSocket, socket_recv_buffer: Vec, next_packet_out: Option<(Vec, SocketAddr)>, } -impl ProviderTrait for Provider { +impl super::Provider for Provider { fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { let socket = UdpSocket::from_std(socket)?; Ok(Provider { socket, - socket_recv_buffer: vec![0; 65536], + socket_recv_buffer: vec![0; super::RECEIVE_BUFFER_SIZE], next_packet_out: None, }) } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 681281ea418..4825dda1f1a 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -18,10 +18,6 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -//! Implementation of the [`Transport`] trait for QUIC. -//! -//! Combines all the objects in the other modules to implement the trait. - use crate::endpoint::{Config, QuinnConfig, ToEndpoint}; use crate::provider::Provider; use crate::{endpoint, Connecting, Connection, Error}; @@ -52,19 +48,22 @@ use std::{ task::{Context, Poll}, }; +/// Implementation of the [`Transport`] trait for QUIC. #[derive(Debug)] pub struct GenTransport

{ + /// Config for the inner [`quinn_proto`] structs. quinn_config: QuinnConfig, + /// Timeout for the [`Connecting`] future. handshake_timeout: Duration, - + /// Streams of active [`Listener`]s. listeners: SelectAll, /// Dialer for each socket family if no matching listener exists. dialer: HashMap, - _marker: PhantomData

, } impl

GenTransport

{ + /// Create a new [`GenTransport`] with the given [`Config`]. pub fn new(config: Config) -> Self { let handshake_timeout = config.handshake_timeout; let quinn_config = config.into(); @@ -98,7 +97,7 @@ impl Transport for GenTransport

{ // Remove dialer endpoint so that the endpoint is dropped once the last // connection that uses it is closed. - // New outbound connections will use a bidirectional (listener) endpoint. + // New outbound connections will use the bidirectional (listener) endpoint. self.dialer.remove(&socket_addr.ip().into()); Ok(listener_id) @@ -128,6 +127,7 @@ impl Transport for GenTransport

{ if socket_addr.port() == 0 || socket_addr.ip().is_unspecified() { return Err(TransportError::MultiaddrNotSupported(addr)); } + let mut listeners = self .listeners .iter_mut() @@ -206,9 +206,13 @@ impl From for TransportError { } } +/// Dialer for addresses if no matching listener exists. #[derive(Debug)] struct Dialer { + /// Channel to the [`crate::endpoint::Driver`] that + /// is driving the endpoint. endpoint_channel: endpoint::Channel, + /// Queued dials for the endpoint. state: DialerState, } @@ -236,6 +240,8 @@ impl Drop for Dialer { } } +/// Pending dials to be sent to the endpoint was the [`endpoint::Channel`] +/// has capacity #[derive(Default, Debug)] struct DialerState { pending_dials: VecDeque, @@ -243,7 +249,6 @@ struct DialerState { } impl DialerState { - // With TAIP, this return signature would be a bit nicer. fn new_dial( &mut self, address: SocketAddr, @@ -274,7 +279,7 @@ impl DialerState { /// Send all pending dials into the given [`endpoint::Channel`]. /// - /// This only ever returns [`Poll::Pending`] or an error in case the channel is closed. + /// This only ever returns [`Poll::Pending`], or an error in case the channel is closed. fn poll(&mut self, channel: &mut endpoint::Channel, cx: &mut Context<'_>) -> Poll { while let Some(to_endpoint) = self.pending_dials.pop_front() { match channel.try_send(to_endpoint, cx) { @@ -291,16 +296,25 @@ impl DialerState { } } +/// Listener for incoming connections. #[derive(Debug)] struct Listener { - endpoint_channel: endpoint::Channel, - + /// Id of the listener. listener_id: ListenerId, + /// Channel to the endpoint to initiate dials. + endpoint_channel: endpoint::Channel, + /// Queued dials. + dialer_state: DialerState, + /// Channel where new connections are being sent. new_connections_rx: mpsc::Receiver, + /// Timeout for connection establishment on inbound connections. handshake_timeout: Duration, + /// Watcher for network interface changes. + /// + /// None if we are only listening on a single interface. if_watcher: Option, /// Whether the listener was closed and the stream should terminate. @@ -308,10 +322,6 @@ struct Listener { /// Pending event to reported. pending_event: Option<::Item>, - - dialer_state: DialerState, - - waker: Option, } impl Listener { @@ -347,7 +357,6 @@ impl Listener { is_closed: false, pending_event, dialer_state: DialerState::default(), - waker: None, }) } @@ -455,7 +464,6 @@ impl Stream for Listener { } Poll::Pending => {} }; - self.waker = Some(cx.waker().clone()); return Poll::Pending; } } @@ -534,6 +542,7 @@ fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { } } +/// Whether an [`Multiaddr`] is a valid for the QUIC transport. fn is_quic_addr(addr: &Multiaddr) -> bool { use Protocol::*; let mut iter = addr.iter(); @@ -692,7 +701,7 @@ mod test { .is_none()); assert!(transport.listeners.is_empty()); - // Check that the [`EndpointDriver`] has shut down. + // Check that the [`Driver`] has shut down. Delay::new(Duration::from_millis(10)).await; poll_fn(|cx| { assert!(channel.try_send(ToEndpoint::Decoupled, cx).is_err()); @@ -744,7 +753,7 @@ mod test { .unwrap(); assert!(!transport.dialer.contains_key(&SocketFamily::Ipv4)); - // Check that the [`EndpointDriver`] has shut down. + // Check that the [`Driver`] has shut down. Delay::new(Duration::from_millis(10)).await; poll_fn(|cx| { assert!(channel.try_send(ToEndpoint::Decoupled, cx).is_err()); From e099867f4b20372cbb03a3c6617abdb267b1f125 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Mon, 24 Oct 2022 12:00:43 +1100 Subject: [PATCH 193/218] Update to use `libp2p_tls` --- transports/quic/Cargo.toml | 5 +- transports/quic/src/connection/connecting.rs | 4 +- transports/quic/src/endpoint.rs | 6 +- transports/quic/src/lib.rs | 1 - transports/quic/src/tls/certificate.rs | 460 ------------------ transports/quic/src/tls/mod.rs | 104 ---- .../quic/src/tls/test_assets/ed25519.der | Bin 324 -> 0 bytes transports/quic/src/tls/test_assets/ed448.der | Bin 400 -> 0 bytes transports/quic/src/tls/test_assets/gen.sh | 63 --- .../src/tls/test_assets/nistp256_sha256.der | Bin 388 -> 0 bytes .../src/tls/test_assets/nistp384_sha256.der | Bin 450 -> 0 bytes .../src/tls/test_assets/nistp384_sha384.der | Bin 450 -> 0 bytes .../src/tls/test_assets/nistp521_sha512.der | Bin 525 -> 0 bytes .../quic/src/tls/test_assets/openssl.cfg | 6 - .../quic/src/tls/test_assets/pkcs1_sha256.der | Bin 324 -> 0 bytes .../src/tls/test_assets/rsa_pkcs1_sha256.der | Bin 785 -> 0 bytes .../src/tls/test_assets/rsa_pkcs1_sha384.der | Bin 785 -> 0 bytes .../src/tls/test_assets/rsa_pkcs1_sha512.der | Bin 785 -> 0 bytes .../src/tls/test_assets/rsa_pss_sha384.der | Bin 878 -> 0 bytes transports/quic/src/tls/verifier.rs | 213 -------- 20 files changed, 7 insertions(+), 855 deletions(-) delete mode 100644 transports/quic/src/tls/certificate.rs delete mode 100644 transports/quic/src/tls/mod.rs delete mode 100644 transports/quic/src/tls/test_assets/ed25519.der delete mode 100644 transports/quic/src/tls/test_assets/ed448.der delete mode 100755 transports/quic/src/tls/test_assets/gen.sh delete mode 100644 transports/quic/src/tls/test_assets/nistp256_sha256.der delete mode 100644 transports/quic/src/tls/test_assets/nistp384_sha256.der delete mode 100644 transports/quic/src/tls/test_assets/nistp384_sha384.der delete mode 100644 transports/quic/src/tls/test_assets/nistp521_sha512.der delete mode 100644 transports/quic/src/tls/test_assets/openssl.cfg delete mode 100644 transports/quic/src/tls/test_assets/pkcs1_sha256.der delete mode 100644 transports/quic/src/tls/test_assets/rsa_pkcs1_sha256.der delete mode 100644 transports/quic/src/tls/test_assets/rsa_pkcs1_sha384.der delete mode 100644 transports/quic/src/tls/test_assets/rsa_pkcs1_sha512.der delete mode 100644 transports/quic/src/tls/test_assets/rsa_pss_sha384.der delete mode 100644 transports/quic/src/tls/verifier.rs diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 662f08f5555..78a0d4b02a9 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -15,13 +15,12 @@ futures = "0.3.15" futures-timer = "3.0.2" if-watch = "2.0.0" libp2p-core = { version = "0.37.0", path = "../../core" } +libp2p-tls = { version = "0.1.0-alpha", path = "../tls" } log = "0.4" parking_lot = "0.12.0" quinn-proto = { version = "0.8.2", default-features = false, features = ["tls-rustls"] } rand = "0.8.5" -rcgen = "0.9.2" -ring = "0.16.20" -rustls = { version = "0.20.2", default-features = false, features = ["dangerous_configuration"] } +rustls = { version = "0.20.2", default-features = false } thiserror = "1.0.26" tokio = { version = "1.21.1", default-features = false, features = ["net", "rt"], optional = true } webpki = "0.22.0" diff --git a/transports/quic/src/connection/connecting.rs b/transports/quic/src/connection/connecting.rs index 3e71808bd67..3124885e772 100644 --- a/transports/quic/src/connection/connecting.rs +++ b/transports/quic/src/connection/connecting.rs @@ -78,9 +78,9 @@ impl Future for Connecting { let end_entity = certificates .get(0) .expect("there should be exactly one certificate; qed"); - let p2p_cert = crate::tls::certificate::parse_certificate(end_entity.as_ref()) + let p2p_cert = libp2p_tls::certificate::parse(end_entity) .expect("the certificate was validated during TLS handshake; qed"); - let peer_id = PeerId::from_public_key(&p2p_cert.extension.public_key); + let peer_id = p2p_cert.peer_id(); return Poll::Ready(Ok((peer_id, self.connection.take().unwrap()))); } diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index b4f1b6c0074..78155ac286f 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -18,7 +18,7 @@ // FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER // DEALINGS IN THE SOFTWARE. -use crate::{provider::Provider, tls, transport::SocketFamily, ConnectError, Connection, Error}; +use crate::{provider::Provider, transport::SocketFamily, ConnectError, Connection, Error}; use bytes::BytesMut; use futures::{ @@ -79,8 +79,8 @@ pub struct Config { impl Config { /// Creates a new configuration object with default values. pub fn new(keypair: &libp2p_core::identity::Keypair) -> Self { - let client_tls_config = Arc::new(tls::make_client_config(keypair).unwrap()); - let server_tls_config = Arc::new(tls::make_server_config(keypair).unwrap()); + let client_tls_config = Arc::new(libp2p_tls::make_client_config(keypair, None).unwrap()); + let server_tls_config = Arc::new(libp2p_tls::make_server_config(keypair).unwrap()); Self { client_tls_config, server_tls_config, diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index 60aac4d6333..b6a3ea0ada9 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -60,7 +60,6 @@ mod connection; mod endpoint; mod provider; -mod tls; mod transport; pub use connection::{Connecting, Connection, Substream}; diff --git a/transports/quic/src/tls/certificate.rs b/transports/quic/src/tls/certificate.rs deleted file mode 100644 index 497c8b9130e..00000000000 --- a/transports/quic/src/tls/certificate.rs +++ /dev/null @@ -1,460 +0,0 @@ -// Copyright 2021 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. - -//! X.509 certificate handling for libp2p -//! -//! This module handles generation, signing, and verification of certificates. - -use libp2p_core::identity; -use x509_parser::prelude::*; - -/// The libp2p Public Key Extension is a X.509 extension -/// with the Object Identier 1.3.6.1.4.1.53594.1.1, -/// allocated by IANA to the libp2p project at Protocol Labs. -const P2P_EXT_OID: [u64; 9] = [1, 3, 6, 1, 4, 1, 53594, 1, 1]; - -/// The peer signs the concatenation of the string `libp2p-tls-handshake:` -/// and the public key that it used to generate the certificate carrying -/// the libp2p Public Key Extension, using its private host key. -/// This signature provides cryptographic proof that the peer was -/// in possession of the private host key at the time the certificate was signed. -const P2P_SIGNING_PREFIX: [u8; 21] = *b"libp2p-tls-handshake:"; - -// Certificates MUST use the NamedCurve encoding for elliptic curve parameters. -// Similarly, hash functions with an output length less than 256 bits MUST NOT be used. -static P2P_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 fn make_certificate( - keypair: &identity::Keypair, -) -> Result { - // Keypair used to sign the certificate. - // SHOULD NOT be related to the host's key. - // Endpoints MAY generate a new key and certificate - // for every connection attempt, or they MAY reuse the same key - // and certificate for multiple connections. - let certif_keypair = rcgen::KeyPair::generate(P2P_SIGNATURE_ALGORITHM)?; - - // Generate the libp2p-specific extension. - // The certificate MUST contain the libp2p Public Key Extension. - let libp2p_extension: rcgen::CustomExtension = { - // The peer signs the concatenation of the string `libp2p-tls-handshake:` - // and the public key that it used to generate the certificate carrying - // the libp2p Public Key Extension, using its private host key. - let signature = { - let mut msg = vec![]; - msg.extend(P2P_SIGNING_PREFIX); - msg.extend(certif_keypair.public_key_der()); - - keypair - .sign(&msg) - .map_err(|_| rcgen::RcgenError::RingUnspecified)? - }; - - // The public host key and the signature are ANS.1-encoded - // into the SignedKey data structure, which is carried - // in the libp2p Public Key Extension. - // SignedKey ::= SEQUENCE { - // publicKey OCTET STRING, - // signature OCTET STRING - // } - let extension_content = { - let serialized_pubkey = keypair.public().to_protobuf_encoding(); - yasna::encode_der(&(serialized_pubkey, signature)) - }; - - // This extension MAY be marked critical. - let mut ext = rcgen::CustomExtension::from_oid_content(&P2P_EXT_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 = P2P_SIGNATURE_ALGORITHM; - params.key_pair = Some(certif_keypair); - rcgen::Certificate::from_params(params)? - }; - - Ok(certificate) -} - -/// The contents of the specific libp2p extension, containing the public host key -/// and a signature performed using the private host key. -pub struct P2pExtension { - pub(crate) public_key: identity::PublicKey, - /// This signature provides cryptographic proof that the peer was - /// in possession of the private host key at the time the certificate was signed. - signature: Vec, -} - -/// An X.509 certificate with a libp2p-specific extension -/// is used to secure libp2p connections. -pub struct P2pCertificate<'a> { - certificate: X509Certificate<'a>, - /// This is a specific libp2p Public Key Extension with two values: - /// * the public host key - /// * a signature performed using the private host key - pub(crate) extension: P2pExtension, -} - -/// Parse TLS certificate from DER input that includes a libp2p-specific -/// certificate extension containing a public key of a peer. -pub fn parse_certificate(der_input: &[u8]) -> Result { - use webpki::Error; - let x509 = X509Certificate::from_der(der_input) - .map(|(_rest_input, x509)| x509) - .map_err(|_| Error::BadDer)?; - - let p2p_ext_oid = der_parser::oid::Oid::from(&P2P_EXT_OID) - .expect("This is a valid OID of p2p extension; qed"); - - let mut libp2p_extension = None; - - for ext in x509.extensions() { - let oid = &ext.oid; - if oid == &p2p_ext_oid && libp2p_extension.is_some() { - // The extension was already parsed - return Err(Error::BadDer); - } - - if oid == &p2p_ext_oid { - // The public host key and the signature are ANS.1-encoded - // into the SignedKey data structure, which is carried - // in the libp2p Public Key Extension. - // SignedKey ::= SEQUENCE { - // publicKey OCTET STRING, - // signature OCTET STRING - // } - let (public_key, signature): (Vec, Vec) = - yasna::decode_der(ext.value).map_err(|_| Error::ExtensionValueInvalid)?; - // The publicKey field of SignedKey contains the public host key - // of the endpoint, encoded using the following protobuf: - // enum KeyType { - // RSA = 0; - // Ed25519 = 1; - // Secp256k1 = 2; - // ECDSA = 3; - // } - // message PublicKey { - // required KeyType Type = 1; - // required bytes Data = 2; - // } - let public_key = identity::PublicKey::from_protobuf_encoding(&public_key) - .map_err(|_| Error::UnknownIssuer)?; - let ext = P2pExtension { - public_key, - signature, - }; - libp2p_extension = Some(ext); - continue; - } - - if ext.critical { - // Endpoints MUST abort the connection attempt if the certificate - // contains critical extensions that the endpoint does not understand. - return Err(Error::UnsupportedCriticalExtension); - } - - // Implementations MUST ignore non-critical extensions with unknown OIDs. - } - - // The certificate MUST contain the libp2p Public Key Extension. - // If this extension is missing, endpoints MUST abort the connection attempt. - let extension = libp2p_extension.ok_or(Error::BadDer)?; - - Ok(P2pCertificate { - certificate: x509, - extension, - }) -} - -impl P2pCertificate<'_> { - /// This method validates the certificate according to libp2p TLS 1.3 specs. - /// The certificate MUST: - /// 1. be valid at the time it is received by the peer; - /// 2. use the NamedCurve encoding; - /// 3. use hash functions with an output length not less than 256 bits; - /// 4. be self signed; - /// 5. contain a valid signature in the specific libp2p extension. - pub fn verify(&self) -> Result<(), webpki::Error> { - use webpki::Error; - // The certificate MUST have NotBefore and NotAfter fields set - // such that the certificate is valid at the time it is received by the peer. - if !self.certificate.validity().is_valid() { - return Err(Error::InvalidCertValidity); - } - - // Certificates MUST use the NamedCurve encoding for elliptic curve parameters. - // Similarly, hash functions with an output length less than 256 bits - // MUST NOT be used, due to the possibility of collision attacks. - // In particular, MD5 and SHA1 MUST NOT be used. - // Endpoints MUST abort the connection attempt if it is not used. - let signature_scheme = self.signature_scheme()?; - // Endpoints MUST abort the connection attempt if the certificate’s - // self-signature is not valid. - let raw_certificate = self.certificate.tbs_certificate.as_ref(); - let signature = self.certificate.signature_value.data; - // check if self signed - self.verify_signature(signature_scheme, raw_certificate, signature) - .map_err(|_| Error::SignatureAlgorithmMismatch)?; - - let subject_pki = self.certificate.public_key().raw; - - // The peer signs the concatenation of the string `libp2p-tls-handshake:` - // and the public key that it used to generate the certificate carrying - // the libp2p Public Key Extension, using its private host key. - let mut msg = vec![]; - msg.extend(P2P_SIGNING_PREFIX); - msg.extend(subject_pki); - - // This signature provides cryptographic proof that the peer was in possession - // of the private host key at the time the certificate was signed. - // Peers MUST verify the signature, and abort the connection attempt - // if signature verification fails. - let user_owns_sk = self - .extension - .public_key - .verify(&msg, &self.extension.signature); - if !user_owns_sk { - return Err(Error::UnknownIssuer); - } - Ok(()) - } - - /// Return the signature scheme corresponding to [`AlgorithmIdentifier`]s - /// of `subject_pki` and `signature_algorithm` - /// according to ``. - pub fn signature_scheme(&self) -> Result { - // Certificates MUST use the NamedCurve encoding for elliptic curve parameters. - // Endpoints MUST abort the connection attempt if it is not used. - use oid_registry::*; - use rustls::SignatureScheme::*; - use webpki::Error; - let signature_algorithm = &self.certificate.signature_algorithm; - let pki_algorithm = &self.certificate.tbs_certificate.subject_pki.algorithm; - - if pki_algorithm.algorithm == OID_PKCS1_RSAENCRYPTION { - if signature_algorithm.algorithm == OID_PKCS1_SHA256WITHRSA { - return Ok(RSA_PKCS1_SHA256); - } - if signature_algorithm.algorithm == OID_PKCS1_SHA384WITHRSA { - return Ok(RSA_PKCS1_SHA384); - } - if signature_algorithm.algorithm == OID_PKCS1_SHA512WITHRSA { - return Ok(RSA_PKCS1_SHA512); - } - if signature_algorithm.algorithm == OID_PKCS1_RSASSAPSS { - // According to https://datatracker.ietf.org/doc/html/rfc4055#section-3.1: - // Inside of params there shuld be a sequence of: - // - Hash Algorithm - // - Mask Algorithm - // - Salt Length - // - Trailer Field - - // We are interested in Hash Algorithm only, however the der parser parses - // params into a mess, so here is a workaround to fix it: - fn get_hash_oid<'a>( - signature_algorithm: &'a AlgorithmIdentifier, - ) -> Option> { - let params = signature_algorithm.parameters.as_ref()?; - let params = params.as_sequence().ok()?; - let first_param = params.get(0)?; - let hash_oid_der = first_param.as_slice().ok()?; - let (_, obj) = der_parser::parse_der(hash_oid_der).ok()?; - let hash_oid = obj.as_sequence().ok()?.get(0)?.as_oid_val().ok()?; - Some(hash_oid) - } - - let hash_oid = get_hash_oid(signature_algorithm).ok_or(Error::BadDer)?; - - if hash_oid == OID_NIST_HASH_SHA256 { - return Ok(RSA_PSS_SHA256); - } - if hash_oid == OID_NIST_HASH_SHA384 { - return Ok(RSA_PSS_SHA384); - } - if hash_oid == OID_NIST_HASH_SHA512 { - return Ok(RSA_PSS_SHA512); - } - - // Default hash algo is SHA-1, however: - // In particular, MD5 and SHA1 MUST NOT be used. - return Err(Error::UnsupportedSignatureAlgorithm); - } - } - - if pki_algorithm.algorithm == OID_KEY_TYPE_EC_PUBLIC_KEY { - let signature_param = pki_algorithm - .parameters - .as_ref() - .ok_or(Error::BadDer)? - .as_oid_val() - .map_err(|_| Error::BadDer)?; - if signature_param == OID_EC_P256 - && signature_algorithm.algorithm == OID_SIG_ECDSA_WITH_SHA256 - { - return Ok(ECDSA_NISTP256_SHA256); - } - if signature_param == OID_NIST_EC_P384 - && signature_algorithm.algorithm == OID_SIG_ECDSA_WITH_SHA384 - { - return Ok(ECDSA_NISTP384_SHA384); - } - if signature_param == OID_NIST_EC_P521 - && signature_algorithm.algorithm == OID_SIG_ECDSA_WITH_SHA512 - { - return Ok(ECDSA_NISTP521_SHA512); - } - return Err(Error::UnsupportedSignatureAlgorithm); - } - - if signature_algorithm.algorithm == OID_SIG_ED25519 { - return Ok(ED25519); - } - if signature_algorithm.algorithm == OID_SIG_ED448 { - return Ok(ED448); - } - - Err(Error::UnsupportedSignatureAlgorithm) - } - - /// Get a [`ring::signature::UnparsedPublicKey`] for this `signature_scheme`. - /// Return `Error` if the `signature_scheme` does not match the public key signature - /// and hashing algorithm or if the `signature_scheme` is not supported. - pub fn public_key( - &self, - signature_scheme: rustls::SignatureScheme, - ) -> Result, webpki::Error> { - use ring::signature; - use rustls::SignatureScheme::*; - use webpki::Error; - - let current_signature_scheme = self.signature_scheme()?; - if signature_scheme != current_signature_scheme { - // This certificate was signed with a different signature scheme - return Err(Error::UnsupportedSignatureAlgorithmForPublicKey); - } - - let verification_algorithm: &dyn signature::VerificationAlgorithm = match signature_scheme { - RSA_PKCS1_SHA256 => &signature::RSA_PKCS1_2048_8192_SHA256, - RSA_PKCS1_SHA384 => &signature::RSA_PKCS1_2048_8192_SHA384, - RSA_PKCS1_SHA512 => &signature::RSA_PKCS1_2048_8192_SHA512, - ECDSA_NISTP256_SHA256 => &signature::ECDSA_P256_SHA256_ASN1, - ECDSA_NISTP384_SHA384 => &signature::ECDSA_P384_SHA384_ASN1, - ECDSA_NISTP521_SHA512 => { - // See https://github.com/briansmith/ring/issues/824 - return Err(Error::UnsupportedSignatureAlgorithm); - } - RSA_PSS_SHA256 => &signature::RSA_PSS_2048_8192_SHA256, - RSA_PSS_SHA384 => &signature::RSA_PSS_2048_8192_SHA384, - RSA_PSS_SHA512 => &signature::RSA_PSS_2048_8192_SHA512, - ED25519 => &signature::ED25519, - ED448 => { - // See https://github.com/briansmith/ring/issues/463 - return Err(Error::UnsupportedSignatureAlgorithm); - } - // Similarly, hash functions with an output length less than 256 bits - // MUST NOT be used, due to the possibility of collision attacks. - // In particular, MD5 and SHA1 MUST NOT be used. - RSA_PKCS1_SHA1 => return Err(Error::UnsupportedSignatureAlgorithm), - ECDSA_SHA1_Legacy => return Err(Error::UnsupportedSignatureAlgorithm), - Unknown(_) => return Err(Error::UnsupportedSignatureAlgorithm), - }; - let spki = &self.certificate.tbs_certificate.subject_pki; - let key = - signature::UnparsedPublicKey::new(verification_algorithm, spki.subject_public_key.data); - Ok(key) - } - /// Verify the `signature` of the `message` signed by the private key corresponding to the public key stored - /// in the certificate. - pub fn verify_signature( - &self, - signature_scheme: rustls::SignatureScheme, - message: &[u8], - signature: &[u8], - ) -> Result<(), webpki::Error> { - let pk = self.public_key(signature_scheme)?; - pk.verify(message, signature) - .map_err(|_| webpki::Error::InvalidSignatureForPublicKey)?; - Ok(()) - } -} - -#[cfg(test)] -mod tests { - use crate::tls::certificate; - #[test] - fn sanity_check() { - let keypair = libp2p_core::identity::Keypair::generate_ed25519(); - let cert = certificate::make_certificate(&keypair).unwrap(); - let cert_der = cert.serialize_der().unwrap(); - let parsed_cert = certificate::parse_certificate(&cert_der).unwrap(); - assert!(parsed_cert.verify().is_ok()); - assert_eq!(keypair.public(), parsed_cert.extension.public_key); - } - - macro_rules! check_cert { - ($name:ident, $path:literal, $scheme:path) => { - #[test] - fn $name() { - let cert: &[u8] = include_bytes!($path); - - let cert = certificate::parse_certificate(cert).unwrap(); - assert!(cert.verify().is_err()); // Because p2p extension - // was not signed with the private key - // of the certificate. - assert_eq!(cert.signature_scheme(), Ok($scheme)); - } - }; - } - - check_cert! {ed448, "./test_assets/ed448.der", rustls::SignatureScheme::ED448} - check_cert! {ed25519, "./test_assets/ed25519.der", rustls::SignatureScheme::ED25519} - check_cert! {rsa_pkcs1_sha256, "./test_assets/rsa_pkcs1_sha256.der", rustls::SignatureScheme::RSA_PKCS1_SHA256} - check_cert! {rsa_pkcs1_sha384, "./test_assets/rsa_pkcs1_sha384.der", rustls::SignatureScheme::RSA_PKCS1_SHA384} - check_cert! {rsa_pkcs1_sha512, "./test_assets/rsa_pkcs1_sha512.der", rustls::SignatureScheme::RSA_PKCS1_SHA512} - check_cert! {nistp256_sha256, "./test_assets/nistp256_sha256.der", rustls::SignatureScheme::ECDSA_NISTP256_SHA256} - check_cert! {nistp384_sha384, "./test_assets/nistp384_sha384.der", rustls::SignatureScheme::ECDSA_NISTP384_SHA384} - check_cert! {nistp521_sha512, "./test_assets/nistp521_sha512.der", rustls::SignatureScheme::ECDSA_NISTP521_SHA512} - - #[test] - fn rsa_pss_sha384() { - let cert: &[u8] = include_bytes!("./test_assets/rsa_pss_sha384.der"); - - let cert = certificate::parse_certificate(cert).unwrap(); - cert.verify().unwrap(); // that was a fairly generated certificate. - assert_eq!( - cert.signature_scheme(), - Ok(rustls::SignatureScheme::RSA_PSS_SHA384) - ); - } - - #[test] - fn nistp384_sha256() { - let cert: &[u8] = include_bytes!("./test_assets/nistp384_sha256.der"); - - let cert = certificate::parse_certificate(cert).unwrap(); - assert!(cert.signature_scheme().is_err()); - } -} diff --git a/transports/quic/src/tls/mod.rs b/transports/quic/src/tls/mod.rs deleted file mode 100644 index 78b31a7d4e9..00000000000 --- a/transports/quic/src/tls/mod.rs +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright 2021 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 QUIC based on libp2p TLS specs. - -pub(crate) mod certificate; -mod verifier; - -use std::sync::Arc; -use thiserror::Error; - -use rustls::{ - cipher_suite::{ - TLS13_AES_128_GCM_SHA256, TLS13_AES_256_GCM_SHA384, TLS13_CHACHA20_POLY1305_SHA256, - }, - SupportedCipherSuite, -}; - -/// A list of the TLS 1.3 cipher suites supported by rustls. -// By default rustls creates client/server configs with both -// TLS 1.3 __and__ 1.2 cipher suites. But we don't need 1.2. -static TLS13_CIPHERSUITES: &[SupportedCipherSuite] = &[ - // TLS1.3 suites - TLS13_CHACHA20_POLY1305_SHA256, - TLS13_AES_256_GCM_SHA384, - TLS13_AES_128_GCM_SHA256, -]; - -const P2P_ALPN: [u8; 6] = *b"libp2p"; - -/// Error creating a configuration -#[derive(Debug, Error)] -pub enum ConfigError { - /// TLS private key or certificate rejected - #[error("TLS private or certificate key rejected: {0}")] - TLSError(#[from] rustls::Error), - /// Certificate generation error - #[error("Certificate generation error: {0}")] - RcgenError(#[from] rcgen::RcgenError), -} - -/// Create a TLS client configuration for libp2p. -pub fn make_client_config( - keypair: &libp2p_core::identity::Keypair, -) -> Result { - let (certificate, key) = make_cert_key(keypair)?; - let verifier = Arc::new(verifier::Libp2pCertificateVerifier); - let mut crypto = rustls::ClientConfig::builder() - .with_cipher_suites(TLS13_CIPHERSUITES) - .with_safe_default_kx_groups() - .with_protocol_versions(&[&rustls::version::TLS13]) - .expect("Cipher suites and kx groups are configured; qed") - .with_custom_certificate_verifier(verifier) - .with_single_cert(vec![certificate], key) - .expect("Client cert key DER is valid; qed"); - crypto.alpn_protocols = vec![P2P_ALPN.to_vec()]; - Ok(crypto) -} - -/// Create a TLS server configuration for libp2p. -pub fn make_server_config( - keypair: &libp2p_core::identity::Keypair, -) -> Result { - let (certificate, key) = make_cert_key(keypair)?; - let verifier = Arc::new(verifier::Libp2pCertificateVerifier); - let mut crypto = rustls::ServerConfig::builder() - .with_cipher_suites(TLS13_CIPHERSUITES) - .with_safe_default_kx_groups() - .with_protocol_versions(&[&rustls::version::TLS13]) - .expect("Cipher suites and kx groups are configured; qed") - .with_client_cert_verifier(verifier) - .with_single_cert(vec![certificate], key) - .expect("Server cert key DER is valid; qed"); - crypto.alpn_protocols = vec![P2P_ALPN.to_vec()]; - Ok(crypto) -} - -/// Create a random private key and certificate signed with this key for rustls. -fn make_cert_key( - keypair: &libp2p_core::identity::Keypair, -) -> Result<(rustls::Certificate, rustls::PrivateKey), ConfigError> { - let cert = certificate::make_certificate(keypair)?; - let private_key = cert.serialize_private_key_der(); - let cert = rustls::Certificate(cert.serialize_der()?); - let key = rustls::PrivateKey(private_key); - Ok((cert, key)) -} diff --git a/transports/quic/src/tls/test_assets/ed25519.der b/transports/quic/src/tls/test_assets/ed25519.der deleted file mode 100644 index 494a199561a67047c63aa847ebd5a734d664a974..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 324 zcmXqLVstQQ{JemfiIIs(gsZ!KuJPrk8h>J!?Gd%_Vy}DSeZYW~jafUjz<|L(PMp`s z(9p=x)WFctz{E5P$Tb2oO`u$$3N5H&W<`by-3L?mpZR?`CQ59!Z)V-Wt`7G%!P(Va z`JM^-Zl^sCEv`4HHK=Ce(q?01VQgL$#RvrdS+Wc=SX4L|g%s|mOgtj`mczUK#Rs0- zUXDkYWBx7udC6Aw|C|-mpZ~qX&*Cs;#k`<1Dt|S%Y?=E^^l+_ObJC0J#}7>VU2{SB z&J?C_tuGxZ4gZd^A3k8ap-S(B*rsiTF6wtQK36_yos(}Zx|Omf`T3iC8RwaGO^j7t(-$c*dCA@9-p7CM LTr=(RtS{UEyA_6f diff --git a/transports/quic/src/tls/test_assets/ed448.der b/transports/quic/src/tls/test_assets/ed448.der deleted file mode 100644 index c74123868473acbc8b680c478d80aabc7371d6b7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 400 zcmXqLV(c+!V&qxC%*4pVB*MQwaM@J6(&HYkoYmTlks*D;u+RYM}vxft)z6 zk)ffHp{aqPp@E5M6p(8KWST&^Ko!nV#mrU=*VyfM-|{N)?>iS8a&+&3Y3u()K5aR+ zIm*WxUn14%uUb0pFKWD}C=YQ|;vp7sy zF)!$h%3sYbTjo9!JzT5Sob=-Q@dML-*IW?3GleN!>q|#U!@r~KhY#3psM0$jwrN|T zi~1dn&y^2a=j2<9?q-Ggp_rlWg<_CUmqNG0l_FrDK;;PW5BCaUp$h2rkL mO`5a*>{jW{X%6E$_KeR(Eb8+q&&WykHGDhu;xlitFaQ9G!K6U| diff --git a/transports/quic/src/tls/test_assets/gen.sh b/transports/quic/src/tls/test_assets/gen.sh deleted file mode 100755 index 4b7718874dd..00000000000 --- a/transports/quic/src/tls/test_assets/gen.sh +++ /dev/null @@ -1,63 +0,0 @@ -#ED25519 (works): -openssl genpkey -algorithm ed25519 -out privateKey.key -openssl req -new -subj="/" -key privateKey.key -out req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out ed25519.der - -#ED448 (works): -openssl genpkey -algorithm ed448 -out privateKey.key -openssl req -new -subj="/" -key privateKey.key -out req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out ed448.der - -#RSA_PKCS1_SHA256 (works): -openssl genpkey -algorithm rsa -out privateKey.key -openssl req -new -subj="/" -key privateKey.key -out req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -sha256 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out rsa_pkcs1_sha256.der - -#RSA_PKCS1_SHA384 (works): -# reuse privateKey.key and req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -sha384 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out rsa_pkcs1_sha384.der - -#RSA_PKCS1_SHA512 (works): -# reuse privateKey.key and req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -sha512 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out rsa_pkcs1_sha512.der - -#RSA-PSS TODO -# openssl genpkey -algorithm rsa-pss -pkeyopt rsa_keygen_bits:2048 -pkeyopt rsa_keygen_pubexp:3 -out privateKey.key -# # -sigopt rsa_pss_saltlen:20 -# # -sigopt rsa_padding_mode:pss -# # -sigopt rsa_mgf1_md:sha256 -# openssl req -x509 -nodes -days 365 -subj="/" -key privateKey.key -sha256 -sigopt rsa_pss_saltlen:20 -sigopt rsa_padding_mode:pss -sigopt rsa_mgf1_md:sha256 -out certificate.crt - -#ECDSA_NISTP256_SHA256 (works): -openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-256 -out privateKey.key -openssl req -new -subj="/" -key privateKey.key -out req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -sha256 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out nistp256_sha256.der - -#ECDSA_NISTP384_SHA384 (works): -openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-384 -out privateKey.key -openssl req -new -subj="/" -key privateKey.key -out req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -sha384 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out nistp384_sha384.der - -#ECDSA_NISTP521_SHA512 (works): -openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-521 -out privateKey.key -openssl req -new -subj="/" -key privateKey.key -out req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -sha512 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out nistp521_sha512.der - -#ECDSA_NISTP384_SHA256 (must fail): -openssl genpkey -algorithm EC -pkeyopt ec_paramgen_curve:P-384 -out privateKey.key -openssl req -new -subj="/" -key privateKey.key -out req.pem -openssl x509 -req -in req.pem -signkey privateKey.key -sha256 -out certificate.crt -extensions p2p_ext -extfile ./openssl.cfg -openssl x509 -outform der -in certificate.crt -out nistp384_sha256.der - - -# Remove tmp files - -rm req.pem certificate.crt privateKey.key diff --git a/transports/quic/src/tls/test_assets/nistp256_sha256.der b/transports/quic/src/tls/test_assets/nistp256_sha256.der deleted file mode 100644 index 8023645e9b07e58ab410f71564699cc8433aebe8..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 388 zcmXqLVr(#IVpLzi%*4pVBoeuh$A9gw`qI<8*#%$TwHM>@k+^8U#m1r4=5fxJg_+5K z!9Y%&*T~S&$k5cl(9podGz!Qy0y0gYT%d|b17S9Huns0hs8(i1c4j9A7AM7ZKl|>U z^|{d|0l!Z7IyXK-&0$Y zm%ozhEISpuY;nCotwA*#mo^(C3uE)5C`KUo&yr=3!J@*!D5P*dW#SRhw;bN>FFx?x z_HsPJ9P@AK&r7zd|L3fj{`~J9einx*E9M2AQTeO6Wy{=WqK9kMnv-5!KYn1^@0tt3 zccw6fYklcRY4~@P{qOzsUR(cP@bVZrRqVBo@}pf&g3 zq6+cO<~!|rv8ifqPo<1>?AvUWbS{K0Vf^U+O^8XswRaWE=@pM=?bs@wcDHSHh2YAC STQ@y5C|BTrHNV5|x;FscJeIfs diff --git a/transports/quic/src/tls/test_assets/nistp384_sha256.der b/transports/quic/src/tls/test_assets/nistp384_sha256.der deleted file mode 100644 index 5d76fa8f4a90ca3bba0a22150e4805d2ca9380a7..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 450 zcmXqLV%%rY#OShsnTe5!Nkl#(dWKqA?wTq_--bescs|x?tDhTiv2kd%d7QIlVP-O5 zFpv}HH8M0bGBhV>ja9pmg+YlqiGjuNWmseH z9Le0PpTAFUy=}kTa^Zx4Gk;#~J0!ko+G1y4v%mg-uB*G(GR7A_TK-f+tgawWX#0Bc zy0kNYFF)Xu$jO|1jxBMnk=@6YGcR`iHDl^K=hTvt*{`|GCcHx6NoM+k#q|cY2Gwj_ z+H8z0jLnOp7=hqFOO`cq%ekGextUSCuds7>a(4+aRfv`!C~?lwexjwC0r@ jpXU1e{N8I>`jY#B^pUBD$~Q(%uW#7c(2*{x@@WzPu{5%@ diff --git a/transports/quic/src/tls/test_assets/nistp384_sha384.der b/transports/quic/src/tls/test_assets/nistp384_sha384.der deleted file mode 100644 index a81a5ce1ab748be7714c385ae4f3525bd9024fd2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 450 zcmXqLV%%rY#OShsnTe5!Nu=gc^!`1Y^>UiEDTD_NenC=ldlAS z-lX}{j&IKH@XKv(|9?$h-Fdi~adXzn9MP9gnKspYlUZqhJm<*YwpSbzECPgz*Js`? z7y56bbH40i%;WC(w-ZZ!x9{fPQ!RKkN9lLsuMDs24O1RX+Q-5+r|-@alg0H0wFcE} zT-t1mER4;Iq8NeTKTDQD28#*@qmaV=l!-?~-*R}jzxcp&+sp9?bIiY`KQGy;{-3jA z`t!ec_*opLte6*cM&+;OmMwFii5{+1YfgG`{rG`tziTcC-}PyRYgN zC(DVX8|v-Lrrsz%C=+1Fx=}fyNxddUMXYWeTi^!~CIio?eBN}+>C6G^l>dC5Y#Z=n lNz%U>Th1%0D~@{HF0c6_Joi(}?Ch7f&pvow&e6Qe1OS~`yc+-j diff --git a/transports/quic/src/tls/test_assets/nistp521_sha512.der b/transports/quic/src/tls/test_assets/nistp521_sha512.der deleted file mode 100644 index 2846361f278e37f4338e35848304af02af4721e5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 525 zcmXqLV&XJtV$52=%*4pVBqDR5TjaPvpWN2z&XlP(!8U^GU0huOHE>K0|Yy$x{cCZ#EMmARMMivHT=EgP#7Dka- zL6Wn6m|iqy516L5b>s6TtB%xP{c$aQ?$V%ZPgi$sZ+4p}wtwyY;FcE`)-f*XWvR1avZ&&|oGI`uZV*mRCW#>&gT-yYb~l)AT5b#c8xtwA*#mo^(C3uE)5C`KUo z&yr=3!J@*!D5P*dW#SRhw;bN>FFx?x_HsPJ9P@AK&r7zd|L3fj{`~J9einx*E9M2A zQTeO6Wy{=WqK9kMnv-5!KYn1^@0tt3ccw6fYklcRY4~@P{qOzsUR(cP@b!O7g%0}QSXCMSj|%2T3ly_8jatJb(T)^6+0-`4B(ZMk|iKikOg zE7wV^x6;2mb-V1e$mx?#HTE(@+^))3nl5JWdGW>YsO9R{W@R)mIWa!J;`~DKE5koE z<)Zly_Dr-$&9`PRS9U2HPnZ=8L8z9UOtY2y{g=1#{X%U7@a7CkZ0 LW8!JtW5NgkxUAVe diff --git a/transports/quic/src/tls/test_assets/openssl.cfg b/transports/quic/src/tls/test_assets/openssl.cfg deleted file mode 100644 index 62f02baee8b..00000000000 --- a/transports/quic/src/tls/test_assets/openssl.cfg +++ /dev/null @@ -1,6 +0,0 @@ -[ p2p_ext ] -1.3.6.1.4.1.53594.1.1 = critical,ASN1:SEQUENCE:ExtBody - -[ ExtBody ] -pubkey = FORMAT:HEX,OCTETSTRING:08011220DF6491C415ED084B87E8F00CDB4A41C4035CFEA5F9D23D25FF9CA897E7FDDC0F -signature = FORMAT:HEX,OCTETSTRING:94A89E52CC24FD29B4B49DE615C37D268362E8D7C7C096FB7CD013DC9402572AF4886480FEC507C3C03DB07A2EC816B2B6714427DC28F379E0859C6F3B15BB05 diff --git a/transports/quic/src/tls/test_assets/pkcs1_sha256.der b/transports/quic/src/tls/test_assets/pkcs1_sha256.der deleted file mode 100644 index 0449728ee28cbf651c604319dde98adccf09a972..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 324 zcmXqLVstQQ{JemfiIIs(M8EgZMO6_-S-n@gy3W5hD&ec=e{I0Z#;l!MV8CD?C(dhR zXlP_;Vq|D!YG4=z2DJv&Y+TxGj4X`Ji=r5T;6F>2K?aKo2cwX}{gjDEMBj3Fx4-zn zbKA@D2y@K8r9Us(s{Ws|V*2yHclcQxrmUD3bVlW`=9VpUpNSr>RclUqasBv#X}@bO z2;Z5)6t4B9BchN82d#7RtwndU!kp^JaLP?jrthO} zWJs;p?nk8;%NP1Nmzb63-iWqi3$a_aT_`a)YN6m+m89z*Z?;?i{eM(y=|t71f4*23 M%(xjA&LNQU!^hIZ7oGD#4|0^Herbh^>%92}FkhHa7utKm zX+DXLrckXxX&9^4=l>jM(E6~^c2D%J0 z@2gk9aBCcbJH4MnIN0VkvFc0+_}*^0VU|*QD&PtUG8S1y_DSehdfJ@7z~@G674Iu1 zLUEn2Bwz&)^fF31+LwsAx!+$i&T;DwxTI~BVHFbxI? zDuzgg_YDC73k3iJf&l>ls{O>va6j?)%|O$*2S_6zx_OsT=e})EL9`9%VdYzD?;oNlFtjR)LJf9x2dSf3xgE<8j*6V5KhZ z1kw>ELCebYN!v&&LNQU!^hIZ7oGD#4|0^Herbh^>%92}FkhHa7utKm zX+DXLrckXxX&9^4=l>jM(E6~^c2D%J0 z@2gk9aBCcbJH4MnIN0VkvFc0+_}*^0VU|*QD&PtUG8S1y_DSehdfJ@7z~@G674Iu1 zLUEn2Bwz&)^fF31+LwsAx!+$i&T;DwxTI~BVHFbxI? zDuzgg_YDC73lAYYhX=D?z&@=>dr?(eP@!eG}fqOFo%}ZW#*l76S6i> zZ1KzN`906wsd>zF`I^0zaN}~+b}pHlqZglMunBk($nm+p9it)W3g5lR!##vLawn|v z60*daTN04wlT8`+hs_S$UNc8*1%WZ6snSs?x0f=yQ&?r!4u{s?A=+ML3A1pxkgy^_ zm}b+S!ZuVb7#oKWf@~z4H2W^Luvc;>FKAMJ1*`5u=xBKDF+pTLQUYDmcW8DIPJ@W; PfHNE258&LNQU!^hIZ7oGD#4|0^Herbh^>%92}FkhHa7utKm zX+DXLrckXxX&9^4=l>jM(E6~^c2D%J0 z@2gk9aBCcbJH4MnIN0VkvFc0+_}*^0VU|*QD&PtUG8S1y_DSehdfJ@7z~@G674Iu1 zLUEn2Bwz&)^fF31+LwsAx!+$i&T;DwxTI~BVHFbxI? zDuzgg_YDC74Fv!Lf&l>lBDp_*l%n5u2o|HCz*VVC^{6CJm2jtSz-!c}4ao5QWO?hx z)1V(q2CcODGd4Jo^_){yo%|YxwKz4L#8LM{&w(PNI%VZ=vg(V|u1hz|-?aC|(Su}p z4BWTyQ$#^356{ToI-QLQfcf~WrZMAt^HL|tHAA#Iei|H-TlJDia@lNbKdt6QXt^mm z=@DU`6NPW5I%k5fQnftOKT<9-z+E=NI|#!P8V&W%HogIa>E)jUk61O_DT@suIFc18 zR0rsQrH7CX$>dO9665jGrO!rm%p2%zn_ZjXoLHSakH}9!39870w0HHFz{|#&(B{FI z%FM#V#LBQx#y|?8f&)!<5zsIL0|o;n34Q|u14A$bG7Jo&B>0UCjSLOUjDVD>fuT_p zP*oG75=cMHI!0Co<|amdkT@4p6C)$TwQ$SX3P%e9pUOXD@ikd{+^y;P@x5yvdjIj8 z--{%t>mA@2B}B&UzG|A?kMT%+b986Wu0Bay^@I z^H1}>O$k;EOP5c0bh)fa$Ys&vq<$ys`@1YS?kZ+Zzs+1{m*e6x=k)@HAFpnPPTj5k zxq8aqPb_=P!$iz}Iu}=eepQukJmF8fzwKwfO$mI@`2tJ5&ITB0c|K#iuahsnb4|an zkH3n4%6>MDEY>X_7 z&5NQK8CkLnGFVhN7=;w}XFX-n+bWZ?{g!*pF2|ruM!%Eq{ie){6`j>#CBN4*hsEKX z(e?!u4wLP6G}+0!|K53&eWU-mP332giU+;EsS;LmxqjioGx=Nxt~Z~tZg=~8Bv^3I zt&J58-^#z#zWc;JnSqrG$pILrj0_AJ{kQu*S3hp{nrfoiTfHWJVdLk#=;iZ%hsDJ9 zZU}t%C__JM$H9_26YD*Av^q~Ribe^ybJg$7P7%9!?kH#f=Z(?V54E_?+iW#kNn-9} znPlJVnph{l~n?D^Fb!O|zdKy;*vejm#bg zYh8Oy_kw*#gsg6#-uXeK>1+75vu}O=rcc_-?dX4W#;3{lb2t1Ky{qmTvHg8d!nUr0 zO@D$fp8L^ZAkF_=RAApphvPFqd%_=&@}5{08G6=uNA#6FW$(YK lNgdWa{IqpPkGqoH+}o-4D^nglx@N$}K5+wIlp(ixFaY)iXKMfe diff --git a/transports/quic/src/tls/verifier.rs b/transports/quic/src/tls/verifier.rs deleted file mode 100644 index f7592816917..00000000000 --- a/transports/quic/src/tls/verifier.rs +++ /dev/null @@ -1,213 +0,0 @@ -// Copyright 2021 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 1.3 certificates and handshakes handling for libp2p -//! -//! This module handles a verification of a client/server certificate chain -//! and signatures allegedly by the given certificates. - -use crate::tls::certificate::parse_certificate; -use rustls::{ - client::{HandshakeSignatureValid, ServerCertVerified, ServerCertVerifier}, - internal::msgs::handshake::DigitallySignedStruct, - server::{ClientCertVerified, ClientCertVerifier}, - Certificate, DistinguishedNames, SignatureScheme, -}; - -/// 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. -impl Libp2pCertificateVerifier { - /// Return the list of SignatureSchemes that this verifier will handle, - /// in `verify_tls12_signature` and `verify_tls13_signature` calls. - /// - /// This should be in priority order, with the most preferred first. - pub fn verification_schemes() -> Vec { - vec![ - // TODO SignatureScheme::ECDSA_NISTP521_SHA512 is not supported by `ring` yet - SignatureScheme::ECDSA_NISTP384_SHA384, - SignatureScheme::ECDSA_NISTP256_SHA256, - // TODO SignatureScheme::ED448 is not supported by `ring` yet - SignatureScheme::ED25519, - // In particular, RSA SHOULD NOT be used unless - // no elliptic curve algorithms are supported. - SignatureScheme::RSA_PSS_SHA512, - SignatureScheme::RSA_PSS_SHA384, - SignatureScheme::RSA_PSS_SHA256, - SignatureScheme::RSA_PKCS1_SHA512, - SignatureScheme::RSA_PKCS1_SHA384, - SignatureScheme::RSA_PKCS1_SHA256, - ] - } -} - -impl ServerCertVerifier for Libp2pCertificateVerifier { - fn verify_server_cert( - &self, - end_entity: &Certificate, - intermediates: &[Certificate], - _server_name: &rustls::ServerName, - _scts: &mut dyn Iterator, - _ocsp_response: &[u8], - _now: std::time::SystemTime, - ) -> Result { - verify_presented_certs(end_entity, intermediates)?; - - Ok(ServerCertVerified::assertion()) - } - - fn verify_tls12_signature( - &self, - _message: &[u8], - _cert: &Certificate, - _dss: &DigitallySignedStruct, - ) -> Result { - // The libp2p handshake uses TLS 1.3 (and higher). - // Endpoints MUST NOT negotiate lower TLS versions. - Err(rustls::Error::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(cert, dss.scheme, message, dss.sig.0.as_ref()) - } - - fn supported_verify_schemes(&self) -> Vec { - Self::verification_schemes() - } -} - -/// 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. -impl ClientCertVerifier for Libp2pCertificateVerifier { - fn offer_client_auth(&self) -> bool { - true - } - - fn client_auth_root_subjects(&self) -> Option { - Some(vec![]) - } - - fn verify_client_cert( - &self, - end_entity: &Certificate, - intermediates: &[Certificate], - _now: std::time::SystemTime, - ) -> Result { - verify_presented_certs(end_entity, intermediates)?; - - Ok(ClientCertVerified::assertion()) - } - - fn verify_tls12_signature( - &self, - _message: &[u8], - _cert: &Certificate, - _dss: &DigitallySignedStruct, - ) -> Result { - // The libp2p handshake uses TLS 1.3 (and higher). - // Endpoints MUST NOT negotiate lower TLS versions. - Err(rustls::Error::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(cert, dss.scheme, message, dss.sig.0.as_ref()) - } - - fn supported_verify_schemes(&self) -> Vec { - Self::verification_schemes() - } -} - -/// When receiving the certificate chain, an endpoint -/// MUST check these conditions and abort the connection attempt if -/// (a) the presented certificate is not yet valid, OR -/// (b) if it is expired. -/// Endpoints MUST abort the connection attempt if more than one certificate is received, -/// or if the certificate’s self-signature is not valid. -fn verify_presented_certs( - end_entity: &Certificate, - intermediates: &[Certificate], -) -> Result<(), rustls::Error> { - if !intermediates.is_empty() { - return Err(rustls::Error::General( - "libp2p-tls requires exactly one certificate".into(), - )); - } - - parse_certificate(end_entity.as_ref()) - .map_err(pki_error)? - .verify() - .map_err(pki_error)?; - - Ok(()) -} - -fn verify_tls13_signature( - cert: &Certificate, - signature_scheme: SignatureScheme, - message: &[u8], - signature: &[u8], -) -> Result { - parse_certificate(cert.as_ref()) - .map_err(pki_error)? - .verify_signature(signature_scheme, message, signature) - .map_err(pki_error)?; - - Ok(HandshakeSignatureValid::assertion()) -} - -fn pki_error(error: webpki::Error) -> rustls::Error { - use webpki::Error::*; - match error { - BadDer | BadDerTime => rustls::Error::InvalidCertificateEncoding, - InvalidSignatureForPublicKey => rustls::Error::InvalidCertificateSignature, - UnsupportedSignatureAlgorithm | UnsupportedSignatureAlgorithmForPublicKey => { - rustls::Error::InvalidCertificateSignatureType - } - e => rustls::Error::InvalidCertificateData(format!("invalid peer certificate: {}", e)), - } -} From eb1b2cde413a7fb4d7f00d9eefbb2af10f77b82e Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 24 Oct 2022 16:33:22 +0200 Subject: [PATCH 194/218] quic: finalize read chunks before return error --- transports/quic/src/connection/substream.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/connection/substream.rs b/transports/quic/src/connection/substream.rs index 5b4ed518dc3..a4fe9d7c3b0 100644 --- a/transports/quic/src/connection/substream.rs +++ b/transports/quic/src/connection/substream.rs @@ -97,6 +97,7 @@ impl AsyncRead for Substream { let mut bytes = 0; let mut pending = false; + let mut error = None; loop { if buf.is_empty() { // Chunks::next will continue returning `Ok(Some(_))` with an @@ -108,7 +109,8 @@ impl AsyncRead for Substream { Ok(Some(chunk)) => chunk, Ok(None) => break, Err(err @ quinn_proto::ReadError::Reset(_)) => { - return Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) + error = Some(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))); + break; } Err(quinn_proto::ReadError::Blocked) => { pending = true; @@ -124,13 +126,17 @@ impl AsyncRead for Substream { waker.wake(); } } + if let Some(err) = error { + return Poll::Ready(err); + } + if pending && bytes == 0 { let substream_state = state.unchecked_substream_state(self.id); substream_state.read_waker = Some(cx.waker().clone()); - Poll::Pending - } else { - Poll::Ready(Ok(bytes)) + return Poll::Pending; } + + Poll::Ready(Ok(bytes)) } } From 55ccdb3f30a5addb6ed7bd2aee60cb08351814c2 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 24 Oct 2022 16:33:38 +0200 Subject: [PATCH 195/218] quic: improve logs --- transports/quic/src/endpoint.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 78155ac286f..b9e7fb07218 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -509,8 +509,8 @@ impl Driver

{ Err(err) if err.is_full() => { // Connection is too busy. Drop the datagram to back-pressure the remote. log::debug!( - "Dropping {:?} because the connection's channel is full.", - err.into_inner() + "Dropping packet for connection {:?} because the connection's channel is full.", + connec_id ); } Err(_) => unreachable!("Error is either `Full` or `Disconnected`."), @@ -609,7 +609,7 @@ impl Future for Driver

{ // robust to packet losses and it is consequently not a logic error to proceed with // normal operations. Poll::Ready(Err(err)) => { - log::error!("Error while sending on QUIC UDP socket: {:?}", err); + log::warn!("Error while sending on QUIC UDP socket: {:?}", err); continue; } Poll::Pending => {} @@ -627,7 +627,7 @@ impl Future for Driver

{ // Errors on the socket are expected to never happen, and we handle them by // simply printing a log message. Poll::Ready(Err(err)) => { - log::error!("Error while receive on QUIC UDP socket: {:?}", err); + log::warn!("Error while receive on QUIC UDP socket: {:?}", err); continue; } Poll::Pending => {} From da2a6db28341c308b25f5bdf7e2b7147f67ce8b1 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 28 Oct 2022 22:11:16 +0200 Subject: [PATCH 196/218] quic: revert 4010be0 The clone is needed to guarantee at least one slot in the channel. --- 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 b9e7fb07218..4d195d68060 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -266,7 +266,7 @@ impl Channel { /// This clones the sender to the endpoint to guarantee delivery. /// This should *not* be called for regular messages. pub fn send_on_drop(&mut self, to_endpoint: ToEndpoint) { - let _ = self.to_endpoint.try_send(to_endpoint); + let _ = self.to_endpoint.clone().try_send(to_endpoint); } } From c17df6d72bc88ab51dff4da5775a91996c547bc6 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 28 Oct 2022 22:20:57 +0200 Subject: [PATCH 197/218] quic: stricter `is_quic_addr` check --- transports/quic/src/transport.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 4825dda1f1a..cb62a51e496 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -558,9 +558,14 @@ fn is_quic_addr(addr: &Multiaddr) -> bool { Some(p) => p, None => return false, }; + let fourth = iter.next(); + let fifth = iter.next(); + matches!(first, Ip4(_) | Ip6(_) | Dns(_) | Dns4(_) | Dns6(_)) && matches!(second, Udp(_)) && matches!(third, Quic) + && matches!(fourth, Some(P2p(_)) | None) + && matches!(fifth, None) } /// Turns an IP address and port into the corresponding QUIC multiaddr. From 58c3f67dc0d49890a407943c456b4e8ecee606a1 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Fri, 28 Oct 2022 22:21:24 +0200 Subject: [PATCH 198/218] quic/tests: add `tcp_and_quic` test --- transports/quic/Cargo.toml | 2 +- transports/quic/tests/smoke.rs | 88 +++++++++++++++++++++++++++++++++- 2 files changed, 87 insertions(+), 3 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 78a0d4b02a9..ab74b52217e 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -40,7 +40,7 @@ rustc-args = ["--cfg", "docsrs"] [dev-dependencies] async-std = { version = "1.12.0", features = ["attributes"] } -libp2p = { path = "../..", features = ["request-response"] } +libp2p = { path = "../..", features = ["request-response", "tcp", "yamux", "noise", "async-std"] } env_logger = "0.9.0" rand = "0.8.4" tokio = { version = "1.21.1", features = ["macros", "rt-multi-thread"] } diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index c0bfa8e88ce..a67a0757674 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -16,7 +16,9 @@ use libp2p::request_response::{ }; use libp2p::swarm::dial_opts::{DialOpts, PeerCondition}; use libp2p::swarm::{ConnectionError, DialError, Swarm, SwarmEvent}; -use libp2p::Multiaddr; +use libp2p::{noise, tcp, yamux, Multiaddr}; +use libp2p_core::either::EitherOutput; +use libp2p_core::transport::OrTransport; use libp2p_quic as quic; use quic::Provider; use rand::RngCore; @@ -566,7 +568,7 @@ async fn ipv4_dial_ipv6() { #[cfg(feature = "async-std")] #[async_std::test] -async fn wrong_peerod() { +async fn wrong_peerid() { use libp2p::PeerId; let _ = env_logger::try_init(); @@ -595,3 +597,85 @@ async fn wrong_peerod() { } } } + +#[cfg(feature = "async-std")] +fn new_tcp_quic_swarm() -> Swarm> { + let keypair = generate_tls_keypair(); + let peer_id = keypair.public().to_peer_id(); + let mut config = quic::Config::new(&keypair); + config.handshake_timeout = Duration::from_secs(1); + let quic_transport = quic::async_std::Transport::new(config); + let tcp_transport = tcp::async_io::Transport::new(tcp::Config::default()) + .upgrade(upgrade::Version::V1Lazy) + .authenticate( + noise::NoiseConfig::xx( + noise::Keypair::::new() + .into_authentic(&keypair) + .unwrap(), + ) + .into_authenticated(), + ) + .multiplex(yamux::YamuxConfig::default()); + + let transport = OrTransport::new(quic_transport, tcp_transport) + .map(|either_output, _| match either_output { + EitherOutput::First((peer_id, muxer)) => (peer_id, StreamMuxerBox::new(muxer)), + EitherOutput::Second((peer_id, muxer)) => (peer_id, StreamMuxerBox::new(muxer)), + }) + .boxed(); + + let protocols = iter::once((PingProtocol(), ProtocolSupport::Full)); + let cfg = RequestResponseConfig::default(); + let behaviour = RequestResponse::new(PingCodec(), protocols, cfg); + + Swarm::new(transport, behaviour, peer_id) +} + +#[cfg(feature = "async-std")] +#[async_std::test] +async fn tcp_and_quic() { + let mut swarm_a = new_tcp_quic_swarm(); + let swarm_a_id = *swarm_a.local_peer_id(); + println!("{}", swarm_a_id); + + let mut swarm_b = new_tcp_quic_swarm(); + + let quic_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/udp/0/quic").await; + let tcp_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/tcp/0").await; + + swarm_b.dial(quic_addr.clone()).unwrap(); + + loop { + select! { + ev = swarm_a.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { .. } => break, + SwarmEvent::IncomingConnection { .. } => { } + e => panic!("{:?}", e), + }, + ev = swarm_b.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { .. } => {}, + e => panic!("{:?}", e), + } + } + } + + swarm_b.dial(tcp_addr).unwrap(); + + loop { + select! { + ev = swarm_a.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { .. } => break, + SwarmEvent::IncomingConnection { .. } + | SwarmEvent::ConnectionClosed { .. } => { } + e => panic!("{:?}", e), + }, + ev = swarm_b.select_next_some() => match ev { + SwarmEvent::ConnectionEstablished { .. } => {}, + SwarmEvent::ConnectionClosed { endpoint, .. } => { + assert_eq!(endpoint.get_remote_address(), &quic_addr ); + } + e => panic!("{:?}", e), + } + } + } +} From d0e4cd7682f22843611fcbaafbdd42e8f4b81b32 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Tue, 1 Nov 2022 13:28:24 +1100 Subject: [PATCH 199/218] Remove unnecessary `async` --- transports/quic/tests/smoke.rs | 26 +++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index a67a0757674..b713bd8ee67 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -30,7 +30,7 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } -async fn create_swarm() -> Swarm> { +fn create_swarm() -> Swarm> { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let mut config = quic::Config::new(&keypair); @@ -72,8 +72,8 @@ async fn smoke() { let _ = env_logger::try_init(); let mut rng = rand::thread_rng(); - let mut a = create_swarm::

().await; - let mut b = create_swarm::

().await; + let mut a = create_swarm::

(); + let mut b = create_swarm::

(); let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; @@ -292,8 +292,8 @@ impl RequestResponseCodec for PingCodec { #[async_std::test] async fn dial_failure() { let _ = env_logger::try_init(); - let mut a = create_swarm::().await; - let mut b = create_swarm::().await; + let mut a = create_swarm::(); + let mut b = create_swarm::(); let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; @@ -338,7 +338,7 @@ fn concurrent_connections_and_streams() { // Spawn the listener nodes. for _ in 0..number_listeners { - let mut listener = pool.run_until(create_swarm::

()); + let mut listener = create_swarm::

(); let addr = pool.run_until(start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic")); listeners.push((*listener.local_peer_id(), addr)); @@ -383,7 +383,7 @@ fn concurrent_connections_and_streams() { .unwrap(); } - let mut dialer = pool.run_until(create_swarm::

()); + let mut dialer = create_swarm::

(); // For each listener node start `number_streams` requests. for (listener_peer_id, listener_addr) in &listeners { @@ -448,8 +448,8 @@ fn concurrent_connections_and_streams() { #[tokio::test] async fn endpoint_reuse() { let _ = env_logger::try_init(); - let mut swarm_a = create_swarm::().await; - let mut swarm_b = create_swarm::().await; + let mut swarm_a = create_swarm::(); + let mut swarm_b = create_swarm::(); let b_peer_id = *swarm_b.local_peer_id(); let a_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/udp/0/quic").await; @@ -540,8 +540,8 @@ async fn endpoint_reuse() { #[async_std::test] async fn ipv4_dial_ipv6() { let _ = env_logger::try_init(); - let mut swarm_a = create_swarm::().await; - let mut swarm_b = create_swarm::().await; + let mut swarm_a = create_swarm::(); + let mut swarm_b = create_swarm::(); let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; @@ -572,8 +572,8 @@ async fn wrong_peerid() { use libp2p::PeerId; let _ = env_logger::try_init(); - let mut swarm_a = create_swarm::().await; - let mut swarm_b = create_swarm::().await; + let mut swarm_a = create_swarm::(); + let mut swarm_b = create_swarm::(); let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; let a_id = *swarm_a.local_peer_id(); From 49134fb7ac10194224dbe992aae80b65742d2cc6 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Tue, 1 Nov 2022 15:07:30 +1100 Subject: [PATCH 200/218] Don't fail inside `AsyncWrite::close` --- transports/quic/src/connection/substream.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/transports/quic/src/connection/substream.rs b/transports/quic/src/connection/substream.rs index a4fe9d7c3b0..2ad0feb9d69 100644 --- a/transports/quic/src/connection/substream.rs +++ b/transports/quic/src/connection/substream.rs @@ -191,7 +191,10 @@ impl AsyncWrite for Substream { Poll::Ready(Err(io::Error::new(io::ErrorKind::ConnectionReset, err))) } Err(quinn_proto::FinishError::UnknownStream) => { - Poll::Ready(Err(io::ErrorKind::BrokenPipe.into())) + // We never make up IDs so the stream must have existed at some point if we get to here. + // `UnknownStream` is also emitted in case the stream is already finished, hence just + // return `Ok(())` here. + Poll::Ready(Ok(())) } } } From 8c50c9ba8ffa2eb8c14fb88a9a7955cefc662dc7 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Tue, 1 Nov 2022 15:08:23 +1100 Subject: [PATCH 201/218] Add muxer compliance test suite for QUIC --- transports/quic/Cargo.toml | 9 ++- transports/quic/tests/stream_compliance.rs | 78 ++++++++++++++++++++++ 2 files changed, 85 insertions(+), 2 deletions(-) create mode 100644 transports/quic/tests/stream_compliance.rs diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index ab74b52217e..d8af17bd080 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -40,8 +40,13 @@ rustc-args = ["--cfg", "docsrs"] [dev-dependencies] async-std = { version = "1.12.0", features = ["attributes"] } -libp2p = { path = "../..", features = ["request-response", "tcp", "yamux", "noise", "async-std"] } env_logger = "0.9.0" +libp2p = { path = "../..", features = ["request-response", "tcp", "yamux", "noise", "async-std"] } +libp2p-muxer-test-harness = { path = "../../muxers/test-harness" } +quickcheck = "1" rand = "0.8.4" tokio = { version = "1.21.1", features = ["macros", "rt-multi-thread"] } -quickcheck = "1" + +[[test]] +name = "stream_compliance" +required-features = ["async-std"] diff --git a/transports/quic/tests/stream_compliance.rs b/transports/quic/tests/stream_compliance.rs new file mode 100644 index 00000000000..c2b6f4bed85 --- /dev/null +++ b/transports/quic/tests/stream_compliance.rs @@ -0,0 +1,78 @@ +use futures::channel::oneshot; +use futures::StreamExt; +use libp2p_core::Transport; +use libp2p_quic as quic; +use std::time::Duration; + +#[async_std::test] +async fn close_implies_flush() { + let (alice, bob) = connected_peers().await; + + libp2p_muxer_test_harness::close_implies_flush(alice, bob).await; +} + +#[async_std::test] +#[ignore] // Hangs forever, same as yamux. We can't read from a stream that we have never written to. +async fn dialer_can_receive() { + let (alice, bob) = connected_peers().await; + + libp2p_muxer_test_harness::dialer_can_receive(alice, bob).await; +} + +#[async_std::test] +async fn read_after_close() { + let (alice, bob) = connected_peers().await; + + libp2p_muxer_test_harness::read_after_close(alice, bob).await; +} + +async fn connected_peers() -> (quic::Connection, quic::Connection) { + let mut dialer = new_transport().boxed(); + let mut listener = new_transport().boxed(); + + listener + .listen_on("/ip4/127.0.0.1/udp/0/quic".parse().unwrap()) + .unwrap(); + let listen_address = listener.next().await.unwrap().into_new_address().unwrap(); + + let (dialer_conn_sender, dialer_conn_receiver) = oneshot::channel(); + let (listener_conn_sender, listener_conn_receiver) = oneshot::channel(); + + async_std::task::spawn(async move { + let (upgrade, _) = listener.next().await.unwrap().into_incoming().unwrap(); + + async_std::task::spawn(async move { + let (_, connection) = upgrade.await.unwrap(); + + let _ = listener_conn_sender.send(connection); + }); + + loop { + listener.next().await; + } + }); + let dial_fut = dialer.dial(listen_address).unwrap(); + async_std::task::spawn(async move { + let connection = dial_fut.await.unwrap().1; + + let _ = dialer_conn_sender.send(connection); + }); + + async_std::task::spawn(async move { + loop { + dialer.next().await; + } + }); + + futures::future::try_join(dialer_conn_receiver, listener_conn_receiver) + .await + .unwrap() +} + +fn new_transport() -> quic::async_std::Transport { + let keypair = libp2p_core::identity::Keypair::generate_ed25519(); + let mut config = quic::Config::new(&keypair); + config.handshake_timeout = Duration::from_secs(1); + + quic::async_std::Transport::new(config) +} From 6fb98a221731e8618a4d96c95e6c0e970ee6c835 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Tue, 1 Nov 2022 18:17:06 +1100 Subject: [PATCH 202/218] Migrate tests away from `Swarm` --- transports/quic/Cargo.toml | 2 +- transports/quic/tests/smoke.rs | 799 ++++++++++++--------------------- 2 files changed, 279 insertions(+), 522 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index d8af17bd080..a5646d3feaf 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -41,7 +41,7 @@ rustc-args = ["--cfg", "docsrs"] [dev-dependencies] async-std = { version = "1.12.0", features = ["attributes"] } env_logger = "0.9.0" -libp2p = { path = "../..", features = ["request-response", "tcp", "yamux", "noise", "async-std"] } +libp2p = { path = "../..", features = ["tcp", "yamux", "noise", "async-std"] } libp2p-muxer-test-harness = { path = "../../muxers/test-harness" } quickcheck = "1" rand = "0.8.4" diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index b713bd8ee67..d9e582a2abd 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -1,57 +1,45 @@ #![cfg(any(feature = "async-std", feature = "tokio"))] -use async_trait::async_trait; -use futures::channel::oneshot; -use futures::future::{join, FutureExt}; -use futures::io::{AsyncRead, AsyncWrite, AsyncWriteExt}; -use futures::select; +use futures::channel::mpsc; +use futures::future::Either; use futures::stream::StreamExt; -use futures::task::Spawn; +use futures::{future, AsyncReadExt, AsyncWriteExt, SinkExt}; use libp2p::core::multiaddr::Protocol; -use libp2p::core::muxing::StreamMuxerBox; -use libp2p::core::{upgrade, ConnectedPoint, Transport}; -use libp2p::request_response::{ - ProtocolName, ProtocolSupport, RequestResponse, RequestResponseCodec, RequestResponseConfig, - RequestResponseEvent, RequestResponseMessage, -}; -use libp2p::swarm::dial_opts::{DialOpts, PeerCondition}; -use libp2p::swarm::{ConnectionError, DialError, Swarm, SwarmEvent}; +use libp2p::core::Transport; use libp2p::{noise, tcp, yamux, Multiaddr}; use libp2p_core::either::EitherOutput; -use libp2p_core::transport::OrTransport; +use libp2p_core::muxing::{StreamMuxerBox, StreamMuxerExt}; +use libp2p_core::transport::{Boxed, OrTransport, TransportEvent}; +use libp2p_core::{upgrade, PeerId}; use libp2p_quic as quic; use quic::Provider; use rand::RngCore; +use std::future::Future; +use std::io; use std::num::NonZeroU8; use std::time::Duration; -use std::{io, iter}; fn generate_tls_keypair() -> libp2p::identity::Keypair { libp2p::identity::Keypair::generate_ed25519() } -fn create_swarm() -> Swarm> { +fn create_transport() -> (PeerId, Boxed<(PeerId, StreamMuxerBox)>) { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let mut config = quic::Config::new(&keypair); config.handshake_timeout = Duration::from_secs(1); - let transport = quic::GenTransport::

::new(config); - let transport = Transport::map(transport, |(peer, connection), _| { - (peer, StreamMuxerBox::new(connection)) - }) - .boxed(); + let transport = quic::GenTransport::

::new(config) + .map(|(p, c), _| (p, StreamMuxerBox::new(c))) + .boxed(); - let protocols = iter::once((PingProtocol(), ProtocolSupport::Full)); - let cfg = RequestResponseConfig::default(); - let behaviour = RequestResponse::new(PingCodec(), protocols, cfg); - Swarm::new(transport, behaviour, peer_id) + (peer_id, transport) } -async fn start_listening(swarm: &mut Swarm>, addr: &str) -> Multiaddr { - swarm.listen_on(addr.parse().unwrap()).unwrap(); - match swarm.next().await { - Some(SwarmEvent::NewListenAddr { address, .. }) => address, +async fn start_listening(transport: &mut Boxed<(PeerId, StreamMuxerBox)>, addr: &str) -> Multiaddr { + transport.listen_on(addr.parse().unwrap()).unwrap(); + match transport.next().await { + Some(TransportEvent::NewAddress { listen_addr, .. }) => listen_addr, e => panic!("{:?}", e), } } @@ -70,543 +58,292 @@ async fn async_std_smoke() { async fn smoke() { let _ = env_logger::try_init(); - let mut rng = rand::thread_rng(); - - let mut a = create_swarm::

(); - let mut b = create_swarm::

(); - - let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; - let mut data = vec![0; 4096 * 10]; - rng.fill_bytes(&mut data); + let (a_peer_id, mut a_transport) = create_transport::

(); + let (b_peer_id, mut b_transport) = create_transport::

(); - b.behaviour_mut().add_address(a.local_peer_id(), addr); - b.behaviour_mut() - .send_request(a.local_peer_id(), Ping(data.clone())); + let addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, addr).await; - let b_id = *b.local_peer_id(); - - let (sync_tx, sync_rx) = oneshot::channel(); + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); +} - let fut_a = async move { - match a.next().await { - Some(SwarmEvent::IncomingConnection { .. }) => {} - e => panic!("{:?}", e), - }; +#[cfg(feature = "async-std")] +#[async_std::test] +async fn dial_failure() { + let _ = env_logger::try_init(); + let mut a = create_transport::().1; + let mut b = create_transport::().1; - match a.next().await { - Some(SwarmEvent::ConnectionEstablished { .. }) => {} - e => panic!("{:?}", e), - }; + let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; + drop(a); // stop a so b can never reach it - 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 dial(&mut b, addr).await { + Ok(_) => panic!("Expected dial to fail"), + Err(error) => { + assert_eq!("Handshake with the remote timed out.", error.to_string()) } + }; +} - match a.next().await { - Some(SwarmEvent::Behaviour(RequestResponseEvent::ResponseSent { .. })) => {} - e => panic!("{:?}", e), - } +// Note: This test should likely be ported to the muxer compliance test suite. +#[cfg(feature = "async-std")] +#[test] +fn concurrent_connections_and_streams_async_std() { + let _ = env_logger::try_init(); - a.behaviour_mut() - .send_request(&b_id, Ping(b"another substream".to_vec())); - - assert!(a.next().now_or_never().is_none()); - - 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), - } + quickcheck::QuickCheck::new() + .min_tests_passed(1) + .quickcheck(prop:: as fn(_, _) -> _); +} - sync_rx.await.unwrap(); +// Note: This test should likely be ported to the muxer compliance test suite. +#[cfg(feature = "tokio")] +#[test] +fn concurrent_connections_and_streams_tokio() { + let _ = env_logger::try_init(); - a.disconnect_peer_id(b_id).unwrap(); + let rt = tokio::runtime::Runtime::new().unwrap(); + let _guard = rt.enter(); + quickcheck::QuickCheck::new() + .min_tests_passed(1) + .quickcheck(prop:: as fn(_, _) -> _); +} - match a.next().await { - Some(SwarmEvent::ConnectionClosed { cause: None, .. }) => {} - e => panic!("{:?}", e), - } - }; +fn prop( + number_listeners: NonZeroU8, + number_streams: NonZeroU8, +) -> quickcheck::TestResult { + const BUFFER_SIZE: usize = 4096 * 10; - let fut_b = async { - match b.next().await { - Some(SwarmEvent::Dialing(_)) => {} - e => panic!("{:?}", e), - } + let number_listeners = u8::from(number_listeners) as usize; + let number_streams = u8::from(number_streams) as usize; - match b.next().await { - Some(SwarmEvent::ConnectionEstablished { .. }) => {} - e => panic!("{:?}", e), - }; + if number_listeners > 10 || number_streams > 10 { + return quickcheck::TestResult::discard(); + } - assert!(b.next().now_or_never().is_none()); - - match b.next().await { - Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { - message: - RequestResponseMessage::Response { - response: Pong(pong), - .. - }, - .. - })) => assert_eq!(data, pong), - e => panic!("{:?}", e), - } + let (listeners_tx, mut listeners_rx) = mpsc::channel(number_listeners); - 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), - } + log::info!("Creating {number_streams} streams on {number_listeners} connections"); - match b.next().await { - Some(SwarmEvent::Behaviour(RequestResponseEvent::ResponseSent { .. })) => {} - e => panic!("{:?}", e), - } + // Spawn the listener nodes. + for _ in 0..number_listeners { + P::spawn({ + let mut listeners_tx = listeners_tx.clone(); - sync_tx.send(()).unwrap(); + async move { + let (peer_id, mut listener) = create_transport::

(); + let addr = start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic").await; - match b.next().await { - Some(SwarmEvent::ConnectionClosed { - cause: Some(ConnectionError::IO(_)), - .. - }) => {} - e => panic!("{:?}", e), - } - }; + listeners_tx.send((peer_id, addr)).await.unwrap(); - join(fut_a, fut_b).await; -} + loop { + if let TransportEvent::Incoming { upgrade, .. } = + listener.select_next_some().await + { + let (_, connection) = upgrade.await.unwrap(); -#[derive(Debug, Clone)] -struct PingProtocol(); + P::spawn(answer_inbound_streams::(connection)); + } + } + } + }) + } -#[derive(Clone)] -struct PingCodec(); + let (completed_streams_tx, completed_streams_rx) = + mpsc::channel(number_streams * number_listeners); -#[derive(Debug, Clone, PartialEq, Eq)] -struct Ping(Vec); + // For each listener node start `number_streams` requests. + P::spawn(async move { + let (_, mut dialer) = create_transport::

(); -#[derive(Debug, Clone, PartialEq, Eq)] -struct Pong(Vec); + while let Some((_, listener_addr)) = listeners_rx.next().await { + let (_, connection) = dial(&mut dialer, listener_addr.clone()).await.unwrap(); -impl ProtocolName for PingProtocol { - fn protocol_name(&self) -> &[u8] { - "/ping/1".as_bytes() - } -} + P::spawn(open_outbound_streams::( + connection, + number_streams, + completed_streams_tx.clone(), + )) + } -#[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 - } + // Drive the dialer. + loop { + dialer.next().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 - } + let completed_streams = number_streams * number_listeners; - 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(()) - } + // Wait for all streams to complete. + P::block_on( + completed_streams_rx + .take(completed_streams as usize) + .collect::>(), + ); - 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(()) - } + quickcheck::TestResult::passed() } -#[cfg(feature = "async-std")] -#[async_std::test] -async fn dial_failure() { - let _ = env_logger::try_init(); - let mut a = create_swarm::(); - let mut b = create_swarm::(); +async fn answer_inbound_streams( + mut connection: StreamMuxerBox, +) { + loop { + let mut inbound_stream = match future::poll_fn(|cx| { + let _ = connection.poll_unpin(cx)?; - let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; + connection.poll_inbound_unpin(cx) + }) + .await + { + Ok(s) => s, + Err(_) => return, + }; - let a_peer_id = &Swarm::local_peer_id(&a).clone(); - drop(a); // stop a swarm so b can never reach it + P::spawn(async move { + // FIXME: Need to write _some_ data before we can read on both sides. + // Do a ping-pong exchange. + { + let mut pong = [0u8; 4]; + inbound_stream.write_all(b"PING").await.unwrap(); + inbound_stream.flush().await.unwrap(); + inbound_stream.read_exact(&mut pong).await.unwrap(); + assert_eq!(&pong, b"PONG"); + } - b.behaviour_mut().add_address(a_peer_id, addr); - b.behaviour_mut() - .send_request(a_peer_id, Ping(b"hello world".to_vec())); + let mut data = vec![0; BUFFER_SIZE]; - match b.next().await { - Some(SwarmEvent::Dialing(_)) => {} - e => panic!("{:?}", e), + inbound_stream.read_exact(&mut data).await.unwrap(); + inbound_stream.write_all(&data).await.unwrap(); + inbound_stream.close().await.unwrap(); + }); } - - match b.next().await { - Some(SwarmEvent::OutgoingConnectionError { .. }) => {} - e => panic!("{:?}", e), - }; - - match b.next().await { - Some(SwarmEvent::Behaviour(RequestResponseEvent::OutboundFailure { .. })) => {} - e => panic!("{:?}", e), - }; } -#[test] -fn concurrent_connections_and_streams() { - use quickcheck::*; - - fn prop(number_listeners: NonZeroU8, number_streams: NonZeroU8) -> TestResult { - let (number_listeners, number_streams): (u8, u8) = - (number_listeners.into(), number_streams.into()); - if number_listeners > 10 || number_streams > 10 { - return TestResult::discard(); - } +async fn open_outbound_streams( + mut connection: StreamMuxerBox, + number_streams: usize, + completed_streams_tx: mpsc::Sender<()>, +) { + for _ in 0..number_streams { + let mut outbound_stream = future::poll_fn(|cx| { + let _ = connection.poll_unpin(cx)?; - let mut pool = futures::executor::LocalPool::default(); - let mut data = vec![0; 4096 * 10]; - rand::thread_rng().fill_bytes(&mut data); - let mut listeners = vec![]; - - // Spawn the listener nodes. - for _ in 0..number_listeners { - let mut listener = create_swarm::

(); - let addr = pool.run_until(start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic")); - - listeners.push((*listener.local_peer_id(), addr)); - - pool.spawner() - .spawn_obj( - async move { - loop { - match listener.next().await { - Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { - message: - RequestResponseMessage::Request { - request: Ping(ping), - channel, - .. - }, - .. - })) => { - listener - .behaviour_mut() - .send_response(channel, Pong(ping)) - .unwrap(); - } - Some(SwarmEvent::Behaviour( - RequestResponseEvent::ResponseSent { .. }, - )) - | Some(SwarmEvent::ConnectionEstablished { .. }) - | Some(SwarmEvent::IncomingConnection { .. }) - | Some(SwarmEvent::ConnectionClosed { .. }) => {} - Some(e) => { - panic!("unexpected event {:?}", e); - } - None => { - panic!("listener stopped"); - } - } - } - } - .boxed() - .into(), - ) - .unwrap(); - } + connection.poll_outbound_unpin(cx) + }) + .await + .unwrap(); + + P::spawn({ + let mut completed_streams_tx = completed_streams_tx.clone(); + + async move { + // FIXME: Need to write _some_ data before we can read on both sides. + // Do a ping-pong exchange. + { + let mut ping = [0u8; 4]; + outbound_stream.write_all(b"PONG").await.unwrap(); + outbound_stream.flush().await.unwrap(); + outbound_stream.read_exact(&mut ping).await.unwrap(); + assert_eq!(&ping, b"PING"); + } - let mut dialer = create_swarm::

(); + let mut data = vec![0; BUFFER_SIZE]; + rand::thread_rng().fill_bytes(&mut data); - // For each listener node start `number_streams` requests. - for (listener_peer_id, listener_addr) in &listeners { - dialer - .behaviour_mut() - .add_address(listener_peer_id, listener_addr.clone()); + let mut received = Vec::new(); - dialer.dial(*listener_peer_id).unwrap(); - } + outbound_stream.write_all(&data).await.unwrap(); + outbound_stream.flush().await.unwrap(); + outbound_stream.read_to_end(&mut received).await.unwrap(); - // Wait for responses to each request. - pool.run_until(async { - let mut num_responses = 0; - loop { - match dialer.next().await { - Some(SwarmEvent::Dialing(_)) => {} - Some(SwarmEvent::ConnectionEstablished { peer_id, .. }) => { - for _ in 0..number_streams { - dialer - .behaviour_mut() - .send_request(&peer_id, Ping(data.clone())); - } - } - Some(SwarmEvent::Behaviour(RequestResponseEvent::Message { - message: - RequestResponseMessage::Response { - response: Pong(pong), - .. - }, - .. - })) => { - num_responses += 1; - assert_eq!(data, pong); - let should_be = number_listeners as usize * (number_streams) as usize; - if num_responses == should_be { - break; - } - } - Some(SwarmEvent::ConnectionClosed { .. }) => {} - e => { - panic!("unexpected event {:?}", e); - } - } + assert_eq!(received, data); + + completed_streams_tx.send(()).await.unwrap(); } }); - - TestResult::passed() } - #[cfg(feature = "tokio")] - { - let rt = tokio::runtime::Runtime::new().unwrap(); - let _guard = rt.enter(); - QuickCheck::new().quickcheck(prop:: as fn(_, _) -> _); - } + log::info!("Created {number_streams} streams"); - #[cfg(feature = "async-std")] - QuickCheck::new().quickcheck(prop:: as fn(_, _) -> _); + while future::poll_fn(|cx| connection.poll_unpin(cx)) + .await + .is_ok() + {} } #[cfg(feature = "tokio")] #[tokio::test] async fn endpoint_reuse() { let _ = env_logger::try_init(); - let mut swarm_a = create_swarm::(); - let mut swarm_b = create_swarm::(); - let b_peer_id = *swarm_b.local_peer_id(); - - let a_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/udp/0/quic").await; - - swarm_b.dial(a_addr.clone()).unwrap(); - let b_send_back_addr = loop { - select! { - ev = swarm_a.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { endpoint, .. } => { - break endpoint.get_remote_address().clone() - } - SwarmEvent::IncomingConnection { local_addr, ..} => { - assert!(swarm_a.listeners().any(|a| a == &local_addr)); - } - e => panic!("{:?}", e), - }, - ev = swarm_b.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { .. } => {}, - e => panic!("{:?}", e), - } - } - }; + let (_, mut a_transport) = create_transport::(); + let (_, mut b_transport) = create_transport::(); - let dial_opts = DialOpts::peer_id(b_peer_id) - .addresses(vec![b_send_back_addr.clone()]) - .extend_addresses_through_behaviour() - .condition(PeerCondition::Always) - .build(); - swarm_a.dial(dial_opts).unwrap(); + let a_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let ((_, b_send_back_addr, _), _) = + connect(&mut a_transport, &mut b_transport, a_addr.clone()).await; // Expect the dial to fail since b is not listening on an address. - loop { - select! { - ev = swarm_a.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { ..} => panic!("Unexpected dial success."), - SwarmEvent::OutgoingConnectionError {error, .. } => { - assert!(matches!(error, DialError::Transport(_))); - break - } - _ => {} - }, - _ = swarm_b.select_next_some() => {}, + match dial(&mut a_transport, b_send_back_addr).await { + Ok(_) => panic!("Expected dial to fail"), + Err(error) => { + assert_eq!("Handshake with the remote timed out.", error.to_string()) } - } + }; - let b_addr = start_listening(&mut swarm_b, "/ip4/127.0.0.1/udp/0/quic").await; - - let dial_opts = DialOpts::peer_id(b_peer_id) - .addresses(vec![b_addr.clone(), b_send_back_addr]) - .condition(PeerCondition::Always) - .build(); - swarm_a.dial(dial_opts).unwrap(); - let expected_b_addr = b_addr.with(Protocol::P2p(b_peer_id.into())); - - let mut a_reported = false; - let mut b_reported = false; - while !a_reported || !b_reported { - select! { - ev = swarm_a.select_next_some() => match ev{ - SwarmEvent::ConnectionEstablished { endpoint, ..} => { - assert!(endpoint.is_dialer()); - assert_eq!(endpoint.get_remote_address(), &expected_b_addr); - a_reported = true; - } - SwarmEvent::OutgoingConnectionError {error, .. } => { - panic!("Unexpected error {:}", error) - } - _ => {} - }, - ev = swarm_b.select_next_some() => { - if let SwarmEvent::ConnectionEstablished { endpoint, ..} = ev { - match endpoint { - ConnectedPoint::Dialer{..} => panic!("Unexpected outbound connection"), - ConnectedPoint::Listener {send_back_addr, local_addr} => { - // Expect that the local listening endpoint was used for dialing. - assert!(swarm_b.listeners().any(|a| a == &local_addr)); - assert_eq!(send_back_addr, a_addr); - b_reported = true; - } - } - } - }, - } - } + let b_addr = start_listening(&mut b_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let ((_, a_send_back_addr, _), _) = connect(&mut b_transport, &mut a_transport, b_addr).await; + + assert_eq!(a_send_back_addr, a_addr); } #[cfg(feature = "async-std")] #[async_std::test] async fn ipv4_dial_ipv6() { let _ = env_logger::try_init(); - let mut swarm_a = create_swarm::(); - let mut swarm_b = create_swarm::(); + let (a_peer_id, mut a_transport) = create_transport::(); + let (b_peer_id, mut b_transport) = create_transport::(); - let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; + let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, a_addr).await; - swarm_b.dial(a_addr.clone()).unwrap(); - - loop { - select! { - ev = swarm_a.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { .. } => { - return; - } - SwarmEvent::IncomingConnection { local_addr, ..} => { - assert!(swarm_a.listeners().any(|a| a == &local_addr)); - } - e => panic!("{:?}", e), - }, - ev = swarm_b.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { .. } => {}, - e => panic!("{:?}", e), - } - } - } + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); } #[cfg(feature = "async-std")] #[async_std::test] +#[ignore] // Transport currently does not validate PeerId. Delete this test? async fn wrong_peerid() { use libp2p::PeerId; - let _ = env_logger::try_init(); - let mut swarm_a = create_swarm::(); - let mut swarm_b = create_swarm::(); + let (a_peer_id, mut a_transport) = create_transport::(); + let (b_peer_id, mut b_transport) = create_transport::(); - let a_addr = start_listening(&mut swarm_a, "/ip6/::1/udp/0/quic").await; - let a_id = *swarm_a.local_peer_id(); + let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; + let a_addr_random_peer = a_addr.with(Protocol::P2p(PeerId::random().into())); - let wrong_id = PeerId::random(); - let dial_ops = DialOpts::peer_id(wrong_id).addresses(vec![a_addr]).build(); - swarm_b.dial(dial_ops).unwrap(); + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, a_addr_random_peer).await; - loop { - select! { - _ = swarm_a.select_next_some() => {}, - ev = swarm_b.select_next_some() => match ev { - SwarmEvent::Dialing(peer_id) => assert_eq!(peer_id, wrong_id), - SwarmEvent::OutgoingConnectionError {peer_id: Some(peer_id), error: DialError::WrongPeerId { obtained, .. }} => { - assert_eq!(peer_id, wrong_id); - assert_eq!(obtained, a_id); - break; - }, - e => panic!("{:?}", e), - } - } - } + assert_ne!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); } #[cfg(feature = "async-std")] -fn new_tcp_quic_swarm() -> Swarm> { +fn new_tcp_quic_transport() -> (PeerId, Boxed<(PeerId, StreamMuxerBox)>) { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); let mut config = quic::Config::new(&keypair); config.handshake_timeout = Duration::from_secs(1); + let quic_transport = quic::async_std::Transport::new(config); let tcp_transport = tcp::async_io::Transport::new(tcp::Config::default()) - .upgrade(upgrade::Version::V1Lazy) + .upgrade(upgrade::Version::V1) .authenticate( noise::NoiseConfig::xx( noise::Keypair::::new() @@ -624,58 +361,78 @@ fn new_tcp_quic_swarm() -> Swarm> { }) .boxed(); - let protocols = iter::once((PingProtocol(), ProtocolSupport::Full)); - let cfg = RequestResponseConfig::default(); - let behaviour = RequestResponse::new(PingCodec(), protocols, cfg); - - Swarm::new(transport, behaviour, peer_id) + (peer_id, transport) } #[cfg(feature = "async-std")] #[async_std::test] async fn tcp_and_quic() { - let mut swarm_a = new_tcp_quic_swarm(); - let swarm_a_id = *swarm_a.local_peer_id(); - println!("{}", swarm_a_id); + let (a_peer_id, mut a_transport) = new_tcp_quic_transport(); + let (b_peer_id, mut b_transport) = new_tcp_quic_transport(); - let mut swarm_b = new_tcp_quic_swarm(); + let quic_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let tcp_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/tcp/0").await; - let quic_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/udp/0/quic").await; - let tcp_addr = start_listening(&mut swarm_a, "/ip4/127.0.0.1/tcp/0").await; + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, quic_addr).await; + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); - swarm_b.dial(quic_addr.clone()).unwrap(); + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, tcp_addr).await; + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); +} - loop { - select! { - ev = swarm_a.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { .. } => break, - SwarmEvent::IncomingConnection { .. } => { } - e => panic!("{:?}", e), - }, - ev = swarm_b.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { .. } => {}, - e => panic!("{:?}", e), - } +/// Helper function for driving two transports until they established a connection. +async fn connect( + listener: &mut Boxed<(PeerId, StreamMuxerBox)>, + dialer: &mut Boxed<(PeerId, StreamMuxerBox)>, + addr: Multiaddr, +) -> ( + (PeerId, Multiaddr, StreamMuxerBox), + (PeerId, StreamMuxerBox), +) { + future::join( + async { + let (upgrade, send_back_addr) = + listener.select_next_some().await.into_incoming().unwrap(); + let (peer_id, connection) = upgrade.await.unwrap(); + + (peer_id, send_back_addr, connection) + }, + async { dial(dialer, addr).await.unwrap() }, + ) + .await +} + +/// Helper function for dialling that also polls the `Transport`. +async fn dial( + transport: &mut Boxed<(PeerId, StreamMuxerBox)>, + addr: Multiaddr, +) -> io::Result<(PeerId, StreamMuxerBox)> { + match future::select(transport.dial(addr).unwrap(), transport.next()).await { + Either::Left((conn, _)) => conn, + Either::Right((event, _)) => { + panic!("Unexpected event: {event:?}") } } +} - swarm_b.dial(tcp_addr).unwrap(); +trait BlockOn { + fn block_on(future: impl Future + Send) -> R; +} - loop { - select! { - ev = swarm_a.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { .. } => break, - SwarmEvent::IncomingConnection { .. } - | SwarmEvent::ConnectionClosed { .. } => { } - e => panic!("{:?}", e), - }, - ev = swarm_b.select_next_some() => match ev { - SwarmEvent::ConnectionEstablished { .. } => {}, - SwarmEvent::ConnectionClosed { endpoint, .. } => { - assert_eq!(endpoint.get_remote_address(), &quic_addr ); - } - e => panic!("{:?}", e), - } - } +#[cfg(feature = "async-std")] +impl BlockOn for libp2p_quic::async_std::Provider { + fn block_on(future: impl Future + Send) -> R { + async_std::task::block_on(future) + } +} + +#[cfg(feature = "tokio")] +impl BlockOn for libp2p_quic::tokio::Provider { + fn block_on(future: impl Future + Send) -> R { + tokio::runtime::Handle::current().block_on(future) } } From 859a3b9328bb87979c54016772c1c9cc10307e22 Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Wed, 2 Nov 2022 12:06:29 +1100 Subject: [PATCH 203/218] Move utilities to the bottom --- transports/quic/tests/smoke.rs | 296 ++++++++++++++++----------------- 1 file changed, 148 insertions(+), 148 deletions(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index d9e582a2abd..ceb04586a9b 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -19,31 +19,6 @@ use std::io; use std::num::NonZeroU8; use std::time::Duration; -fn generate_tls_keypair() -> libp2p::identity::Keypair { - libp2p::identity::Keypair::generate_ed25519() -} - -fn create_transport() -> (PeerId, Boxed<(PeerId, StreamMuxerBox)>) { - let keypair = generate_tls_keypair(); - let peer_id = keypair.public().to_peer_id(); - let mut config = quic::Config::new(&keypair); - config.handshake_timeout = Duration::from_secs(1); - - let transport = quic::GenTransport::

::new(config) - .map(|(p, c), _| (p, StreamMuxerBox::new(c))) - .boxed(); - - (peer_id, transport) -} - -async fn start_listening(transport: &mut Boxed<(PeerId, StreamMuxerBox)>, addr: &str) -> Multiaddr { - transport.listen_on(addr.parse().unwrap()).unwrap(); - match transport.next().await { - Some(TransportEvent::NewAddress { listen_addr, .. }) => listen_addr, - e => panic!("{:?}", e), - } -} - #[cfg(feature = "tokio")] #[tokio::test] async fn tokio_smoke() { @@ -56,20 +31,6 @@ async fn async_std_smoke() { smoke::().await } -async fn smoke() { - let _ = env_logger::try_init(); - - let (a_peer_id, mut a_transport) = create_transport::

(); - let (b_peer_id, mut b_transport) = create_transport::

(); - - let addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; - let ((a_connected, _, _), (b_connected, _)) = - connect(&mut a_transport, &mut b_transport, addr).await; - - assert_eq!(a_connected, b_peer_id); - assert_eq!(b_connected, a_peer_id); -} - #[cfg(feature = "async-std")] #[async_std::test] async fn dial_failure() { @@ -88,6 +49,115 @@ async fn dial_failure() { }; } +#[cfg(feature = "tokio")] +#[tokio::test] +async fn endpoint_reuse() { + let _ = env_logger::try_init(); + let (_, mut a_transport) = create_transport::(); + let (_, mut b_transport) = create_transport::(); + + let a_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let ((_, b_send_back_addr, _), _) = + connect(&mut a_transport, &mut b_transport, a_addr.clone()).await; + + // Expect the dial to fail since b is not listening on an address. + match dial(&mut a_transport, b_send_back_addr).await { + Ok(_) => panic!("Expected dial to fail"), + Err(error) => { + assert_eq!("Handshake with the remote timed out.", error.to_string()) + } + }; + + let b_addr = start_listening(&mut b_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let ((_, a_send_back_addr, _), _) = connect(&mut b_transport, &mut a_transport, b_addr).await; + + assert_eq!(a_send_back_addr, a_addr); +} + +#[cfg(feature = "async-std")] +#[async_std::test] +async fn ipv4_dial_ipv6() { + let _ = env_logger::try_init(); + let (a_peer_id, mut a_transport) = create_transport::(); + let (b_peer_id, mut b_transport) = create_transport::(); + + let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, a_addr).await; + + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); +} + +#[cfg(feature = "async-std")] +#[async_std::test] +#[ignore] // Transport currently does not validate PeerId. Delete this test? +async fn wrong_peerid() { + use libp2p::PeerId; + + let (a_peer_id, mut a_transport) = create_transport::(); + let (b_peer_id, mut b_transport) = create_transport::(); + + let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; + let a_addr_random_peer = a_addr.with(Protocol::P2p(PeerId::random().into())); + + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, a_addr_random_peer).await; + + assert_ne!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); +} + +#[cfg(feature = "async-std")] +fn new_tcp_quic_transport() -> (PeerId, Boxed<(PeerId, StreamMuxerBox)>) { + let keypair = generate_tls_keypair(); + let peer_id = keypair.public().to_peer_id(); + let mut config = quic::Config::new(&keypair); + config.handshake_timeout = Duration::from_secs(1); + + let quic_transport = quic::async_std::Transport::new(config); + let tcp_transport = tcp::async_io::Transport::new(tcp::Config::default()) + .upgrade(upgrade::Version::V1) + .authenticate( + noise::NoiseConfig::xx( + noise::Keypair::::new() + .into_authentic(&keypair) + .unwrap(), + ) + .into_authenticated(), + ) + .multiplex(yamux::YamuxConfig::default()); + + let transport = OrTransport::new(quic_transport, tcp_transport) + .map(|either_output, _| match either_output { + EitherOutput::First((peer_id, muxer)) => (peer_id, StreamMuxerBox::new(muxer)), + EitherOutput::Second((peer_id, muxer)) => (peer_id, StreamMuxerBox::new(muxer)), + }) + .boxed(); + + (peer_id, transport) +} + +#[cfg(feature = "async-std")] +#[async_std::test] +async fn tcp_and_quic() { + let (a_peer_id, mut a_transport) = new_tcp_quic_transport(); + let (b_peer_id, mut b_transport) = new_tcp_quic_transport(); + + let quic_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let tcp_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/tcp/0").await; + + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, quic_addr).await; + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); + + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, tcp_addr).await; + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); +} + // Note: This test should likely be ported to the muxer compliance test suite. #[cfg(feature = "async-std")] #[test] @@ -112,6 +182,45 @@ fn concurrent_connections_and_streams_tokio() { .quickcheck(prop:: as fn(_, _) -> _); } +async fn smoke() { + let _ = env_logger::try_init(); + + let (a_peer_id, mut a_transport) = create_transport::

(); + let (b_peer_id, mut b_transport) = create_transport::

(); + + let addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let ((a_connected, _, _), (b_connected, _)) = + connect(&mut a_transport, &mut b_transport, addr).await; + + assert_eq!(a_connected, b_peer_id); + assert_eq!(b_connected, a_peer_id); +} + +fn generate_tls_keypair() -> libp2p::identity::Keypair { + libp2p::identity::Keypair::generate_ed25519() +} + +fn create_transport() -> (PeerId, Boxed<(PeerId, StreamMuxerBox)>) { + let keypair = generate_tls_keypair(); + let peer_id = keypair.public().to_peer_id(); + let mut config = quic::Config::new(&keypair); + config.handshake_timeout = Duration::from_secs(1); + + let transport = quic::GenTransport::

::new(config) + .map(|(p, c), _| (p, StreamMuxerBox::new(c))) + .boxed(); + + (peer_id, transport) +} + +async fn start_listening(transport: &mut Boxed<(PeerId, StreamMuxerBox)>, addr: &str) -> Multiaddr { + transport.listen_on(addr.parse().unwrap()).unwrap(); + match transport.next().await { + Some(TransportEvent::NewAddress { listen_addr, .. }) => listen_addr, + e => panic!("{:?}", e), + } +} + fn prop( number_listeners: NonZeroU8, number_streams: NonZeroU8, @@ -275,115 +384,6 @@ async fn open_outbound_streams( {} } -#[cfg(feature = "tokio")] -#[tokio::test] -async fn endpoint_reuse() { - let _ = env_logger::try_init(); - let (_, mut a_transport) = create_transport::(); - let (_, mut b_transport) = create_transport::(); - - let a_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; - let ((_, b_send_back_addr, _), _) = - connect(&mut a_transport, &mut b_transport, a_addr.clone()).await; - - // Expect the dial to fail since b is not listening on an address. - match dial(&mut a_transport, b_send_back_addr).await { - Ok(_) => panic!("Expected dial to fail"), - Err(error) => { - assert_eq!("Handshake with the remote timed out.", error.to_string()) - } - }; - - let b_addr = start_listening(&mut b_transport, "/ip4/127.0.0.1/udp/0/quic").await; - let ((_, a_send_back_addr, _), _) = connect(&mut b_transport, &mut a_transport, b_addr).await; - - assert_eq!(a_send_back_addr, a_addr); -} - -#[cfg(feature = "async-std")] -#[async_std::test] -async fn ipv4_dial_ipv6() { - let _ = env_logger::try_init(); - let (a_peer_id, mut a_transport) = create_transport::(); - let (b_peer_id, mut b_transport) = create_transport::(); - - let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; - let ((a_connected, _, _), (b_connected, _)) = - connect(&mut a_transport, &mut b_transport, a_addr).await; - - assert_eq!(a_connected, b_peer_id); - assert_eq!(b_connected, a_peer_id); -} - -#[cfg(feature = "async-std")] -#[async_std::test] -#[ignore] // Transport currently does not validate PeerId. Delete this test? -async fn wrong_peerid() { - use libp2p::PeerId; - - let (a_peer_id, mut a_transport) = create_transport::(); - let (b_peer_id, mut b_transport) = create_transport::(); - - let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; - let a_addr_random_peer = a_addr.with(Protocol::P2p(PeerId::random().into())); - - let ((a_connected, _, _), (b_connected, _)) = - connect(&mut a_transport, &mut b_transport, a_addr_random_peer).await; - - assert_ne!(a_connected, b_peer_id); - assert_eq!(b_connected, a_peer_id); -} - -#[cfg(feature = "async-std")] -fn new_tcp_quic_transport() -> (PeerId, Boxed<(PeerId, StreamMuxerBox)>) { - let keypair = generate_tls_keypair(); - let peer_id = keypair.public().to_peer_id(); - let mut config = quic::Config::new(&keypair); - config.handshake_timeout = Duration::from_secs(1); - - let quic_transport = quic::async_std::Transport::new(config); - let tcp_transport = tcp::async_io::Transport::new(tcp::Config::default()) - .upgrade(upgrade::Version::V1) - .authenticate( - noise::NoiseConfig::xx( - noise::Keypair::::new() - .into_authentic(&keypair) - .unwrap(), - ) - .into_authenticated(), - ) - .multiplex(yamux::YamuxConfig::default()); - - let transport = OrTransport::new(quic_transport, tcp_transport) - .map(|either_output, _| match either_output { - EitherOutput::First((peer_id, muxer)) => (peer_id, StreamMuxerBox::new(muxer)), - EitherOutput::Second((peer_id, muxer)) => (peer_id, StreamMuxerBox::new(muxer)), - }) - .boxed(); - - (peer_id, transport) -} - -#[cfg(feature = "async-std")] -#[async_std::test] -async fn tcp_and_quic() { - let (a_peer_id, mut a_transport) = new_tcp_quic_transport(); - let (b_peer_id, mut b_transport) = new_tcp_quic_transport(); - - let quic_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; - let tcp_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/tcp/0").await; - - let ((a_connected, _, _), (b_connected, _)) = - connect(&mut a_transport, &mut b_transport, quic_addr).await; - assert_eq!(a_connected, b_peer_id); - assert_eq!(b_connected, a_peer_id); - - let ((a_connected, _, _), (b_connected, _)) = - connect(&mut a_transport, &mut b_transport, tcp_addr).await; - assert_eq!(a_connected, b_peer_id); - assert_eq!(b_connected, a_peer_id); -} - /// Helper function for driving two transports until they established a connection. async fn connect( listener: &mut Boxed<(PeerId, StreamMuxerBox)>, From 6833b91b99d4a1d625cb12b495bf3e1dbcf5c6dc Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 11 Nov 2022 09:01:04 +1100 Subject: [PATCH 204/218] Add timeout to concurrency test --- transports/quic/tests/smoke.rs | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index ceb04586a9b..0ec7a00b3a2 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -292,6 +292,7 @@ fn prop( completed_streams_rx .take(completed_streams as usize) .collect::>(), + Duration::from_secs(30), ); quickcheck::TestResult::passed() @@ -420,19 +421,21 @@ async fn dial( } trait BlockOn { - fn block_on(future: impl Future + Send) -> R; + fn block_on(future: impl Future + Send, timeout: Duration) -> R; } #[cfg(feature = "async-std")] impl BlockOn for libp2p_quic::async_std::Provider { - fn block_on(future: impl Future + Send) -> R { - async_std::task::block_on(future) + fn block_on(future: impl Future + Send, timeout: Duration) -> R { + async_std::task::block_on(async_std::future::timeout(timeout, future)).unwrap() } } #[cfg(feature = "tokio")] impl BlockOn for libp2p_quic::tokio::Provider { - fn block_on(future: impl Future + Send) -> R { - tokio::runtime::Handle::current().block_on(future) + fn block_on(future: impl Future + Send, timeout: Duration) -> R { + tokio::runtime::Handle::current() + .block_on(tokio::time::timeout(timeout, future)) + .unwrap() } } From f2c390dbadd2ba9afa5587cde9adbb603caf022a Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 11 Nov 2022 09:02:09 +1100 Subject: [PATCH 205/218] Change note --- transports/quic/tests/smoke.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 0ec7a00b3a2..fb2c223c749 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -91,7 +91,7 @@ async fn ipv4_dial_ipv6() { #[cfg(feature = "async-std")] #[async_std::test] -#[ignore] // Transport currently does not validate PeerId. Delete this test? +#[ignore] // Transport currently does not validate PeerId. Enable once we make use of PeerId validation in rustls. async fn wrong_peerid() { use libp2p::PeerId; From 0fadf358f48c592bbb886f8bba88645c0714b69e Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 11 Nov 2022 09:02:40 +1100 Subject: [PATCH 206/218] Fix missing feature --- transports/quic/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index a5646d3feaf..cf7681997ba 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -45,7 +45,7 @@ libp2p = { path = "../..", features = ["tcp", "yamux", "noise", "async-std"] } libp2p-muxer-test-harness = { path = "../../muxers/test-harness" } quickcheck = "1" rand = "0.8.4" -tokio = { version = "1.21.1", features = ["macros", "rt-multi-thread"] } +tokio = { version = "1.21.1", features = ["macros", "rt-multi-thread", "time"] } [[test]] name = "stream_compliance" From aa768f7b018844af9008f05233b607da0306227f Mon Sep 17 00:00:00 2001 From: Thomas Eizinger Date: Fri, 11 Nov 2022 11:57:53 +1100 Subject: [PATCH 207/218] Run tests with default config --- transports/quic/tests/smoke.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index fb2c223c749..25f410b5d38 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -203,10 +203,8 @@ fn generate_tls_keypair() -> libp2p::identity::Keypair { fn create_transport() -> (PeerId, Boxed<(PeerId, StreamMuxerBox)>) { let keypair = generate_tls_keypair(); let peer_id = keypair.public().to_peer_id(); - let mut config = quic::Config::new(&keypair); - config.handshake_timeout = Duration::from_secs(1); - let transport = quic::GenTransport::

::new(config) + let transport = quic::GenTransport::

::new(quic::Config::new(&keypair)) .map(|(p, c), _| (p, StreamMuxerBox::new(c))) .boxed(); From 1feb6291af11227a5f115382d775e6f116f99fd6 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 17:09:53 +0100 Subject: [PATCH 208/218] quic: fix test_dialer_drop test --- 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 cb62a51e496..acbc8802467 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -721,7 +721,7 @@ mod test { async fn test_dialer_drop() { let keypair = libp2p_core::identity::Keypair::generate_ed25519(); let config = Config::new(&keypair); - let mut transport = crate::async_std::Transport::new(config); + let mut transport = crate::tokio::Transport::new(config); let _dial = transport .dial("/ip4/123.45.67.8/udp/1234/quic".parse().unwrap()) From a7acce49e26f48ca5cc5dab015bf0cef9c8d4d32 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 17:13:40 +0100 Subject: [PATCH 209/218] quic: update to quinn-protov0.9 --- transports/quic/Cargo.toml | 2 +- transports/quic/src/endpoint.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index f01ebd3e4f1..41eb3750845 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -18,7 +18,7 @@ libp2p-core = { version = "0.38.0", path = "../../core" } libp2p-tls = { version = "0.1.0-alpha", path = "../tls" } log = "0.4" parking_lot = "0.12.0" -quinn-proto = { version = "0.8.2", default-features = false, features = ["tls-rustls"] } +quinn-proto = { version = "0.9.0", default-features = false, features = ["tls-rustls"] } rand = "0.8.5" rustls = { version = "0.20.2", default-features = false } thiserror = "1.0.26" diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 4d195d68060..2cfe67ed55e 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -137,7 +137,7 @@ impl From for QuinnConfig { server_config.migration(false); let mut client_config = quinn_proto::ClientConfig::new(client_tls_config); - client_config.transport = transport; + client_config.transport_config(transport); let endpoint_config = quinn_proto::EndpointConfig::default(); From 20c3fce046a76e6ef6e22d18a506f80546bbc6f6 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 17:21:24 +0100 Subject: [PATCH 210/218] quic: clean dependencies --- transports/quic/Cargo.toml | 5 ----- transports/quic/src/endpoint.rs | 3 +-- transports/quic/src/provider/tokio.rs | 3 +-- 3 files changed, 2 insertions(+), 9 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index 41eb3750845..a7a8b8bef22 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -9,7 +9,6 @@ license = "MIT" [dependencies] async-std = { version = "1.12.0", default-features = false, optional = true } -async-trait = "0.1.50" bytes = "1.2.1" futures = "0.3.15" futures-timer = "3.0.2" @@ -23,9 +22,6 @@ rand = "0.8.5" rustls = { version = "0.20.2", default-features = false } thiserror = "1.0.26" tokio = { version = "1.21.1", default-features = false, features = ["net", "rt"], optional = true } -webpki = "0.22.0" -x509-parser = "0.13.0" -yasna = "0.5.0" [features] tokio = ["dep:tokio"] @@ -44,7 +40,6 @@ env_logger = "0.9.0" libp2p = { path = "../..", features = ["tcp", "yamux", "noise", "async-std"] } libp2p-muxer-test-harness = { path = "../../muxers/test-harness" } quickcheck = "1" -rand = "0.8.4" tokio = { version = "1.21.1", features = ["macros", "rt-multi-thread", "time"] } [[test]] diff --git a/transports/quic/src/endpoint.rs b/transports/quic/src/endpoint.rs index 2cfe67ed55e..ff92f2eda19 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -35,7 +35,6 @@ use std::{ task::{Context, Poll}, time::{Duration, Instant}, }; -use x509_parser::nom::AsBytes; // The `Driver` drops packets if the channel to the connection // or transport is full. @@ -618,7 +617,7 @@ impl Future for Driver

{ // Poll for new packets from the remote. match self.provider_socket.poll_recv_from(cx) { Poll::Ready(Ok((bytes, packet_src))) => { - let bytes_mut = bytes.as_bytes().into(); + let bytes_mut = bytes.as_slice().into(); match self.handle_datagram(bytes_mut, packet_src) { ControlFlow::Continue(()) => continue, ControlFlow::Break(()) => break, diff --git a/transports/quic/src/provider/tokio.rs b/transports/quic/src/provider/tokio.rs index e0583c3705a..ab1d56fc1f5 100644 --- a/transports/quic/src/provider/tokio.rs +++ b/transports/quic/src/provider/tokio.rs @@ -25,7 +25,6 @@ use std::{ task::{Context, Poll}, }; use tokio::{io::ReadBuf, net::UdpSocket}; -use x509_parser::nom::AsBytes; use crate::GenTransport; @@ -55,7 +54,7 @@ impl super::Provider for Provider { Some(pending) => pending, None => return Poll::Ready(Ok(())), }; - match self.socket.poll_send_to(cx, data.as_bytes(), *addr) { + match self.socket.poll_send_to(cx, data.as_slice(), *addr) { Poll::Ready(result) => { self.next_packet_out = None; Poll::Ready(result.map(|_| ())) From bea79a8e00056a3bec92c58b57fa92e6112e2096 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 17:23:12 +0100 Subject: [PATCH 211/218] quic: make clippy happy --- 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 ff92f2eda19..318af8b71b9 100644 --- a/transports/quic/src/endpoint.rs +++ b/transports/quic/src/endpoint.rs @@ -188,7 +188,7 @@ impl Channel { socket_addr: SocketAddr, new_connections: Option>, ) -> Result { - let socket = std::net::UdpSocket::bind(&socket_addr)?; + let socket = std::net::UdpSocket::bind(socket_addr)?; // NOT blocking, as per man:bind(2), as we pass an IP address. socket.set_nonblocking(true)?; // Capacity 0 to back-pressure the rest of the application if From 0ab672ab402dcb5c925467307d16f33b30ffe988 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 17:24:52 +0100 Subject: [PATCH 212/218] quic/tests: remove `dialer_can_receive` test Test will be removed completely from the test harness in #3108. --- transports/quic/tests/stream_compliance.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/transports/quic/tests/stream_compliance.rs b/transports/quic/tests/stream_compliance.rs index c2b6f4bed85..b8f95178879 100644 --- a/transports/quic/tests/stream_compliance.rs +++ b/transports/quic/tests/stream_compliance.rs @@ -11,14 +11,6 @@ async fn close_implies_flush() { libp2p_muxer_test_harness::close_implies_flush(alice, bob).await; } -#[async_std::test] -#[ignore] // Hangs forever, same as yamux. We can't read from a stream that we have never written to. -async fn dialer_can_receive() { - let (alice, bob) = connected_peers().await; - - libp2p_muxer_test_harness::dialer_can_receive(alice, bob).await; -} - #[async_std::test] async fn read_after_close() { let (alice, bob) = connected_peers().await; From 4b042b2ee417d8131c755e5edcb1a9cc3c1a5d7e Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 18:04:37 +0100 Subject: [PATCH 213/218] quic: Update to if-watchv3.0.0 Make `IfWatcher` a `Provider` item. Use `if_watch::smol::IfWatcher` in `async-std::Provider` and `if_watch::tokio::IfWatcher` in `tokio::Provider`. --- transports/quic/Cargo.toml | 6 +-- transports/quic/src/provider.rs | 12 ++++++ transports/quic/src/provider/async_std.rs | 13 +++++++ transports/quic/src/provider/tokio.rs | 13 +++++++ transports/quic/src/transport.rs | 47 ++++++++++++++--------- 5 files changed, 70 insertions(+), 21 deletions(-) diff --git a/transports/quic/Cargo.toml b/transports/quic/Cargo.toml index a7a8b8bef22..0a1235cd731 100644 --- a/transports/quic/Cargo.toml +++ b/transports/quic/Cargo.toml @@ -12,7 +12,7 @@ async-std = { version = "1.12.0", default-features = false, optional = true } bytes = "1.2.1" futures = "0.3.15" futures-timer = "3.0.2" -if-watch = "2.0.0" +if-watch = "3.0.0" libp2p-core = { version = "0.38.0", path = "../../core" } libp2p-tls = { version = "0.1.0-alpha", path = "../tls" } log = "0.4" @@ -24,8 +24,8 @@ thiserror = "1.0.26" tokio = { version = "1.21.1", default-features = false, features = ["net", "rt"], optional = true } [features] -tokio = ["dep:tokio"] -async-std = ["dep:async-std"] +tokio = ["dep:tokio", "if-watch/tokio"] +async-std = ["dep:async-std", "if-watch/smol"] # Passing arguments to the docsrs builder in order to properly document cfg's. # More information: https://docs.rs/about/builds#cross-compiling diff --git a/transports/quic/src/provider.rs b/transports/quic/src/provider.rs index f413625aaf7..f0b53aa699d 100644 --- a/transports/quic/src/provider.rs +++ b/transports/quic/src/provider.rs @@ -19,6 +19,7 @@ // DEALINGS IN THE SOFTWARE. use futures::Future; +use if_watch::IfEvent; use std::{ io, net::SocketAddr, @@ -36,6 +37,8 @@ const RECEIVE_BUFFER_SIZE: usize = 65536; /// Provider for non-blocking receiving and sending on a [`std::net::UdpSocket`] /// and spawning tasks. pub trait Provider: Unpin + Send + Sized + 'static { + type IfWatcher: Unpin + Send; + /// Create a new providing that is wrapping the socket. /// /// Note: The socket must be set to non-blocking. @@ -61,4 +64,13 @@ pub trait Provider: Unpin + Send + Sized + 'static { /// /// This is used to spawn the task that is driving the endpoint. fn spawn(future: impl Future + Send + 'static); + + /// Create a new [`if_watch`] watcher that reports [`IfEvent`]s for network interface changes. + fn new_if_watcher() -> io::Result; + + /// Poll for an address change event. + fn poll_if_event( + watcher: &mut Self::IfWatcher, + cx: &mut Context<'_>, + ) -> Poll>; } diff --git a/transports/quic/src/provider/async_std.rs b/transports/quic/src/provider/async_std.rs index a9d10b4d4f7..222c8e55e90 100644 --- a/transports/quic/src/provider/async_std.rs +++ b/transports/quic/src/provider/async_std.rs @@ -45,6 +45,8 @@ pub struct Provider { } impl super::Provider for Provider { + type IfWatcher = if_watch::smol::IfWatcher; + fn from_socket(socket: std::net::UdpSocket) -> io::Result { let socket = Arc::new(socket.into()); let recv_stream = ReceiveStream::new(Arc::clone(&socket)); @@ -91,6 +93,17 @@ impl super::Provider for Provider { fn spawn(future: impl Future + Send + 'static) { spawn(future); } + + fn new_if_watcher() -> io::Result { + if_watch::smol::IfWatcher::new() + } + + fn poll_if_event( + watcher: &mut Self::IfWatcher, + cx: &mut Context<'_>, + ) -> Poll> { + watcher.poll_if_event(cx) + } } type ReceiveStreamItem = ( diff --git a/transports/quic/src/provider/tokio.rs b/transports/quic/src/provider/tokio.rs index ab1d56fc1f5..07e23f8813c 100644 --- a/transports/quic/src/provider/tokio.rs +++ b/transports/quic/src/provider/tokio.rs @@ -40,6 +40,8 @@ pub struct Provider { } impl super::Provider for Provider { + type IfWatcher = if_watch::tokio::IfWatcher; + fn from_socket(socket: std::net::UdpSocket) -> std::io::Result { let socket = UdpSocket::from_std(socket)?; Ok(Provider { @@ -82,4 +84,15 @@ impl super::Provider for Provider { fn spawn(future: impl Future + Send + 'static) { tokio::spawn(future); } + + fn new_if_watcher() -> io::Result { + if_watch::tokio::IfWatcher::new() + } + + fn poll_if_event( + watcher: &mut Self::IfWatcher, + cx: &mut Context<'_>, + ) -> Poll> { + watcher.poll_if_event(cx) + } } diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index acbc8802467..88672344b20 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -28,7 +28,7 @@ use futures::ready; use futures::stream::StreamExt; use futures::{prelude::*, stream::SelectAll}; -use if_watch::{IfEvent, IfWatcher}; +use if_watch::IfEvent; use libp2p_core::{ multiaddr::{Multiaddr, Protocol}, @@ -37,8 +37,8 @@ use libp2p_core::{ }; use std::collections::hash_map::{DefaultHasher, Entry}; use std::collections::{HashMap, VecDeque}; +use std::fmt; use std::hash::{Hash, Hasher}; -use std::marker::PhantomData; use std::net::IpAddr; use std::task::Waker; use std::time::Duration; @@ -50,19 +50,18 @@ use std::{ /// Implementation of the [`Transport`] trait for QUIC. #[derive(Debug)] -pub struct GenTransport

{ +pub struct GenTransport { /// Config for the inner [`quinn_proto`] structs. quinn_config: QuinnConfig, /// Timeout for the [`Connecting`] future. handshake_timeout: Duration, /// Streams of active [`Listener`]s. - listeners: SelectAll, + listeners: SelectAll>, /// Dialer for each socket family if no matching listener exists. dialer: HashMap, - _marker: PhantomData

, } -impl

GenTransport

{ +impl GenTransport

{ /// Create a new [`GenTransport`] with the given [`Config`]. pub fn new(config: Config) -> Self { let handshake_timeout = config.handshake_timeout; @@ -72,7 +71,6 @@ impl

GenTransport

{ quinn_config, handshake_timeout, dialer: HashMap::new(), - _marker: Default::default(), } } } @@ -87,7 +85,7 @@ impl Transport for GenTransport

{ let socket_addr = multiaddr_to_socketaddr(&addr).ok_or(TransportError::MultiaddrNotSupported(addr))?; let listener_id = ListenerId::new(); - let listener = Listener::new::

( + let listener = Listener::new( listener_id, socket_addr, self.quinn_config.clone(), @@ -297,8 +295,7 @@ impl DialerState { } /// Listener for incoming connections. -#[derive(Debug)] -struct Listener { +struct Listener { /// Id of the listener. listener_id: ListenerId, @@ -315,7 +312,7 @@ struct Listener { /// Watcher for network interface changes. /// /// None if we are only listening on a single interface. - if_watcher: Option, + if_watcher: Option, /// Whether the listener was closed and the stream should terminate. is_closed: bool, @@ -324,8 +321,8 @@ struct Listener { pending_event: Option<::Item>, } -impl Listener { - fn new( +impl Listener

{ + fn new( listener_id: ListenerId, socket_addr: SocketAddr, config: QuinnConfig, @@ -337,7 +334,7 @@ impl Listener { let if_watcher; let pending_event; if socket_addr.ip().is_unspecified() { - if_watcher = Some(IfWatcher::new()?); + if_watcher = Some(P::new_if_watcher()?); pending_event = None; } else { if_watcher = None; @@ -380,7 +377,7 @@ impl Listener { None => return Poll::Pending, }; loop { - match ready!(if_watcher.poll_if_event(cx)) { + match ready!(P::poll_if_event(if_watcher, cx)) { Ok(IfEvent::Up(inet)) => { if let Some(listen_addr) = ip_to_listenaddr(self.endpoint_channel.socket_addr(), inet.addr()) @@ -425,7 +422,7 @@ impl Listener { } } -impl Stream for Listener { +impl Stream for Listener

{ type Item = TransportEvent; fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { loop { @@ -469,7 +466,21 @@ impl Stream for Listener { } } -impl Drop for Listener { +impl fmt::Debug for Listener

{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Listener") + .field("listener_id", &self.listener_id) + .field("endpoint_channel", &self.endpoint_channel) + .field("dialer_state", &self.dialer_state) + .field("new_connections_rx", &self.new_connections_rx) + .field("handshake_timeout", &self.handshake_timeout) + .field("is_closed", &self.is_closed) + .field("pending_event", &self.pending_event) + .finish() + } +} + +impl Drop for Listener

{ fn drop(&mut self) { self.endpoint_channel.send_on_drop(ToEndpoint::Decoupled); } @@ -499,7 +510,7 @@ impl From for SocketFamily { } } -/// Turn an [`IpAddr`] reported byt the [`IfWatcher`] into a +/// Turn an [`IpAddr`] reported by the interface watcher into a /// listen-address for the endpoint. /// /// For this, the `ip` is combined with the port that the endpoint From 7770943a79781636fba82fe49ed254a7420c7edd Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 18:08:06 +0100 Subject: [PATCH 214/218] quic: rm debug_assert(!is_closed) on new connection Depending on what the remote sent in the initial packet, a new inbound connection may directly be marked as closed. --- transports/quic/src/connection.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index 45038e038eb..b6eebcd3d8d 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -82,7 +82,6 @@ impl Connection { connection_id: quinn_proto::ConnectionHandle, from_endpoint: mpsc::Receiver, ) -> Self { - debug_assert!(!connection.is_closed()); let state = State { connection, substreams: HashMap::new(), From e3c955cc8ccd2dc9af90a9e2f8f3150732273c66 Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 18:39:39 +0100 Subject: [PATCH 215/218] quic/connection: send to endpoint before receiving Block on sending `pending_to_endpoint` before polling `from_endpoint`. If the endpoint is blocked on writing to the udp socket it will backpressure the connection, which will in return then stop polling for events from the endpoint. The endpoint will drop new inbound packets if the channel to the connection is full, thus eventually backpressuring remote peers. --- transports/quic/src/connection.rs | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/transports/quic/src/connection.rs b/transports/quic/src/connection.rs index b6eebcd3d8d..9e292df74e9 100644 --- a/transports/quic/src/connection.rs +++ b/transports/quic/src/connection.rs @@ -131,25 +131,14 @@ impl Connection { fn poll_event(&mut self, cx: &mut Context<'_>) -> Poll> { let mut inner = self.state.lock(); loop { - match self.from_endpoint.poll_next_unpin(cx) { - Poll::Ready(Some(event)) => { - inner.connection.handle_event(event); - continue; - } - Poll::Ready(None) => { - return Poll::Ready(None); - } - Poll::Pending => {} - } - // Sending the pending event to the endpoint. If the endpoint is too busy, we just // stop the processing here. - // However we don't deliver substream-related events to the user as long as + // We don't deliver substream-related events to the user as long as // `to_endpoint` is full. This should propagate the back-pressure of `to_endpoint` // being full to the user. if let Some(to_endpoint) = self.pending_to_endpoint.take() { match self.endpoint_channel.try_send(to_endpoint, cx) { - Ok(Ok(())) => continue, // The endpoint may send back an event. + Ok(Ok(())) => {} Ok(Err(to_endpoint)) => { self.pending_to_endpoint = Some(to_endpoint); return Poll::Pending; @@ -160,6 +149,17 @@ impl Connection { } } + match self.from_endpoint.poll_next_unpin(cx) { + Poll::Ready(Some(event)) => { + inner.connection.handle_event(event); + continue; + } + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => {} + } + // The maximum amount of segments which can be transmitted in a single Transmit // if a platform supports Generic Send Offload (GSO). // Set to 1 for now since not all platforms support GSO. @@ -208,7 +208,7 @@ impl Connection { continue; } - // The final step consists in handling the events related to the various substreams. + // The final step consists in returning the events related to the various substreams. if let Some(ev) = inner.connection.poll() { return Poll::Ready(Some(ev)); } From 56e39b8f463985eb634f48f621fde5a923c7be2c Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 18:49:32 +0100 Subject: [PATCH 216/218] quic: Add initial release entry to CHANGELOG --- transports/quic/CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 transports/quic/CHANGELOG.md diff --git a/transports/quic/CHANGELOG.md b/transports/quic/CHANGELOG.md new file mode 100644 index 00000000000..b334c71ecac --- /dev/null +++ b/transports/quic/CHANGELOG.md @@ -0,0 +1,3 @@ +# 0.7.0-alpha [unreleased] + +- Initial alpha release. From cd7f38fdedc0000b731811dc6005795db9923a4b Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Sat, 12 Nov 2022 20:31:11 +0100 Subject: [PATCH 217/218] quic: Add entry to libp2p CHANGELOG --- CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 28bc6874eef..0cd0078f7ce 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -25,6 +25,7 @@ - [`libp2p-noise` CHANGELOG](transports/noise/CHANGELOG.md) - [`libp2p-plaintext` CHANGELOG](transports/plaintext/CHANGELOG.md) - [`libp2p-pnet` CHANGELOG](transports/pnet/CHANGELOG.md) +- [`libp2p-quic` CHANGELOG](transports/quic/CHANGELOG.md) - [`libp2p-tcp` CHANGELOG](transports/tcp/CHANGELOG.md) - [`libp2p-uds` CHANGELOG](transports/uds/CHANGELOG.md) - [`libp2p-wasm-ext` CHANGELOG](transports/wasm-ext/CHANGELOG.md) @@ -47,6 +48,7 @@ # 0.50.0 - [unreleased] - Introduce [`libp2p-tls` `v0.1.0-alpha`](transports/tls/CHANGELOG.md#010-alpha). See [PR 2945]. +- Introduce [`libp2p-quic` `v0.7.0-alpha`](transports/quic/CHANGELOG.md#070-alpha). See [PR 2289]. - Remove deprecated features: `tcp-tokio`, `mdns-tokio`, `dns-tokio`, `tcp-async-io`, `mdns-async-io`, `dns-async-std`. See [PR 3001]. - Introduce [`libp2p-tls` `v0.1.0`](transports/tls/CHANGELOG.md#010). See [PR 2945]. @@ -79,6 +81,7 @@ [PR 2945]: https://github.com/libp2p/rust-libp2p/pull/2945 [PR 3001]: https://github.com/libp2p/rust-libp2p/pull/3001 [PR 2945]: https://github.com/libp2p/rust-libp2p/pull/2945 +[PR 2289]: https://github.com/libp2p/rust-libp2p/pull/2289 # 0.49.0 From b14516a77e304474fbd8bd76683ff19a05ae942f Mon Sep 17 00:00:00 2001 From: Elena Frank Date: Mon, 14 Nov 2022 00:04:35 +0100 Subject: [PATCH 218/218] quic: Use `quic-v1` codepoint With multiformats/multiaddr#145 the `quic` codepoint should be interpreted as QUIC draft-29. For QUIC v1 (RFC9000) the new codepoint `quic-v1` should be used. Quinn supports both, draft-29 and v1 as server, for clients however the version has to be set when dialing. Right now we use the default, v1. Proper support for `Protocol::QUIC` / draft-29 will be added in a follow-up PR. --- transports/quic/src/lib.rs | 2 +- transports/quic/src/transport.rs | 26 +++++++++++----------- transports/quic/tests/smoke.rs | 16 ++++++------- transports/quic/tests/stream_compliance.rs | 2 +- transports/tcp/src/lib.rs | 2 +- 5 files changed, 24 insertions(+), 24 deletions(-) diff --git a/transports/quic/src/lib.rs b/transports/quic/src/lib.rs index b6a3ea0ada9..8195bb3b136 100644 --- a/transports/quic/src/lib.rs +++ b/transports/quic/src/lib.rs @@ -39,7 +39,7 @@ //! //! let mut quic_transport = quic::async_std::Transport::new(quic_config); //! -//! let addr = "/ip4/127.0.0.1/udp/12345/quic".parse().expect("address should be valid"); +//! let addr = "/ip4/127.0.0.1/udp/12345/quic-v1".parse().expect("address should be valid"); //! quic_transport.listen_on(addr).expect("listen error."); //! # //! # Ok(()) diff --git a/transports/quic/src/transport.rs b/transports/quic/src/transport.rs index 88672344b20..52eb7428ff1 100644 --- a/transports/quic/src/transport.rs +++ b/transports/quic/src/transport.rs @@ -543,10 +543,10 @@ fn multiaddr_to_socketaddr(addr: &Multiaddr) -> Option { } match (proto1, proto2, proto3) { - (Protocol::Ip4(ip), Protocol::Udp(port), Protocol::Quic) => { + (Protocol::Ip4(ip), Protocol::Udp(port), Protocol::QuicV1) => { Some(SocketAddr::new(ip.into(), port)) } - (Protocol::Ip6(ip), Protocol::Udp(port), Protocol::Quic) => { + (Protocol::Ip6(ip), Protocol::Udp(port), Protocol::QuicV1) => { Some(SocketAddr::new(ip.into(), port)) } _ => None, @@ -574,7 +574,7 @@ fn is_quic_addr(addr: &Multiaddr) -> bool { matches!(first, Ip4(_) | Ip6(_) | Dns(_) | Dns4(_) | Dns6(_)) && matches!(second, Udp(_)) - && matches!(third, Quic) + && matches!(third, QuicV1) && matches!(fourth, Some(P2p(_)) | None) && matches!(fifth, None) } @@ -584,7 +584,7 @@ fn socketaddr_to_multiaddr(socket_addr: &SocketAddr) -> Multiaddr { Multiaddr::empty() .with(socket_addr.ip().into()) .with(Protocol::Udp(socket_addr.port())) - .with(Protocol::Quic) + .with(Protocol::QuicV1) } #[cfg(test)] @@ -605,7 +605,7 @@ mod test { assert_eq!( multiaddr_to_socketaddr( - &"/ip4/127.0.0.1/udp/12345/quic" + &"/ip4/127.0.0.1/udp/12345/quic-v1" .parse::() .unwrap() ), @@ -616,7 +616,7 @@ mod test { ); assert_eq!( multiaddr_to_socketaddr( - &"/ip4/255.255.255.255/udp/8080/quic" + &"/ip4/255.255.255.255/udp/8080/quic-v1" .parse::() .unwrap() ), @@ -627,7 +627,7 @@ mod test { ); assert_eq!( multiaddr_to_socketaddr( - &"/ip4/127.0.0.1/udp/55148/quic/p2p/12D3KooW9xk7Zp1gejwfwNpfm6L9zH5NL4Bx5rm94LRYJJHJuARZ" + &"/ip4/127.0.0.1/udp/55148/quic-v1/p2p/12D3KooW9xk7Zp1gejwfwNpfm6L9zH5NL4Bx5rm94LRYJJHJuARZ" .parse::() .unwrap() ), @@ -637,7 +637,7 @@ mod test { )) ); assert_eq!( - multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic".parse::().unwrap()), + multiaddr_to_socketaddr(&"/ip6/::1/udp/12345/quic-v1".parse::().unwrap()), Some(SocketAddr::new( IpAddr::V6(Ipv6Addr::new(0, 0, 0, 0, 0, 0, 0, 1)), 12345, @@ -645,7 +645,7 @@ mod test { ); assert_eq!( multiaddr_to_socketaddr( - &"/ip6/ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/udp/8080/quic" + &"/ip6/ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/udp/8080/quic-v1" .parse::() .unwrap() ), @@ -672,7 +672,7 @@ mod test { // is temporarily empty. for _ in 0..2 { let id = transport - .listen_on("/ip4/0.0.0.0/udp/0/quic".parse().unwrap()) + .listen_on("/ip4/0.0.0.0/udp/0/quic-v1".parse().unwrap()) .unwrap(); // Copy channel to use it later. @@ -696,7 +696,7 @@ mod test { assert!( matches!(listen_addr.iter().nth(1), Some(Protocol::Udp(port)) if port != 0) ); - assert!(matches!(listen_addr.iter().nth(2), Some(Protocol::Quic))); + assert!(matches!(listen_addr.iter().nth(2), Some(Protocol::QuicV1))); } e => panic!("Unexpected event: {:?}", e), } @@ -735,7 +735,7 @@ mod test { let mut transport = crate::tokio::Transport::new(config); let _dial = transport - .dial("/ip4/123.45.67.8/udp/1234/quic".parse().unwrap()) + .dial("/ip4/123.45.67.8/udp/1234/quic-v1".parse().unwrap()) .unwrap(); // Expect a dialer and its background task to exist. @@ -765,7 +765,7 @@ mod test { // Start listening so that the dialer and driver are dropped. let _ = transport - .listen_on("/ip4/0.0.0.0/udp/0/quic".parse().unwrap()) + .listen_on("/ip4/0.0.0.0/udp/0/quic-v1".parse().unwrap()) .unwrap(); assert!(!transport.dialer.contains_key(&SocketFamily::Ipv4)); diff --git a/transports/quic/tests/smoke.rs b/transports/quic/tests/smoke.rs index 25f410b5d38..6eedcdfb0c2 100644 --- a/transports/quic/tests/smoke.rs +++ b/transports/quic/tests/smoke.rs @@ -38,7 +38,7 @@ async fn dial_failure() { let mut a = create_transport::().1; let mut b = create_transport::().1; - let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic").await; + let addr = start_listening(&mut a, "/ip4/127.0.0.1/udp/0/quic-v1").await; drop(a); // stop a so b can never reach it match dial(&mut b, addr).await { @@ -56,7 +56,7 @@ async fn endpoint_reuse() { let (_, mut a_transport) = create_transport::(); let (_, mut b_transport) = create_transport::(); - let a_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let a_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic-v1").await; let ((_, b_send_back_addr, _), _) = connect(&mut a_transport, &mut b_transport, a_addr.clone()).await; @@ -68,7 +68,7 @@ async fn endpoint_reuse() { } }; - let b_addr = start_listening(&mut b_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let b_addr = start_listening(&mut b_transport, "/ip4/127.0.0.1/udp/0/quic-v1").await; let ((_, a_send_back_addr, _), _) = connect(&mut b_transport, &mut a_transport, b_addr).await; assert_eq!(a_send_back_addr, a_addr); @@ -81,7 +81,7 @@ async fn ipv4_dial_ipv6() { let (a_peer_id, mut a_transport) = create_transport::(); let (b_peer_id, mut b_transport) = create_transport::(); - let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; + let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic-v1").await; let ((a_connected, _, _), (b_connected, _)) = connect(&mut a_transport, &mut b_transport, a_addr).await; @@ -98,7 +98,7 @@ async fn wrong_peerid() { let (a_peer_id, mut a_transport) = create_transport::(); let (b_peer_id, mut b_transport) = create_transport::(); - let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic").await; + let a_addr = start_listening(&mut a_transport, "/ip6/::1/udp/0/quic-v1").await; let a_addr_random_peer = a_addr.with(Protocol::P2p(PeerId::random().into())); let ((a_connected, _, _), (b_connected, _)) = @@ -144,7 +144,7 @@ async fn tcp_and_quic() { let (a_peer_id, mut a_transport) = new_tcp_quic_transport(); let (b_peer_id, mut b_transport) = new_tcp_quic_transport(); - let quic_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let quic_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic-v1").await; let tcp_addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/tcp/0").await; let ((a_connected, _, _), (b_connected, _)) = @@ -188,7 +188,7 @@ async fn smoke() { let (a_peer_id, mut a_transport) = create_transport::

(); let (b_peer_id, mut b_transport) = create_transport::

(); - let addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic").await; + let addr = start_listening(&mut a_transport, "/ip4/127.0.0.1/udp/0/quic-v1").await; let ((a_connected, _, _), (b_connected, _)) = connect(&mut a_transport, &mut b_transport, addr).await; @@ -243,7 +243,7 @@ fn prop( async move { let (peer_id, mut listener) = create_transport::

(); - let addr = start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic").await; + let addr = start_listening(&mut listener, "/ip4/127.0.0.1/udp/0/quic-v1").await; listeners_tx.send((peer_id, addr)).await.unwrap(); diff --git a/transports/quic/tests/stream_compliance.rs b/transports/quic/tests/stream_compliance.rs index b8f95178879..fc652f0d854 100644 --- a/transports/quic/tests/stream_compliance.rs +++ b/transports/quic/tests/stream_compliance.rs @@ -23,7 +23,7 @@ async fn connected_peers() -> (quic::Connection, quic::Connection) { let mut listener = new_transport().boxed(); listener - .listen_on("/ip4/127.0.0.1/udp/0/quic".parse().unwrap()) + .listen_on("/ip4/127.0.0.1/udp/0/quic-v1".parse().unwrap()) .unwrap(); let listen_address = listener.next().await.unwrap().into_new_address().unwrap(); diff --git a/transports/tcp/src/lib.rs b/transports/tcp/src/lib.rs index 34ea22377fb..6e760ae5024 100644 --- a/transports/tcp/src/lib.rs +++ b/transports/tcp/src/lib.rs @@ -1326,7 +1326,7 @@ mod tests { let quic_addr = Multiaddr::empty() .with(Protocol::Ip4(Ipv4Addr::new(87, 65, 43, 21))) .with(Protocol::Udp(1)) - .with(Protocol::Quic); + .with(Protocol::QuicV1); assert!(transport .address_translation(&tcp_listen_addr, &quic_addr)