From 8f2aba2ea777f3dba8e9decbfc82aeff0ab8b3e3 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 13:36:58 +0100 Subject: [PATCH 01/29] Basic using of QUIC NAT Traversal for holepunching --- .../magicsock/endpoint_map/endpoint_state.rs | 228 ++++++++---------- iroh/src/magicsock/endpoint_map/path_state.rs | 2 +- 2 files changed, 101 insertions(+), 129 deletions(-) diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index 08ba5951fb..f23e9b3211 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -36,6 +36,12 @@ use crate::{ util::MaybeFuture, }; +/// How often to attempt holepunching. +/// +/// If there have been no changes to the NAT address candidates, holepunching will not be +/// attempted more frequently than at this interval. +const HOLEPUNCH_ATTEMPTS_INTERVAL: Duration = Duration::from_secs(5); + // TODO: use this // /// Number of addresses that are not active that we keep around per endpoint. // /// @@ -76,6 +82,12 @@ type PathEvents = MergeUnbounded< >, >; +/// A stream of events of announced NAT traversal candidate addresses for all connections. +/// +/// The connection is identified using [`ConnId`]. +type AddrEvents = + MergeUnbounded)> + Send + Sync>>>; + /// The state we need to know about a single remote endpoint. /// /// This actor manages all connections to the remote endpoint. It will trigger holepunching @@ -109,6 +121,8 @@ pub(super) struct EndpointStateActor { connections: FxHashMap, /// Events emitted by Quinn about path changes, for all paths, all connections. path_events: PathEvents, + /// A stream of events of announced NAT traversal candidate addresses for all connections. + addr_events: AddrEvents, // Internal state - Holepunching and path state. // @@ -158,6 +172,7 @@ impl EndpointStateActor { disco, connections: FxHashMap::default(), path_events: Default::default(), + addr_events: Default::default(), paths: FxHashMap::default(), last_holepunch: None, selected_path: None, @@ -228,6 +243,11 @@ impl EndpointStateActor { Some((id, evt)) = self.path_events.next() => { self.handle_path_event(id, evt); } + Some((id, evt)) = self.addr_events.next() => { + self.handle_addr_event(id, evt); + trace!("remote addrs updated, triggering holepunching"); + self.trigger_holepunching().await; + } _ = self.local_addrs.updated() => { trace!("local addrs updated, triggering holepunching"); self.trigger_holepunching().await; @@ -330,15 +350,20 @@ impl EndpointStateActor { // This is a good time to clean up connections. self.cleanup_connections(); - // Store the connection and hook up paths events stream. - let events = BroadcastStream::new(conn.path_events()); - let stream = events.map(move |evt| (conn_id, evt)); - self.path_events.push(Box::pin(stream)); + // Hook up paths and NAT addresses event streams. + self.path_events.push(Box::pin( + BroadcastStream::new(conn.path_events()).map(move |evt| (conn_id, evt)), + )); + self.addr_events + .push(Box::pin(conn.addr_events().map(move |evt| (conn_id, evt)))); + + // Store the connection. self.connections.insert( conn_id, ConnectionState { handle: handle.clone(), pub_path_info: paths_info, + nat_candidates: Default::default(), paths: Default::default(), open_paths: Default::default(), path_ids: Default::default(), @@ -529,58 +554,54 @@ impl EndpointStateActor { /// /// This will manage the entire process of holepunching with the remote endpoint. /// - /// - If there already is a direct connection, nothing happens. - /// - If there is no relay address known, nothing happens. - /// - If there was a recent attempt, it will schedule holepunching instead. - /// - Unless there are new addresses to try. - /// - The scheduled attempt will only run if holepunching has not yet succeeded by - /// then. - /// - DISCO pings will be sent to addresses recently advertised in a call-me-maybe - /// message. - /// - A DISCO call-me-maybe message advertising our own addresses will be sent. - /// - /// If a next trigger needs to be scheduled the delay until when to call this again is - /// returned. + /// - Holepunching happens on the Connection with the lowest [`ConnId`] which is a + /// client. + /// - Both endpoints may initiate holepunching if both have a client connection. + /// - Any opened paths are opened on all other connections without holepunching. + /// - If there are no changes in local or remote candidate addresses since the + /// last attempt **and** there was a recent attempt, a trigger_holepunching call + /// will be scheduled instead. async fn trigger_holepunching(&mut self) { - const HOLEPUNCH_ATTEMPTS_INTERVAL: Duration = Duration::from_secs(5); - if self.connections.is_empty() { trace!("not holepunching: no connections"); return; } - if self - .selected_path - .as_ref() - .map(|addr| addr.is_ip()) - .unwrap_or_default() - { - // TODO: We should ping this path to make sure it still works. Because we now - // know things could be broken. - trace!("not holepunching: already have a direct connection"); - // TODO: If the latency is kind of bad we should retry holepunching at times. + let Some(conn) = self + .connections + .iter() + .filter_map(|(id, state)| state.handle.upgrade().map(|conn| (*id, conn))) + .filter(|(_, conn)| conn.side().is_client()) + .min_by_key(|(id, _)| *id) + .map(|(_, conn)| conn) + else { + trace!("not holepunching: no client connection"); return; - } - - let remote_addrs: BTreeSet = self.remote_hp_addrs(); - let local_addrs: BTreeSet = self + }; + let remote_candidates = BTreeSet::from_iter(conn.nat_candidates()); + let local_candidates: BTreeSet = self .local_addrs .get() .iter() .map(|daddr| daddr.addr) .collect(); - let new_addrs = self + let new_candidates = self .last_holepunch .as_ref() .map(|last_hp| { // Addrs are allowed to disappear, but if there are new ones we need to // holepunch again. - trace!(?last_hp, ?local_addrs, ?remote_addrs, "addrs to holepunch?"); - !remote_addrs.is_subset(&last_hp.remote_addrs) - || !local_addrs.is_subset(&last_hp.local_addrs) + trace!( + ?last_hp, + ?local_candidates, + ?remote_candidates, + "candidates to holepunch?" + ); + !remote_candidates.is_subset(&last_hp.remote_candidates) + || !local_candidates.is_subset(&last_hp.local_candidates) }) .unwrap_or(true); - if !new_addrs { + if !new_candidates { if let Some(ref last_hp) = self.last_holepunch { let next_hp = last_hp.when + HOLEPUNCH_ATTEMPTS_INTERVAL; let now = Instant::now(); @@ -592,101 +613,36 @@ impl EndpointStateActor { } } - self.do_holepunching().await; - } - - /// Returns the remote addresses to holepunch against. - fn remote_hp_addrs(&self) -> BTreeSet { - const CALL_ME_MAYBE_VALIDITY: Duration = Duration::from_secs(30); - - self.paths - .iter() - .filter_map(|(addr, state)| match addr { - transports::Addr::Ip(socket_addr) => Some((socket_addr, state)), - transports::Addr::Relay(_, _) => None, - }) - .filter_map(|(addr, state)| { - if state - .sources - .get(&Source::CallMeMaybe) - .map(|when| when.elapsed() <= CALL_ME_MAYBE_VALIDITY) - .unwrap_or_default() - || state - .sources - .get(&Source::Ping) - .map(|when| when.elapsed() <= CALL_ME_MAYBE_VALIDITY) - .unwrap_or_default() - { - Some(*addr) - } else { - None - } - }) - .collect() + self.do_holepunching(conn).await; } /// Unconditionally perform holepunching. - /// - /// - DISCO pings will be sent to addresses recently advertised in a call-me-maybe - /// message. - /// - A DISCO call-me-maybe message advertising our own addresses will be sent. #[instrument(skip_all)] - async fn do_holepunching(&mut self) { - let Some(relay_addr) = self - .paths - .iter() - .filter_map(|(addr, _)| match addr { - transports::Addr::Ip(_) => None, - transports::Addr::Relay(_, _) => Some(addr), - }) - .next() - .cloned() - else { - warn!("holepunching requested but have no relay address"); - return; - }; - let remote_addrs = self.remote_hp_addrs(); - - // Send DISCO Ping messages to all CallMeMaybe-advertised paths. - for dst in remote_addrs.iter() { - let msg = disco::Ping::new(self.local_endpoint_id); - event!( - target: "iroh::_events::ping::sent", - Level::DEBUG, - remote = %self.endpoint_id.fmt_short(), - ?dst, - txn = ?msg.tx_id, - ); - let addr = transports::Addr::Ip(*dst); - self.paths.entry(addr.clone()).or_default().ping_sent = Some(msg.tx_id); - self.send_disco_message(addr, disco::Message::Ping(msg)) - .await; - } - - // Send the DISCO CallMeMaybe message over the relay. - let my_numbers: Vec = self + async fn do_holepunching(&mut self, conn: quinn::Connection) { + let local_candidates = self .local_addrs .get() .iter() .map(|daddr| daddr.addr) - .collect(); - let local_addrs: BTreeSet = my_numbers.iter().copied().collect(); - let msg = disco::CallMeMaybe { my_numbers }; - event!( - target: "iroh::_events::call_me_maybe::sent", - Level::DEBUG, - remote = %self.endpoint_id.fmt_short(), - dst = ?relay_addr, - my_numbers = ?msg.my_numbers, - ); - self.send_disco_message(relay_addr, disco::Message::CallMeMaybe(msg)) - .await; - - self.last_holepunch = Some(HolepunchAttempt { - when: Instant::now(), - local_addrs, - remote_addrs, - }); + .collect::>(); + match conn.initiate_nat_traversal(Vec::from_iter(local_candidates.iter().copied())) { + Ok(remote_candidates) => { + trace!( + ?local_candidates, + ?remote_candidates, + "nat traversal initiated" + ); + self.last_holepunch = Some(HolepunchAttempt { + when: Instant::now(), + local_candidates, + remote_candidates: BTreeSet::from_iter(remote_candidates), + }); + } + Err(_) => { + // TODO: log error + warn!("failed to initiate NAT traversal"); + } + } } /// Sends a DISCO message to the remote endpoint this actor manages. @@ -865,6 +821,18 @@ impl EndpointStateActor { } } + /// Handles a NAT address candidate added or removed by the server. + /// + /// When the server adds an address we want to (re-)holepunch. Same for when a local + /// address changes, though that is not handled here. + fn handle_addr_event(&mut self, conn_id: ConnId, event: Vec) { + if let Some(conn_state) = self.connections.get_mut(&conn_id) { + let _ = std::mem::replace(&mut conn_state.nat_candidates, event); + } else { + warn!(?conn_id, "NAT candidate event for unknown connection"); + } + } + /// Clean up connections which no longer exist. // TODO: Call this on a schedule. fn cleanup_connections(&mut self) { @@ -1043,11 +1011,11 @@ struct HolepunchAttempt { /// /// We do not store this as a [`DirectAddr`] because this is checked for equality and we /// do not want to compare the sources of these addresses. - local_addrs: BTreeSet, + local_candidates: BTreeSet, /// The set of remote addresses which could take part in holepunching. /// - /// Like `local_addrs` we may not have used them. - remote_addrs: BTreeSet, + /// Like [`Self::local_candidates`] we may not have used them. + remote_candidates: BTreeSet, } /// The type of connection we have to the endpoint. @@ -1075,7 +1043,7 @@ pub enum ConnectionType { /// /// The wrapped value is the [`Connection::stable_id`] value, and is thus only valid for /// active connections. -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] struct ConnId(usize); /// State about one connection. @@ -1085,6 +1053,10 @@ struct ConnectionState { handle: WeakConnectionHandle, /// The information we publish to users about the paths used in this connection. pub_path_info: Watchable>, + /// NAT candidate addresses we received from the server. + /// + /// If we are the server for this connection, this will remain empty. + nat_candidates: Vec, /// The paths that exist on this connection. /// /// This could be in any state, e.g. while still validating the path or already closed diff --git a/iroh/src/magicsock/endpoint_map/path_state.rs b/iroh/src/magicsock/endpoint_map/path_state.rs index 44fdef65b9..68f27199c5 100644 --- a/iroh/src/magicsock/endpoint_map/path_state.rs +++ b/iroh/src/magicsock/endpoint_map/path_state.rs @@ -20,6 +20,6 @@ pub(super) struct PathState { /// We keep track of only the latest [`Instant`] for each [`Source`], keeping the size /// of the map of sources down to one entry per type of source. pub(super) sources: HashMap, - /// The last ping sent on this path. + /// The last DISCO ping sent on this path. pub(super) ping_sent: Option, } From bbf947ac5fe5a5d00cd7bf0491ba311075af2056 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 17:15:25 +0100 Subject: [PATCH 02/29] Do not receive DISCO messages, we no longer need them --- iroh/src/magicsock.rs | 188 ++++-------------- iroh/src/magicsock/endpoint_map.rs | 43 +--- .../magicsock/endpoint_map/endpoint_state.rs | 17 -- 3 files changed, 40 insertions(+), 208 deletions(-) diff --git a/iroh/src/magicsock.rs b/iroh/src/magicsock.rs index 42b6d1ba19..49dcdbf072 100644 --- a/iroh/src/magicsock.rs +++ b/iroh/src/magicsock.rs @@ -578,165 +578,55 @@ impl MagicSock { #[cfg(windows)] let dst_ip = None; - let mut quic_packets_total = 0; - - for ((quinn_meta, buf), source_addr) in metas - .iter_mut() - .zip(bufs.iter_mut()) - .zip(source_addrs.iter()) - { - let mut buf_contains_quic_datagrams = false; - let mut quic_datagram_count = 0; + for (quinn_meta, source_addr) in metas.iter_mut().zip(source_addrs.iter()) { + let datagram_count = quinn_meta.len.div_ceil(quinn_meta.stride); + self.metrics + .magicsock + .recv_datagrams + .inc_by(datagram_count as _); if quinn_meta.len > quinn_meta.stride { - trace!(%quinn_meta.len, %quinn_meta.stride, "GRO datagram received"); + trace!( + src = ?source_addr, + len = quinn_meta.len, + stride = %quinn_meta.stride, + datagram_count = quinn_meta.len.div_ceil(quinn_meta.stride), + "GRO datagram received", + ); self.metrics.magicsock.recv_gro_datagrams.inc(); + } else { + trace!(src = ?source_addr, len = quinn_meta.len, "datagram received"); } - - // Chunk through the datagrams in this GRO payload to find disco - // packets and forward them to the actor - for datagram in buf[..quinn_meta.len].chunks_mut(quinn_meta.stride) { - if datagram.len() < quinn_meta.stride { - trace!( - len = %datagram.len(), - %quinn_meta.stride, - "Last GRO datagram smaller than stride", - ); + match source_addr { + transports::Addr::Ip(SocketAddr::V4(..)) => { + self.metrics + .magicsock + .recv_data_ipv4 + .inc_by(quinn_meta.len as _); } - - // Detect DISCO datagrams and process them. Overwrite the first - // byte of those packets with zero to make Quinn ignore the packet. This - // relies on quinn::EndpointConfig::grease_quic_bit being set to `false`, - // which we do in Endpoint::bind. - if let Some((sender, sealed_box)) = disco::source_and_box(datagram) { - trace!(src = ?source_addr, len = datagram.len(), "UDP recv: DISCO packet"); - self.handle_disco_message(sender, sealed_box, source_addr); - datagram[0] = 0u8; - } else { - trace!(src = ?source_addr, len = datagram.len(), "UDP recv: QUIC packet"); - match source_addr { - transports::Addr::Ip(SocketAddr::V4(..)) => { - self.metrics - .magicsock - .recv_data_ipv4 - .inc_by(datagram.len() as _); - } - transports::Addr::Ip(SocketAddr::V6(..)) => { - self.metrics - .magicsock - .recv_data_ipv6 - .inc_by(datagram.len() as _); - } - transports::Addr::Relay(..) => { - self.metrics - .magicsock - .recv_data_relay - .inc_by(datagram.len() as _); - } - } - - quic_datagram_count += 1; - buf_contains_quic_datagrams = true; + transports::Addr::Ip(SocketAddr::V6(..)) => { + self.metrics + .magicsock + .recv_data_ipv6 + .inc_by(quinn_meta.len as _); } - } - - if buf_contains_quic_datagrams { - match source_addr { - #[cfg(wasm_browser)] - transports::Addr::Ip(_addr) => { - panic!("cannot use IP based addressing in the browser"); - } - #[cfg(not(wasm_browser))] - transports::Addr::Ip(_addr) => { - quic_packets_total += quic_datagram_count; - } - transports::Addr::Relay(src_url, src_node) => { - let mapped_addr = self - .endpoint_map - .relay_mapped_addrs - .get(&(src_url.clone(), *src_node)); - quinn_meta.addr = mapped_addr.private_socket_addr(); - } + transports::Addr::Relay(src_url, src_node) => { + self.metrics + .magicsock + .recv_data_relay + .inc_by(quinn_meta.len as _); + + // Fill in the correct mapped address + let mapped_addr = self + .endpoint_map + .relay_mapped_addrs + .get(&(src_url.clone(), *src_node)); + quinn_meta.addr = mapped_addr.private_socket_addr(); } - } else { - // If all datagrams in this buf are DISCO, set len to zero to make - // Quinn skip the buf completely. - quinn_meta.len = 0; } + // Normalize local_ip quinn_meta.dst_ip = dst_ip; } - - if quic_packets_total > 0 { - self.metrics - .magicsock - .recv_datagrams - .inc_by(quic_packets_total as _); - trace!("UDP recv: {} packets", quic_packets_total); - } - } - - /// Handles a discovery message. - #[instrument("disco_in", skip_all, fields(endpoint = %sender.fmt_short(), ?src))] - fn handle_disco_message(&self, sender: PublicKey, sealed_box: &[u8], src: &transports::Addr) { - if self.is_closed() { - return; - } - - if let transports::Addr::Relay(_, endpoint_id) = src { - if endpoint_id != &sender { - // TODO: return here? - warn!( - "Received relay disco message from connection for {}, but with message from {}", - endpoint_id.fmt_short(), - sender.fmt_short() - ); - } - } - - // We're now reasonably sure we're expecting communication from - // this endpoint, do the heavy crypto lifting to see what they want. - let dm = match self.disco.unseal_and_decode(sender, sealed_box) { - Ok(dm) => dm, - Err(DiscoBoxError::Open { source, .. }) => { - warn!(?source, "failed to open disco box"); - self.metrics.magicsock.recv_disco_bad_key.inc(); - return; - } - Err(DiscoBoxError::Parse { source, .. }) => { - // Couldn't parse it, but it was inside a correctly - // signed box, so just ignore it, assuming it's from a - // newer version of Tailscale that we don't - // understand. Not even worth logging about, lest it - // be too spammy for old clients. - - self.metrics.magicsock.recv_disco_bad_parse.inc(); - debug!(?source, "failed to parse disco message"); - return; - } - }; - - if src.is_relay() { - self.metrics.magicsock.recv_disco_relay.inc(); - } else { - self.metrics.magicsock.recv_disco_udp.inc(); - } - - trace!(?dm, "receive disco message"); - match dm { - disco::Message::Ping(ping) => { - self.metrics.magicsock.recv_disco_ping.inc(); - self.endpoint_map.handle_ping(ping, sender, src.clone()); - } - disco::Message::Pong(pong) => { - self.metrics.magicsock.recv_disco_pong.inc(); - self.endpoint_map.handle_pong(pong, sender, src.clone()); - } - disco::Message::CallMeMaybe(cm) => { - self.metrics.magicsock.recv_disco_call_me_maybe.inc(); - self.endpoint_map - .handle_call_me_maybe(cm, sender, src.clone()); - } - } } /// Sends out a disco message. diff --git a/iroh/src/magicsock/endpoint_map.rs b/iroh/src/magicsock/endpoint_map.rs index 4ba29ef327..4a80e0ff62 100644 --- a/iroh/src/magicsock/endpoint_map.rs +++ b/iroh/src/magicsock/endpoint_map.rs @@ -9,7 +9,7 @@ use iroh_base::{EndpointAddr, EndpointId, RelayUrl, TransportAddr}; use n0_future::task::{self, AbortOnDropHandle}; use serde::{Deserialize, Serialize}; use tokio::sync::mpsc; -use tracing::{Instrument, error, info_span, trace, warn}; +use tracing::{Instrument, error, info_span, trace}; #[cfg(any(test, feature = "test-utils"))] use super::transports::TransportsSender; @@ -20,7 +20,6 @@ use super::{ mapped_addrs::{AddrMap, EndpointIdMappedAddr, MultipathMappedAddr, RelayMappedAddr}, transports::{self, OwnedTransmit}, }; -use crate::disco::{self}; #[cfg(any(test, feature = "test-utils"))] use crate::endpoint::PathSelection; @@ -223,46 +222,6 @@ impl EndpointMap { } } } - - pub(super) fn handle_ping(&self, msg: disco::Ping, sender: EndpointId, src: transports::Addr) { - if msg.endpoint_key != sender { - warn!("DISCO Ping EndpointId mismatch, ignoring ping"); - return; - } - let endpoint_state = self.endpoint_state_actor(sender); - if let Err(err) = endpoint_state.try_send(EndpointStateMessage::PingReceived(msg, src)) { - // TODO: This is really, really bad and will drop pings under load. But - // DISCO pings are going away with QUIC-NAT-TRAVERSAL so I don't care. - warn!("DISCO Ping dropped: {err:#}"); - } - } - - pub(super) fn handle_pong(&self, msg: disco::Pong, sender: EndpointId, src: transports::Addr) { - let actor = self.endpoint_state_actor(sender); - if let Err(err) = actor.try_send(EndpointStateMessage::PongReceived(msg, src)) { - // TODO: This is really, really bad and will drop pongs under load. But - // DISCO pongs are going away with QUIC-NAT-TRAVERSAL so I don't care. - warn!("DISCO Pong dropped: {err:#}"); - } - } - - pub(super) fn handle_call_me_maybe( - &self, - msg: disco::CallMeMaybe, - sender: EndpointId, - src: transports::Addr, - ) { - if !src.is_relay() { - warn!("DISCO CallMeMaybe packets should only come via relay"); - return; - } - let actor = self.endpoint_state_actor(sender); - if let Err(err) = actor.try_send(EndpointStateMessage::CallMeMaybeReceived(msg)) { - // TODO: This is bad and will drop call-me-maybe's under load. But - // DISCO CallMeMaybe going away with QUIC-NAT-TRAVERSAL so I don't care. - warn!("DISCO CallMeMaybe dropped: {err:#}"); - } - } } impl EndpointMapInner { diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index f23e9b3211..b959bd68b7 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -287,15 +287,6 @@ impl EndpointStateActor { EndpointStateMessage::AddEndpointAddr(addr, source) => { self.handle_msg_add_endpoint_addr(addr, source); } - EndpointStateMessage::CallMeMaybeReceived(msg) => { - self.handle_msg_call_me_maybe_received(msg).await; - } - EndpointStateMessage::PingReceived(ping, src) => { - self.handle_msg_ping_received(ping, src).await; - } - EndpointStateMessage::PongReceived(pong, src) => { - self.handle_msg_pong_received(pong, src); - } EndpointStateMessage::CanSend(tx) => { self.handle_msg_can_send(tx); } @@ -970,14 +961,6 @@ pub(crate) enum EndpointStateMessage { ), /// Adds a [`EndpointAddr`] with locations where the endpoint might be reachable. AddEndpointAddr(EndpointAddr, Source), - /// Process a received DISCO CallMeMaybe message. - CallMeMaybeReceived(disco::CallMeMaybe), - /// Process a received DISCO Ping message. - #[debug("PingReceived({:?}, src: {_1:?})", _0.tx_id)] - PingReceived(disco::Ping, transports::Addr), - /// Process a received DISCO Pong message. - #[debug("PongReceived({:?}, src: {_1:?})", _0.tx_id)] - PongReceived(disco::Pong, transports::Addr), /// Asks if there is any possible path that could be used. /// /// This does not mean there is any guarantee that the remote endpoint is reachable. From 697583ea06cb3034177bf7046cebffdbc66dcced Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 17:26:02 +0100 Subject: [PATCH 03/29] remove disco from endpoint_state.rs --- iroh/src/magicsock/endpoint_map.rs | 2 - .../magicsock/endpoint_map/endpoint_state.rs | 115 +----------------- iroh/src/magicsock/endpoint_map/path_state.rs | 3 - 3 files changed, 1 insertion(+), 119 deletions(-) diff --git a/iroh/src/magicsock/endpoint_map.rs b/iroh/src/magicsock/endpoint_map.rs index 4a80e0ff62..703838a369 100644 --- a/iroh/src/magicsock/endpoint_map.rs +++ b/iroh/src/magicsock/endpoint_map.rs @@ -201,14 +201,12 @@ impl EndpointMap { // Create a new EndpointStateActor and insert it into the endpoint map. let sender = inner.transports_handle.inbox.clone(); let local_addrs = inner.local_addrs.clone(); - let disco = inner.disco.clone(); let metrics = inner.metrics.clone(); let actor = EndpointStateActor::new( eid, self.local_endpoint_id, sender, local_addrs, - disco, self.relay_mapped_addrs.clone(), metrics, ); diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index b959bd68b7..2c0ea64bac 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -26,10 +26,9 @@ use super::{Source, TransportsSenderMessage, path_state::PathState}; // #[cfg(any(test, feature = "test-utils"))] // use crate::endpoint::PathSelection; use crate::{ - disco::{self}, endpoint::DirectAddr, magicsock::{ - DiscoState, HEARTBEAT_INTERVAL, MagicsockMetrics, PATH_MAX_IDLE_TIMEOUT, + HEARTBEAT_INTERVAL, MagicsockMetrics, PATH_MAX_IDLE_TIMEOUT, mapped_addrs::{AddrMap, MappedAddr, MultipathMappedAddr, RelayMappedAddr}, transports::{self, OwnedTransmit}, }, @@ -110,8 +109,6 @@ pub(super) struct EndpointStateActor { /// /// These are our local addresses and any reflexive transport addresses. local_addrs: n0_watcher::Direct>, - /// Shared state to allow to encrypt DISCO messages to peers. - disco: DiscoState, /// The mapping between endpoints via a relay and their [`RelayMappedAddr`]s. relay_mapped_addrs: AddrMap<(RelayUrl, EndpointId), RelayMappedAddr>, @@ -158,7 +155,6 @@ impl EndpointStateActor { local_endpoint_id: EndpointId, transports_sender: mpsc::Sender, local_addrs: n0_watcher::Direct>, - disco: DiscoState, relay_mapped_addrs: AddrMap<(RelayUrl, EndpointId), RelayMappedAddr>, metrics: Arc, ) -> Self { @@ -169,7 +165,6 @@ impl EndpointStateActor { transports_sender, local_addrs, relay_mapped_addrs, - disco, connections: FxHashMap::default(), path_events: Default::default(), addr_events: Default::default(), @@ -428,90 +423,6 @@ impl EndpointStateActor { trace!("added addressing information"); } - /// Handles [`EndpointStateMessage::CallMeMaybeReceived`]. - async fn handle_msg_call_me_maybe_received(&mut self, msg: disco::CallMeMaybe) { - event!( - target: "iroh::_events::call_me_maybe::recv", - Level::DEBUG, - remote = %self.endpoint_id.fmt_short(), - addrs = ?msg.my_numbers, - ); - let now = Instant::now(); - for addr in msg.my_numbers { - let dst = transports::Addr::Ip(addr); - let ping = disco::Ping::new(self.local_endpoint_id); - - let path = self.paths.entry(dst.clone()).or_default(); - path.sources.insert(Source::CallMeMaybe, now); - path.ping_sent = Some(ping.tx_id); - - event!( - target: "iroh::_events::ping::sent", - Level::DEBUG, - remote = %self.endpoint_id.fmt_short(), - ?dst, - ); - self.send_disco_message(dst, disco::Message::Ping(ping)) - .await; - } - } - - /// Handles [`EndpointStateMessage::PingReceived`]. - async fn handle_msg_ping_received(&mut self, ping: disco::Ping, src: transports::Addr) { - let transports::Addr::Ip(addr) = src else { - warn!("received ping via relay transport, ignored"); - return; - }; - event!( - target: "iroh::_events::ping::recv", - Level::DEBUG, - remote = %self.endpoint_id.fmt_short(), - ?src, - txn = ?ping.tx_id, - ); - let pong = disco::Pong { - tx_id: ping.tx_id, - ping_observed_addr: addr.into(), - }; - event!( - target: "iroh::_events::pong::sent", - Level::DEBUG, - remote = %self.endpoint_id.fmt_short(), - dst = ?src, - txn = ?pong.tx_id, - ); - self.send_disco_message(src.clone(), disco::Message::Pong(pong)) - .await; - - let path = self.paths.entry(src).or_default(); - path.sources.insert(Source::Ping, Instant::now()); - - trace!("ping received, triggering holepunching"); - self.trigger_holepunching().await; - } - - /// Handles [`EndpointStateMessage::PongReceived`]. - fn handle_msg_pong_received(&mut self, pong: disco::Pong, src: transports::Addr) { - let Some(state) = self.paths.get(&src) else { - warn!(path = ?src, ?self.paths, "ignoring DISCO Pong for unknown path"); - return; - }; - if state.ping_sent != Some(pong.tx_id) { - debug!(path = ?src, ?state.ping_sent, pong_tx = ?pong.tx_id, - "ignoring unknown DISCO Pong for path"); - return; - } - event!( - target: "iroh::_events::pong::recv", - Level::DEBUG, - remote_endpoint = %self.endpoint_id.fmt_short(), - ?src, - txn = ?pong.tx_id, - ); - - self.open_path(&src); - } - /// Handles [`EndpointStateMessage::CanSend`]. fn handle_msg_can_send(&self, tx: oneshot::Sender) { let can_send = !self.paths.is_empty(); @@ -636,30 +547,6 @@ impl EndpointStateActor { } } - /// Sends a DISCO message to the remote endpoint this actor manages. - #[instrument(skip(self), fields(remote = %self.endpoint_id.fmt_short()))] - async fn send_disco_message(&self, dst: transports::Addr, msg: disco::Message) { - let pkt = self.disco.encode_and_seal(self.endpoint_id, &msg); - let transmit = transports::OwnedTransmit { - ecn: None, - contents: pkt, - segment_size: None, - }; - let counter = match dst { - transports::Addr::Ip(_) => &self.metrics.send_disco_udp, - transports::Addr::Relay(_, _) => &self.metrics.send_disco_relay, - }; - match self.transports_sender.send((dst, transmit).into()).await { - Ok(()) => { - trace!("sent"); - counter.inc(); - } - Err(err) => { - warn!("failed to send disco message: {err:#}"); - } - } - } - /// Open the path on all connections. /// /// This goes through all the connections for which we are the client, and makes sure diff --git a/iroh/src/magicsock/endpoint_map/path_state.rs b/iroh/src/magicsock/endpoint_map/path_state.rs index 68f27199c5..c087a28eaa 100644 --- a/iroh/src/magicsock/endpoint_map/path_state.rs +++ b/iroh/src/magicsock/endpoint_map/path_state.rs @@ -5,7 +5,6 @@ use std::collections::HashMap; use n0_future::time::Instant; use super::Source; -use crate::disco::TransactionId; /// The state of a single path to the remote endpoint. /// @@ -20,6 +19,4 @@ pub(super) struct PathState { /// We keep track of only the latest [`Instant`] for each [`Source`], keeping the size /// of the map of sources down to one entry per type of source. pub(super) sources: HashMap, - /// The last DISCO ping sent on this path. - pub(super) ping_sent: Option, } From 343a533a178c28142b436442d7802f2379fdb0e8 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 17:33:14 +0100 Subject: [PATCH 04/29] remove disco from endpoint_map.rs --- iroh/src/magicsock.rs | 1 - iroh/src/magicsock/endpoint_map.rs | 14 +++----------- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/iroh/src/magicsock.rs b/iroh/src/magicsock.rs index 49dcdbf072..2acca418c2 100644 --- a/iroh/src/magicsock.rs +++ b/iroh/src/magicsock.rs @@ -922,7 +922,6 @@ impl Handle { metrics.magicsock.clone(), sender, direct_addrs.addrs.watch(), - disco.clone(), ) }; diff --git a/iroh/src/magicsock/endpoint_map.rs b/iroh/src/magicsock/endpoint_map.rs index 703838a369..24d64f5bbd 100644 --- a/iroh/src/magicsock/endpoint_map.rs +++ b/iroh/src/magicsock/endpoint_map.rs @@ -16,7 +16,7 @@ use super::transports::TransportsSender; #[cfg(not(any(test, feature = "test-utils")))] use super::transports::TransportsSender; use super::{ - DirectAddr, DiscoState, MagicsockMetrics, + DirectAddr, MagicsockMetrics, mapped_addrs::{AddrMap, EndpointIdMappedAddr, MultipathMappedAddr, RelayMappedAddr}, transports::{self, OwnedTransmit}, }; @@ -53,7 +53,6 @@ pub(super) struct EndpointMapInner { /// Handle to an actor that can send over the transports. transports_handle: TransportsSenderHandle, local_addrs: n0_watcher::Direct>, - disco: DiscoState, #[cfg(any(test, feature = "test-utils"))] path_selection: PathSelection, /// The [`EndpointStateActor`] for each remote endpoint. @@ -94,10 +93,6 @@ pub enum Source { /// The name of the application that added the endpoint name: String, }, - /// The address was advertised by a call-me-maybe DISCO message. - CallMeMaybe, - /// We received a ping on the path. - Ping, /// We established a connection on this address. /// /// Currently this means the path was in uses as [`PathId::ZERO`] when the a connection @@ -116,14 +111,13 @@ impl EndpointMap { metrics: Arc, sender: TransportsSender, local_addrs: n0_watcher::Direct>, - disco: DiscoState, ) -> Self { #[cfg(not(any(test, feature = "test-utils")))] - let inner = EndpointMapInner::new(metrics, sender, local_addrs, disco); + let inner = EndpointMapInner::new(metrics, sender, local_addrs); #[cfg(any(test, feature = "test-utils"))] let inner = { - let mut inner = EndpointMapInner::new(metrics, sender, local_addrs, disco); + let mut inner = EndpointMapInner::new(metrics, sender, local_addrs); inner.path_selection = path_selection; inner }; @@ -227,14 +221,12 @@ impl EndpointMapInner { metrics: Arc, sender: TransportsSender, local_addrs: n0_watcher::Direct>, - disco: DiscoState, ) -> Self { let transports_handle = Self::start_transports_sender(sender); Self { metrics, transports_handle, local_addrs, - disco, #[cfg(any(test, feature = "test-utils"))] path_selection: Default::default(), endpoint_states: Default::default(), From 22b80f1fed2e507ef2d6a114fe01d59f7c5c1635 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 17:51:35 +0100 Subject: [PATCH 05/29] remove disco from magicsock.rs --- iroh/src/magicsock.rs | 192 +----------------------------------------- 1 file changed, 2 insertions(+), 190 deletions(-) diff --git a/iroh/src/magicsock.rs b/iroh/src/magicsock.rs index 2acca418c2..934ad44f8f 100644 --- a/iroh/src/magicsock.rs +++ b/iroh/src/magicsock.rs @@ -26,7 +26,6 @@ use std::{ }, }; -use bytes::Bytes; use iroh_base::{EndpointAddr, EndpointId, PublicKey, RelayUrl, SecretKey, TransportAddr}; use iroh_relay::{RelayConfig, RelayMap}; use n0_future::{ @@ -52,7 +51,7 @@ use self::transports::IpTransport; use self::{ endpoint_map::{EndpointMap, EndpointStateMessage}, metrics::Metrics as MagicsockMetrics, - transports::{RelayActorConfig, RelayTransport, Transports, TransportsSender}, + transports::{RelayActorConfig, RelayTransport, Transports}, }; #[cfg(not(wasm_browser))] use crate::dns::DnsResolver; @@ -62,9 +61,7 @@ use crate::endpoint::PathSelection; use crate::net_report::QuicConfig; use crate::{ defaults::timeouts::NET_REPORT_TIMEOUT, - disco::{self, SendAddr}, discovery::{ConcurrentDiscovery, Discovery, EndpointData, UserData}, - key::{DecryptionError, SharedSecret, public_ed_box, secret_ed_box}, metrics::EndpointMetrics, net_report::{self, IfStateDetails, Report}, }; @@ -209,9 +206,6 @@ pub(crate) struct MagicSock { dns_resolver: DnsResolver, relay_map: RelayMap, - /// Disco - disco: DiscoState, - // - Discovery /// Optional discovery service discovery: ConcurrentDiscovery, @@ -629,50 +623,6 @@ impl MagicSock { } } - /// Sends out a disco message. - async fn send_disco_message( - &self, - sender: &TransportsSender, - dst: SendAddr, - dst_key: PublicKey, - msg: disco::Message, - ) -> io::Result<()> { - let dst = match dst { - SendAddr::Udp(addr) => transports::Addr::Ip(addr), - SendAddr::Relay(url) => transports::Addr::Relay(url, dst_key), - }; - - trace!(?dst, %msg, "send disco message (UDP)"); - if self.is_closed() { - return Err(io::Error::new( - io::ErrorKind::NotConnected, - "connection closed", - )); - } - - let pkt = self.disco.encode_and_seal(dst_key, &msg); - - let transmit = transports::Transmit { - contents: &pkt, - ecn: None, - segment_size: None, - }; - - let dst2 = dst.clone(); - match sender.send(&dst2, None, &transmit).await { - Ok(()) => { - trace!(?dst, %msg, "sent disco message"); - self.metrics.magicsock.sent_disco_udp.inc(); - disco_message_sent(&msg, &self.metrics.magicsock); - Ok(()) - } - Err(err) => { - warn!(?dst, ?msg, ?err, "failed to send disco message"); - Err(err) - } - } - } - /// Publishes our address to a discovery service, if configured. /// /// Called whenever our addresses or home relay endpoint changes. @@ -911,7 +861,6 @@ impl Handle { let transports = Transports::new(relay_transports); let direct_addrs = DiscoveredDirectAddrs::default(); - let (disco, disco_receiver) = DiscoState::new(&secret_key); let endpoint_map = { let sender = transports.create_sender(); @@ -929,7 +878,6 @@ impl Handle { public_key: secret_key.public(), closing: AtomicBool::new(false), closed: AtomicBool::new(false), - disco, actor_sender: actor_sender.clone(), ipv6_reported, endpoint_map, @@ -954,7 +902,6 @@ impl Handle { // the packet if grease_quic_bit is set to false. endpoint_config.grease_quic_bit(false); - let sender = transports.create_sender(); let local_addrs_watch = transports.local_addrs_watch(); let network_change_sender = transports.create_network_change_sender(); @@ -1027,8 +974,6 @@ impl Handle { direct_addr_update_state, network_change_sender, direct_addr_done_rx, - pending_call_me_maybes: Default::default(), - disco_receiver, }; // Initialize addresses #[cfg(not(wasm_browser))] @@ -1039,7 +984,7 @@ impl Handle { let actor_task = task::spawn( actor - .run(token, local_addrs_watch, sender) + .run(token, local_addrs_watch) .instrument(info_span!("actor")), ); @@ -1132,92 +1077,6 @@ fn default_quic_client_config() -> rustls::ClientConfig { .with_no_client_auth() } -#[derive(Debug, Clone)] -struct DiscoState { - /// The EndpointId/PublikeKey of this node. - this_id: EndpointId, - /// Encryption key for this endpoint. - secret_encryption_key: Arc, - /// The state for an active DiscoKey. - secrets: Arc>>, - /// Disco (ping) queue - sender: mpsc::Sender<(SendAddr, PublicKey, disco::Message)>, -} - -impl DiscoState { - fn new( - secret_key: &SecretKey, - ) -> (Self, mpsc::Receiver<(SendAddr, PublicKey, disco::Message)>) { - let this_id = secret_key.public(); - let secret_encryption_key = secret_ed_box(secret_key); - let (disco_sender, disco_receiver) = mpsc::channel(256); - - ( - Self { - this_id, - secret_encryption_key: Arc::new(secret_encryption_key), - secrets: Default::default(), - sender: disco_sender, - }, - disco_receiver, - ) - } - - fn try_send(&self, dst: SendAddr, dst_key: PublicKey, msg: disco::Message) -> bool { - self.sender.try_send((dst, dst_key, msg)).is_ok() - } - - fn encode_and_seal(&self, other_key: PublicKey, msg: &disco::Message) -> Bytes { - let mut seal = msg.as_bytes(); - self.get_secret(other_key, |secret| secret.seal(&mut seal)); - disco::encode_message(&self.this_id, seal).into() - } - - fn unseal_and_decode( - &self, - endpoint_key: PublicKey, - sealed_box: &[u8], - ) -> Result { - let mut sealed_box = sealed_box.to_vec(); - self.get_secret(endpoint_key, |secret| secret.open(&mut sealed_box)) - .context(OpenSnafu)?; - disco::Message::from_bytes(&sealed_box).context(ParseSnafu) - } - - fn get_secret(&self, endpoint_id: PublicKey, cb: F) -> T - where - F: FnOnce(&mut SharedSecret) -> T, - { - let mut inner = self.secrets.lock().expect("poisoned"); - let x = inner.entry(endpoint_id).or_insert_with(|| { - let public_key = public_ed_box(&endpoint_id); - SharedSecret::new(&self.secret_encryption_key, &public_key) - }); - cb(x) - } -} - -#[allow(missing_docs)] -#[common_fields({ - backtrace: Option, - #[snafu(implicit)] - span_trace: n0_snafu::SpanTrace, -})] -#[derive(Debug, Snafu)] -#[non_exhaustive] -enum DiscoBoxError { - #[snafu(display("Failed to open crypto box"))] - Open { - #[snafu(source(from(DecryptionError, Box::new)))] - source: Box, - }, - #[snafu(display("Failed to parse disco message"))] - Parse { - #[snafu(source(from(disco::ParseError, Box::new)))] - source: Box, - }, -} - #[derive(Debug)] #[allow(clippy::enum_variant_names)] enum ActorMessage { @@ -1239,11 +1098,6 @@ struct Actor { /// Indicates the direct addr update state. direct_addr_update_state: DirectAddrUpdateState, direct_addr_done_rx: mpsc::Receiver<()>, - - /// List of CallMeMaybe disco messages that should be sent out after - /// the next endpoint update completes - pending_call_me_maybes: HashMap, - disco_receiver: mpsc::Receiver<(SendAddr, PublicKey, disco::Message)>, } #[cfg(not(wasm_browser))] @@ -1301,7 +1155,6 @@ impl Actor { mut self, shutdown_token: CancellationToken, mut watcher: impl Watcher> + Send + Sync, - sender: TransportsSender, ) { // Setup network monitoring let mut current_netmon_state = self.netmon_watcher.get(); @@ -1427,11 +1280,6 @@ impl Actor { self.msock.metrics.magicsock.actor_link_change.inc(); self.handle_network_change(is_major).await; } - Some((dst, dst_key, msg)) = self.disco_receiver.recv() => { - if let Err(err) = self.msock.send_disco_message(&sender, dst.clone(), dst_key, msg).await { - warn!(%dst, endpoint = %dst_key.fmt_short(), ?err, "failed to send disco message (UDP)"); - } - } } } } @@ -1556,7 +1404,6 @@ impl Actor { }) .collect(), ); - self.send_queued_call_me_maybes(); } #[cfg(not(wasm_browser))] @@ -1623,22 +1470,6 @@ impl Actor { } } - fn send_queued_call_me_maybes(&mut self) { - let msg = self.msock.direct_addrs.to_call_me_maybe_message(); - let msg = disco::Message::CallMeMaybe(msg); - // allocate, to minimize locking duration - - for (public_key, url) in self.pending_call_me_maybes.drain() { - if !self - .msock - .disco - .try_send(SendAddr::Relay(url), public_key, msg.clone()) - { - warn!(endpoint = %public_key.fmt_short(), "relay channel full, dropping call-me-maybe"); - } - } - } - fn handle_net_report_report(&mut self, mut report: Option) { if let Some(ref mut r) = report { self.msock.ipv6_reported.store(r.udp_v6, Ordering::Relaxed); @@ -1733,25 +1564,6 @@ impl DiscoveredDirectAddrs { fn sockaddrs(&self) -> impl Iterator { self.addrs.get().into_iter().map(|da| da.addr) } - - fn to_call_me_maybe_message(&self) -> disco::CallMeMaybe { - let my_numbers = self.addrs.get().into_iter().map(|da| da.addr).collect(); - disco::CallMeMaybe { my_numbers } - } -} - -fn disco_message_sent(msg: &disco::Message, metrics: &MagicsockMetrics) { - match msg { - disco::Message::Ping(_) => { - metrics.sent_disco_ping.inc(); - } - disco::Message::Pong(_) => { - metrics.sent_disco_pong.inc(); - } - disco::Message::CallMeMaybe(_) => { - metrics.sent_disco_call_me_maybe.inc(); - } - } } /// A *direct address* on which an iroh-endpoint might be contactable. From ff13fa0170f0f93888d4d26bbaf704f8283d8cd7 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 17:56:03 +0100 Subject: [PATCH 06/29] delete disco and key modules, all unused now --- iroh/src/disco.rs | 622 ---------------------------------------------- iroh/src/key.rs | 163 ------------ iroh/src/lib.rs | 2 - 3 files changed, 787 deletions(-) delete mode 100644 iroh/src/disco.rs delete mode 100644 iroh/src/key.rs diff --git a/iroh/src/disco.rs b/iroh/src/disco.rs deleted file mode 100644 index c25d8fe0b3..0000000000 --- a/iroh/src/disco.rs +++ /dev/null @@ -1,622 +0,0 @@ -//! Contains the discovery message types. -//! -//! A discovery message is: -//! -//! Header: -//! -//! ```ignore -//! magic: [u8; 6] // “TS💬” (0x54 53 f0 9f 92 ac) -//! sender_disco_pub: [u8; 32] // nacl public key -//! nonce: [u8; 24] -//! ```` -//! The recipient then decrypts the bytes following (the nacl secretbox) -//! and then the inner payload structure is: -//! -//! ```ignore -//! message_type: u8 // (the MessageType constants below) -//! message_version: u8 // (0 for now; but always ignore bytes at the end) -//! message_payload: &[u8] -//! ``` - -use std::{ - fmt::{self, Display}, - net::{IpAddr, SocketAddr}, -}; - -use data_encoding::HEXLOWER; -use iroh_base::{EndpointId, PublicKey, RelayUrl}; -use nested_enum_utils::common_fields; -use rand::Rng; -use serde::{Deserialize, Serialize}; -use snafu::{Snafu, ensure}; -use url::Url; - -use crate::magicsock::transports; - -// TODO: custom magicn -/// The 6 byte header of all discovery messages. -pub const MAGIC: &str = "TS💬"; // 6 bytes: 0x54 53 f0 9f 92 ac -pub const MAGIC_LEN: usize = MAGIC.len(); - -/// Current Version. -const V0: u8 = 0; - -pub(crate) const KEY_LEN: usize = 32; -const TX_LEN: usize = 12; - -// Sizes for the inner message structure. - -/// Header: Type | Version -const HEADER_LEN: usize = 2; - -const PING_LEN: usize = TX_LEN + iroh_base::PublicKey::LENGTH; -const EP_LENGTH: usize = 16 + 2; // 16 byte IP address + 2 byte port - -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] -#[repr(u8)] -pub enum MessageType { - Ping = 0x01, - Pong = 0x02, - CallMeMaybe = 0x03, -} - -impl TryFrom for MessageType { - type Error = u8; - - fn try_from(value: u8) -> std::result::Result { - match value { - 0x01 => Ok(MessageType::Ping), - 0x02 => Ok(MessageType::Pong), - 0x03 => Ok(MessageType::CallMeMaybe), - _ => Err(value), - } - } -} - -const MESSAGE_HEADER_LEN: usize = MAGIC_LEN + KEY_LEN; - -pub fn encode_message(sender: &PublicKey, seal: Vec) -> Vec { - let mut out = Vec::with_capacity(MESSAGE_HEADER_LEN); - out.extend_from_slice(MAGIC.as_bytes()); - out.extend_from_slice(sender.as_bytes()); - out.extend(seal); - - out -} - -/// Reports whether p looks like it's a packet containing an encrypted disco message. -pub fn looks_like_disco_wrapper(p: &[u8]) -> bool { - if p.len() < MESSAGE_HEADER_LEN { - return false; - } - - &p[..MAGIC_LEN] == MAGIC.as_bytes() -} - -/// If `p` looks like a disco message it returns the slice of `p` that represents the disco public key source, -/// and the part that is the box. -pub fn source_and_box(p: &[u8]) -> Option<(PublicKey, &[u8])> { - if !looks_like_disco_wrapper(p) { - return None; - } - - let source = &p[MAGIC_LEN..MAGIC_LEN + KEY_LEN]; - let sender = PublicKey::try_from(source).ok()?; - let sealed_box = &p[MAGIC_LEN + KEY_LEN..]; - Some((sender, sealed_box)) -} - -/// A discovery message. -#[derive(Debug, Clone, PartialEq, Eq)] -pub enum Message { - Ping(Ping), - Pong(Pong), - CallMeMaybe(CallMeMaybe), -} - -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct Ping { - /// Random client-generated per-ping transaction ID. - pub tx_id: TransactionId, - - /// Allegedly the ping sender's public key. - /// - /// It shouldn't be trusted by itself. - pub endpoint_key: PublicKey, -} - -impl Ping { - /// Creates a ping message to ping `node_id`. - /// - /// Uses a randomly generated STUN transaction ID. - pub(crate) fn new(endpoint_id: EndpointId) -> Self { - Self { - tx_id: TransactionId::default(), - endpoint_key: endpoint_id, - } - } - - fn from_bytes(p: &[u8]) -> Result { - // Deliberately lax on longer-than-expected messages, for future compatibility. - ensure!(p.len() >= PING_LEN, TooShortSnafu); - let tx_id: [u8; TX_LEN] = p[..TX_LEN].try_into().expect("length checked"); - let raw_key = &p[TX_LEN..TX_LEN + iroh_base::PublicKey::LENGTH]; - let endpoint_key = - PublicKey::try_from(raw_key).map_err(|_| InvalidEncodingSnafu.build())?; - let tx_id = TransactionId::from(tx_id); - - Ok(Ping { - tx_id, - endpoint_key, - }) - } - - fn as_bytes(&self) -> Vec { - let header = msg_header(MessageType::Ping, V0); - let mut out = vec![0u8; PING_LEN + HEADER_LEN]; - - out[..HEADER_LEN].copy_from_slice(&header); - out[HEADER_LEN..HEADER_LEN + TX_LEN].copy_from_slice(&self.tx_id); - out[HEADER_LEN + TX_LEN..].copy_from_slice(self.endpoint_key.as_ref()); - - out - } -} - -/// A response a Ping. -/// -/// It includes the sender's source IP + port, so it's effectively a STUN response. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct Pong { - pub tx_id: TransactionId, - /// The observed address off the ping sender. - /// - /// 18 bytes (16+2) on the wire; v4-mapped ipv6 for IPv4. - pub ping_observed_addr: SendAddr, -} - -/// Addresses to which we can send. This is either a UDP or a relay address. -#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] -pub enum SendAddr { - /// UDP, the ip addr. - Udp(SocketAddr), - /// Relay Url. - Relay(RelayUrl), -} - -impl From for SendAddr { - fn from(addr: transports::Addr) -> Self { - match addr { - transports::Addr::Ip(addr) => SendAddr::Udp(addr), - transports::Addr::Relay(url, _) => SendAddr::Relay(url), - } - } -} - -impl From for SendAddr { - fn from(source: SocketAddr) -> Self { - SendAddr::Udp(source) - } -} - -impl From for SendAddr { - fn from(source: RelayUrl) -> Self { - SendAddr::Relay(source) - } -} - -impl PartialEq for SendAddr { - fn eq(&self, other: &SocketAddr) -> bool { - match self { - Self::Relay(_) => false, - Self::Udp(addr) => addr.eq(other), - } - } -} - -impl Display for SendAddr { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - SendAddr::Relay(id) => write!(f, "Relay({id})"), - SendAddr::Udp(addr) => write!(f, "UDP({addr})"), - } - } -} - -/// Message sent only over the relay to request that the recipient try -/// to open up a magicsock path back to the sender. -/// -/// The sender should've already sent UDP packets to the peer to open -/// up the stateful firewall mappings inbound. -/// -/// The recipient may choose to not open a path back, if it's already happy with its path. -/// But usually it will. -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct CallMeMaybe { - /// What the peer believes its endpoints are. - pub my_numbers: Vec, -} - -#[allow(missing_docs)] -#[common_fields({ - backtrace: Option, - #[snafu(implicit)] - span_trace: n0_snafu::SpanTrace, -})] -#[derive(Debug, Snafu)] -#[non_exhaustive] -pub enum ParseError { - #[snafu(display("message is too short"))] - TooShort {}, - #[snafu(display("invalid encoding"))] - InvalidEncoding {}, - #[snafu(display("unknown format"))] - UnknownFormat {}, -} - -fn send_addr_from_bytes(p: &[u8]) -> Result { - ensure!(p.len() > 2, TooShortSnafu); - match p[0] { - 0u8 => { - let bytes: [u8; EP_LENGTH] = p[1..].try_into().map_err(|_| TooShortSnafu.build())?; - let addr = socket_addr_from_bytes(bytes); - Ok(SendAddr::Udp(addr)) - } - 1u8 => { - let s = std::str::from_utf8(&p[1..]).map_err(|_| InvalidEncodingSnafu.build())?; - let u: Url = s.parse().map_err(|_| InvalidEncodingSnafu.build())?; - Ok(SendAddr::Relay(u.into())) - } - _ => Err(UnknownFormatSnafu.build()), - } -} - -fn send_addr_to_vec(addr: &SendAddr) -> Vec { - match addr { - SendAddr::Relay(url) => { - let mut out = vec![1u8]; - out.extend_from_slice(url.to_string().as_bytes()); - out - } - SendAddr::Udp(ip) => { - let mut out = vec![0u8]; - out.extend_from_slice(&socket_addr_as_bytes(ip)); - out - } - } -} - -// Assumes p.len() == EP_LENGTH -fn socket_addr_from_bytes(p: [u8; EP_LENGTH]) -> SocketAddr { - debug_assert_eq!(EP_LENGTH, 16 + 2); - - let raw_src_ip: [u8; 16] = p[..16].try_into().expect("array long enough"); - let raw_port: [u8; 2] = p[16..].try_into().expect("array long enough"); - - let src_ip = IpAddr::from(raw_src_ip).to_canonical(); - let src_port = u16::from_le_bytes(raw_port); - - SocketAddr::new(src_ip, src_port) -} - -fn socket_addr_as_bytes(addr: &SocketAddr) -> [u8; EP_LENGTH] { - let mut out = [0u8; EP_LENGTH]; - let ipv6 = match addr.ip() { - IpAddr::V4(v4) => v4.to_ipv6_mapped(), - IpAddr::V6(v6) => v6, - }; - out[..16].copy_from_slice(&ipv6.octets()); - out[16..].copy_from_slice(&addr.port().to_le_bytes()); - - out -} - -impl Pong { - fn from_bytes(p: &[u8]) -> Result { - let tx_id: [u8; TX_LEN] = p[..TX_LEN].try_into().map_err(|_| TooShortSnafu.build())?; - - let tx_id = TransactionId::from(tx_id); - let src = send_addr_from_bytes(&p[TX_LEN..])?; - - Ok(Pong { - tx_id, - ping_observed_addr: src, - }) - } - - fn as_bytes(&self) -> Vec { - let header = msg_header(MessageType::Pong, V0); - let mut out = header.to_vec(); - out.extend_from_slice(&self.tx_id); - - let src_bytes = send_addr_to_vec(&self.ping_observed_addr); - out.extend(src_bytes); - out - } -} - -impl CallMeMaybe { - fn from_bytes(p: &[u8]) -> Result { - ensure!(p.len() % EP_LENGTH == 0, InvalidEncodingSnafu); - - let num_entries = p.len() / EP_LENGTH; - let mut m = CallMeMaybe { - my_numbers: Vec::with_capacity(num_entries), - }; - - for chunk in p.chunks_exact(EP_LENGTH) { - let bytes: [u8; EP_LENGTH] = - chunk.try_into().map_err(|_| InvalidEncodingSnafu.build())?; - let src = socket_addr_from_bytes(bytes); - m.my_numbers.push(src); - } - - Ok(m) - } - - fn as_bytes(&self) -> Vec { - let header = msg_header(MessageType::CallMeMaybe, V0); - let mut out = vec![0u8; HEADER_LEN + self.my_numbers.len() * EP_LENGTH]; - out[..HEADER_LEN].copy_from_slice(&header); - - for (m, chunk) in self - .my_numbers - .iter() - .zip(out[HEADER_LEN..].chunks_exact_mut(EP_LENGTH)) - { - let raw = socket_addr_as_bytes(m); - chunk.copy_from_slice(&raw); - } - - out - } -} - -impl Message { - /// Parses the encrypted part of the message from inside the nacl secretbox. - pub fn from_bytes(p: &[u8]) -> Result { - ensure!(p.len() >= 2, TooShortSnafu); - - let t = MessageType::try_from(p[0]).map_err(|_| UnknownFormatSnafu.build())?; - let version = p[1]; - ensure!(version == V0, UnknownFormatSnafu); - - let p = &p[2..]; - match t { - MessageType::Ping => { - let ping = Ping::from_bytes(p)?; - Ok(Message::Ping(ping)) - } - MessageType::Pong => { - let pong = Pong::from_bytes(p)?; - Ok(Message::Pong(pong)) - } - MessageType::CallMeMaybe => { - let cm = CallMeMaybe::from_bytes(p)?; - Ok(Message::CallMeMaybe(cm)) - } - } - } - - /// Serialize this message to bytes. - pub fn as_bytes(&self) -> Vec { - match self { - Message::Ping(ping) => ping.as_bytes(), - Message::Pong(pong) => pong.as_bytes(), - Message::CallMeMaybe(cm) => cm.as_bytes(), - } - } -} - -impl Display for Message { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Message::Ping(ping) => { - write!(f, "Ping(tx={})", HEXLOWER.encode(&ping.tx_id)) - } - Message::Pong(pong) => { - write!(f, "Pong(tx={})", HEXLOWER.encode(&pong.tx_id)) - } - Message::CallMeMaybe(_) => { - write!(f, "CallMeMaybe") - } - } - } -} - -const fn msg_header(t: MessageType, ver: u8) -> [u8; HEADER_LEN] { - [t as u8, ver] -} - -const TRANSACTION_ID_SIZE: usize = 12; - -/// The transaction ID is a 96-bit identifier -/// -/// It is used to uniquely identify STUN transactions. -/// It primarily serves to correlate requests with responses, -/// though it also plays a small role in helping to prevent -/// certain types of attacks. The server also uses the transaction ID as -/// a key to identify each transaction uniquely across all clients. -#[derive(Clone, Copy, PartialEq, Eq, Hash, PartialOrd, Ord)] -pub(crate) struct TransactionId([u8; TRANSACTION_ID_SIZE]); - -impl fmt::Debug for TransactionId { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "TransactionId(0x")?; - fmt_transcation_id(self.as_ref(), f) - } -} - -impl fmt::Display for TransactionId { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "transaction id (0x")?; - fmt_transcation_id(self.as_ref(), f) - } -} - -fn fmt_transcation_id(bytes: &[u8], f: &mut fmt::Formatter) -> fmt::Result { - for byte in bytes { - write!(f, "{:02X}", byte)?; - } - write!(f, ")") -} - -impl std::ops::Deref for TransactionId { - type Target = [u8]; - - fn deref(&self) -> &[u8] { - &self.0 - } -} - -impl AsRef<[u8]> for TransactionId { - fn as_ref(&self) -> &[u8] { - &self.0[..] - } -} - -impl From<&[u8; TRANSACTION_ID_SIZE]> for TransactionId { - fn from(buff: &[u8; TRANSACTION_ID_SIZE]) -> Self { - Self(*buff) - } -} - -impl From<[u8; TRANSACTION_ID_SIZE]> for TransactionId { - fn from(buff: [u8; TRANSACTION_ID_SIZE]) -> Self { - Self(buff) - } -} - -impl rand::distr::Distribution for rand::distr::StandardUniform { - fn sample(&self, rng: &mut R) -> TransactionId { - let mut buffer = [0u8; TRANSACTION_ID_SIZE]; - rng.fill_bytes(&mut buffer); - TransactionId::from(buffer) - } -} - -impl Default for TransactionId { - /// Creates a cryptographically random transaction ID chosen from the interval 0 .. 2**96-1. - fn default() -> Self { - let mut rng = rand::rng(); - rng.random() - } -} - -#[cfg(test)] -mod tests { - use iroh_base::SecretKey; - use rand::SeedableRng; - - use super::*; - use crate::key::{SharedSecret, public_ed_box, secret_ed_box}; - - #[test] - fn test_to_from_bytes() { - struct Test { - name: &'static str, - m: Message, - want: &'static str, - } - let tests = [ - Test { - name: "ping_with_endpointkey_src", - m: Message::Ping(Ping { - tx_id: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12].into(), - endpoint_key: PublicKey::try_from( - &[ - 190, 243, 65, 104, 37, 102, 175, 75, 243, 22, 69, 200, 167, 107, 24, - 63, 216, 140, 120, 43, 4, 112, 16, 62, 117, 155, 45, 215, 72, 175, 40, - 189, - ][..], - ) - .unwrap(), - }), - want: "01 00 01 02 03 04 05 06 07 08 09 0a 0b 0c be f3 41 68 25 66 af 4b f3 16 45 c8 a7 6b 18 3f d8 8c 78 2b 04 70 10 3e 75 9b 2d d7 48 af 28 bd", - }, - Test { - name: "pong", - m: Message::Pong(Pong { - tx_id: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12].into(), - ping_observed_addr: SendAddr::Udp("2.3.4.5:1234".parse().unwrap()), - }), - want: "02 00 01 02 03 04 05 06 07 08 09 0a 0b 0c 00 00 00 00 00 00 00 00 00 00 00 ff ff 02 03 04 05 d2 04", - }, - Test { - name: "pongv6", - m: Message::Pong(Pong { - tx_id: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12].into(), - ping_observed_addr: SendAddr::Udp("[fed0::12]:6666".parse().unwrap()), - }), - want: "02 00 01 02 03 04 05 06 07 08 09 0a 0b 0c 00 fe d0 00 00 00 00 00 00 00 00 00 00 00 00 00 12 0a 1a", - }, - Test { - name: "call_me_maybe", - m: Message::CallMeMaybe(CallMeMaybe { - my_numbers: Vec::new(), - }), - want: "03 00", - }, - Test { - name: "call_me_maybe_endpoints", - m: Message::CallMeMaybe(CallMeMaybe { - my_numbers: vec![ - "1.2.3.4:567".parse().unwrap(), - "[2001::3456]:789".parse().unwrap(), - ], - }), - want: "03 00 00 00 00 00 00 00 00 00 00 00 ff ff 01 02 03 04 37 02 20 01 00 00 00 00 00 00 00 00 00 00 00 00 34 56 15 03", - }, - ]; - for test in tests { - println!("{}", test.name); - - let got = test.m.as_bytes(); - assert_eq!( - got, - data_encoding::HEXLOWER - .decode(test.want.replace(' ', "").as_bytes()) - .unwrap(), - "wrong as_bytes" - ); - - let back = Message::from_bytes(&got).expect("failed to parse"); - assert_eq!(test.m, back, "wrong from_bytes"); - } - } - - #[test] - fn test_extraction() { - let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64); - let sender_key = SecretKey::generate(&mut rng); - let recv_key = SecretKey::generate(&mut rng); - - let msg = Message::Ping(Ping { - tx_id: TransactionId::default(), - endpoint_key: sender_key.public(), - }); - - let sender_secret = secret_ed_box(&sender_key); - let shared = SharedSecret::new(&sender_secret, &public_ed_box(&recv_key.public())); - let mut seal = msg.as_bytes(); - shared.seal(&mut seal); - - let bytes = encode_message(&sender_key.public(), seal.clone()); - - assert!(looks_like_disco_wrapper(&bytes)); - assert_eq!(source_and_box(&bytes).unwrap().0, sender_key.public()); - - let (raw_key, seal_back) = source_and_box(&bytes).unwrap(); - assert_eq!(raw_key, sender_key.public()); - assert_eq!(seal_back, seal); - - let recv_secret = secret_ed_box(&recv_key); - let shared_recv = SharedSecret::new(&recv_secret, &public_ed_box(&sender_key.public())); - let mut open_seal = seal_back.to_vec(); - shared_recv - .open(&mut open_seal) - .expect("failed to open seal_back"); - let msg_back = Message::from_bytes(&open_seal).unwrap(); - assert_eq!(msg_back, msg); - } -} diff --git a/iroh/src/key.rs b/iroh/src/key.rs deleted file mode 100644 index 64f8d8bb05..0000000000 --- a/iroh/src/key.rs +++ /dev/null @@ -1,163 +0,0 @@ -//! The private and public keys of an endpoint. - -use std::fmt::Debug; - -use aead::{AeadCore, AeadInOut, Buffer}; -use iroh_base::{PublicKey, SecretKey}; -use nested_enum_utils::common_fields; -use snafu::{ResultExt, Snafu, ensure}; - -pub(crate) const NONCE_LEN: usize = 24; - -const AEAD_DATA: &[u8] = &[]; - -pub(super) fn public_ed_box(key: &PublicKey) -> crypto_box::PublicKey { - let key = key.as_verifying_key(); - crypto_box::PublicKey::from(key.to_montgomery()) -} - -pub(super) fn secret_ed_box(key: &SecretKey) -> crypto_box::SecretKey { - let key = key.as_signing_key(); - crypto_box::SecretKey::from(key.to_scalar()) -} - -/// Shared Secret. -pub struct SharedSecret(crypto_box::ChaChaBox); - -/// Errors that can occur during [`SharedSecret::open`]. -#[common_fields({ - backtrace: Option, - #[snafu(implicit)] - span_trace: n0_snafu::SpanTrace, -})] -#[derive(Debug, Snafu)] -#[non_exhaustive] -pub enum DecryptionError { - /// The nonce had the wrong size. - #[snafu(display("Invalid nonce"))] - InvalidNonce {}, - /// AEAD decryption failed. - #[snafu(display("Aead error"))] - Aead { source: aead::Error }, -} - -impl Debug for SharedSecret { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "SharedSecret(crypto_box::ChaChaBox)") - } -} - -impl SharedSecret { - pub fn new(this: &crypto_box::SecretKey, other: &crypto_box::PublicKey) -> Self { - SharedSecret(crypto_box::ChaChaBox::new_from_clamped(other, this)) - } - - /// Seals the provided cleartext. - pub fn seal(&self, buffer: &mut dyn Buffer) { - let nonce = crypto_box::ChaChaBox::try_generate_nonce_with_rng(&mut rand::rng()) - .expect("not enough randomness"); - - self.0 - .encrypt_in_place(&nonce, AEAD_DATA, buffer) - .expect("encryption failed"); - - buffer.extend_from_slice(&nonce).expect("buffer too small"); - } - - /// Opens the ciphertext, which must have been created using `Self::seal`, and places the clear text into the provided buffer. - pub fn open(&self, buffer: &mut dyn Buffer) -> Result<(), DecryptionError> { - ensure!(buffer.len() >= NONCE_LEN, InvalidNonceSnafu); - - let offset = buffer.len() - NONCE_LEN; - let nonce: [u8; NONCE_LEN] = buffer.as_ref()[offset..] - .try_into() - .map_err(|_| InvalidNonceSnafu.build())?; - - buffer.truncate(offset); - self.0 - .decrypt_in_place(&nonce.into(), AEAD_DATA, buffer) - .context(AeadSnafu)?; - - Ok(()) - } -} - -#[cfg(test)] -mod tests { - use rand::SeedableRng; - - use super::*; - - fn shared(this: &iroh_base::SecretKey, other: &iroh_base::PublicKey) -> SharedSecret { - let secret_key = secret_ed_box(this); - let public_key = public_ed_box(other); - - SharedSecret::new(&secret_key, &public_key) - } - - #[test] - fn test_seal_open_roundtrip() { - let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64); - let key_a = iroh_base::SecretKey::generate(&mut rng); - let key_b = iroh_base::SecretKey::generate(&mut rng); - - println!("a -> a"); - seal_open_roundtrip(&key_a, &key_a); - println!("b -> b"); - seal_open_roundtrip(&key_b, &key_b); - - println!("a -> b"); - seal_open_roundtrip(&key_a, &key_b); - println!("b -> a"); - seal_open_roundtrip(&key_b, &key_a); - } - - fn seal_open_roundtrip(key_a: &iroh_base::SecretKey, key_b: &iroh_base::SecretKey) { - let msg = b"super secret message!!!!".to_vec(); - let shared_a = shared(key_a, &key_b.public()); - let mut sealed_message = msg.clone(); - shared_a.seal(&mut sealed_message); - - let shared_b = shared(key_b, &key_a.public()); - let mut decrypted_message = sealed_message.clone(); - shared_b.open(&mut decrypted_message).unwrap(); - assert_eq!(&msg[..], &decrypted_message); - } - - #[test] - fn test_roundtrip_public_key() { - let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64); - let key = crypto_box::SecretKey::generate(&mut rng); - let public_bytes = *key.public_key().as_bytes(); - let public_key_back = crypto_box::PublicKey::from(public_bytes); - assert_eq!(key.public_key(), public_key_back); - } - - #[test] - fn test_same_public_key_api() { - let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64); - let key = iroh_base::SecretKey::generate(&mut rng); - let public_key1: crypto_box::PublicKey = public_ed_box(&key.public()); - let public_key2: crypto_box::PublicKey = secret_ed_box(&key).public_key(); - - assert_eq!(public_key1, public_key2); - } - - #[test] - fn test_same_public_key_low_level() { - let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64); - let key = ed25519_dalek::SigningKey::generate(&mut rng); - let public_key1 = { - let m = key.verifying_key().to_montgomery(); - crypto_box::PublicKey::from(m) - }; - - let public_key2 = { - let s = key.to_scalar(); - let cs = crypto_box::SecretKey::from(s); - cs.public_key() - }; - - assert_eq!(public_key1, public_key2); - } -} diff --git a/iroh/src/lib.rs b/iroh/src/lib.rs index eb109682d1..62c16b069c 100644 --- a/iroh/src/lib.rs +++ b/iroh/src/lib.rs @@ -252,8 +252,6 @@ #![cfg_attr(not(test), deny(clippy::unwrap_used))] #![cfg_attr(iroh_docsrs, feature(doc_cfg))] -mod disco; -mod key; mod magicsock; mod tls; From 77f63b92917c3992bd1818d078ea244a969712e2 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 18:00:50 +0100 Subject: [PATCH 07/29] delete disco metrics --- iroh/src/magicsock/metrics.rs | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/iroh/src/magicsock/metrics.rs b/iroh/src/magicsock/metrics.rs index c4ee2a331c..3cd6eab223 100644 --- a/iroh/src/magicsock/metrics.rs +++ b/iroh/src/magicsock/metrics.rs @@ -27,24 +27,6 @@ pub struct Metrics { /// Number of datagrams received using GRO pub recv_gro_datagrams: Counter, - // Disco packets - pub send_disco_udp: Counter, - pub send_disco_relay: Counter, - pub sent_disco_udp: Counter, - pub sent_disco_relay: Counter, - pub sent_disco_ping: Counter, - pub sent_disco_pong: Counter, - pub sent_disco_call_me_maybe: Counter, - pub recv_disco_bad_key: Counter, - pub recv_disco_bad_parse: Counter, - - pub recv_disco_udp: Counter, - pub recv_disco_relay: Counter, - pub recv_disco_ping: Counter, - pub recv_disco_pong: Counter, - pub recv_disco_call_me_maybe: Counter, - pub recv_disco_call_me_maybe_bad_disco: Counter, - // How many times our relay home endpoint DI has changed from non-zero to a different non-zero. pub relay_home_change: Counter, From 5c4851a1f097e00a12584134cd711af2121ffb82 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 31 Oct 2025 18:04:46 +0100 Subject: [PATCH 08/29] emit a single metric, so metrics are used I don't want to remove the metrics plumbing, but it is currently unused. So emit a single metric for now. Need to do a proper metrics review later, we need more metrics than this. --- iroh/src/magicsock/endpoint_map/endpoint_state.rs | 1 + iroh/src/magicsock/metrics.rs | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index 2c0ea64bac..e6fde131ec 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -521,6 +521,7 @@ impl EndpointStateActor { /// Unconditionally perform holepunching. #[instrument(skip_all)] async fn do_holepunching(&mut self, conn: quinn::Connection) { + self.metrics.nat_traversal.inc(); let local_candidates = self .local_addrs .get() diff --git a/iroh/src/magicsock/metrics.rs b/iroh/src/magicsock/metrics.rs index 3cd6eab223..8047356fc6 100644 --- a/iroh/src/magicsock/metrics.rs +++ b/iroh/src/magicsock/metrics.rs @@ -30,6 +30,12 @@ pub struct Metrics { // How many times our relay home endpoint DI has changed from non-zero to a different non-zero. pub relay_home_change: Counter, + /* + * Holepunching metrics + */ + /// The number of NAT traversal attempts initiated. + pub nat_traversal: Counter, + /* * Connection Metrics */ From a8d97485095a573129b4555a840558f15180bfdb Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Mon, 3 Nov 2025 11:22:21 +0100 Subject: [PATCH 09/29] Don't store remote NAT candidates Turns out we don't do anything with those. We only use this to decide to check if we need to holepunch again. This calls trigger_holepunching for every connection event which will wake it up too much, because we only really need to do it for the connection we are using to holepunch. But the state tracking is easier like this and the cost probably not too high. --- .../magicsock/endpoint_map/endpoint_state.rs | 22 ++----------------- 1 file changed, 2 insertions(+), 20 deletions(-) diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index e6fde131ec..530dbe78ac 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -238,9 +238,8 @@ impl EndpointStateActor { Some((id, evt)) = self.path_events.next() => { self.handle_path_event(id, evt); } - Some((id, evt)) = self.addr_events.next() => { - self.handle_addr_event(id, evt); - trace!("remote addrs updated, triggering holepunching"); + Some((id, _)) = self.addr_events.next() => { + trace!(?id, "remote addrs updated, triggering holepunching"); self.trigger_holepunching().await; } _ = self.local_addrs.updated() => { @@ -349,7 +348,6 @@ impl EndpointStateActor { ConnectionState { handle: handle.clone(), pub_path_info: paths_info, - nat_candidates: Default::default(), paths: Default::default(), open_paths: Default::default(), path_ids: Default::default(), @@ -700,18 +698,6 @@ impl EndpointStateActor { } } - /// Handles a NAT address candidate added or removed by the server. - /// - /// When the server adds an address we want to (re-)holepunch. Same for when a local - /// address changes, though that is not handled here. - fn handle_addr_event(&mut self, conn_id: ConnId, event: Vec) { - if let Some(conn_state) = self.connections.get_mut(&conn_id) { - let _ = std::mem::replace(&mut conn_state.nat_candidates, event); - } else { - warn!(?conn_id, "NAT candidate event for unknown connection"); - } - } - /// Clean up connections which no longer exist. // TODO: Call this on a schedule. fn cleanup_connections(&mut self) { @@ -924,10 +910,6 @@ struct ConnectionState { handle: WeakConnectionHandle, /// The information we publish to users about the paths used in this connection. pub_path_info: Watchable>, - /// NAT candidate addresses we received from the server. - /// - /// If we are the server for this connection, this will remain empty. - nat_candidates: Vec, /// The paths that exist on this connection. /// /// This could be in any state, e.g. while still validating the path or already closed From 68e673b6d6939cbb3979b3bd690e0d4c64b09878 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Mon, 3 Nov 2025 12:37:15 +0100 Subject: [PATCH 10/29] Clean up connections at an interval --- iroh/src/magicsock/endpoint_map/endpoint_state.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index 530dbe78ac..e125fb521e 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -41,6 +41,9 @@ use crate::{ /// attempted more frequently than at this interval. const HOLEPUNCH_ATTEMPTS_INTERVAL: Duration = Duration::from_secs(5); +/// Interval at which [`ConnectionState`]s for closed connections are cleaned up. +const CONN_CLEANUP_INTERVAL: Duration = Duration::from_secs(5); + // TODO: use this // /// Number of addresses that are not active that we keep around per endpoint. // /// @@ -216,6 +219,7 @@ impl EndpointStateActor { mut inbox: mpsc::Receiver, ) -> Result<(), Whatever> { trace!("actor started"); + let mut conn_cleanup = std::pin::pin!(time::interval(CONN_CLEANUP_INTERVAL)); loop { let scheduled_path_open = match self.scheduled_open_path { Some(when) => MaybeFuture::Some(time::sleep_until(when)), @@ -259,6 +263,9 @@ impl EndpointStateActor { self.scheduled_holepunch = None; self.trigger_holepunching().await; } + _ = conn_cleanup.tick() => { + self.cleanup_connections(); + } } } trace!("actor terminating"); @@ -699,7 +706,6 @@ impl EndpointStateActor { } /// Clean up connections which no longer exist. - // TODO: Call this on a schedule. fn cleanup_connections(&mut self) { self.connections.retain(|_, c| c.handle.upgrade().is_some()); } From ce227f5186fcdb7320888969a3451ecc13a69d30 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Tue, 18 Nov 2025 14:18:47 +0100 Subject: [PATCH 11/29] compile against protocol-simplification branch it's something, if not much --- .../magicsock/endpoint_map/endpoint_state.rs | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index e4321a1b31..1e75eb1c5f 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -14,7 +14,7 @@ use n0_future::{ }; use n0_watcher::{Watchable, Watcher}; use quinn::{PathStats, WeakConnectionHandle}; -use quinn_proto::{PathError, PathEvent, PathId, PathStatus}; +use quinn_proto::{PathError, PathEvent, PathId, PathStatus, iroh_hp}; use rustc_hash::FxHashMap; use smallvec::SmallVec; use tokio::sync::oneshot; @@ -93,8 +93,15 @@ type PathEvents = MergeUnbounded< /// A stream of events of announced NAT traversal candidate addresses for all connections. /// /// The connection is identified using [`ConnId`]. -type AddrEvents = - MergeUnbounded)> + Send + Sync>>>; +type AddrEvents = MergeUnbounded< + Pin< + Box< + dyn Stream)> + + Send + + Sync, + >, + >, +>; /// List of addrs and path ids for open paths in a connection. pub(crate) type PathAddrList = SmallVec<[(TransportAddr, PathId); 4]>; @@ -382,8 +389,9 @@ impl EndpointStateActor { self.path_events.push(Box::pin( BroadcastStream::new(conn.path_events()).map(move |evt| (conn_id, evt)), )); - self.addr_events - .push(Box::pin(conn.addr_events().map(move |evt| (conn_id, evt)))); + self.addr_events.push(Box::pin( + BroadcastStream::new(conn.nat_traversal_updates()).map(move |evt| (conn_id, evt)), + )); self.connections_close.push(OnClosed::new(&conn)); // Store the connection @@ -522,7 +530,15 @@ impl EndpointStateActor { trace!("not holepunching: no client connection"); return; }; - let remote_candidates = BTreeSet::from_iter(conn.nat_candidates()); + // TODO: these are the local addresses, so this will be very sad. + let Ok(remote_candidates) = conn + .get_nat_traversal_addresses() + .and_then(|addrs| Ok(BTreeSet::from_iter(addrs))) + else { + warn!("boo"); + return; + }; + // let remote_candidates = BTreeSet::from_iter(conn.get_nat_traversal_addresses()); let local_candidates: BTreeSet = self .local_addrs .get() @@ -570,7 +586,7 @@ impl EndpointStateActor { .iter() .map(|daddr| daddr.addr) .collect::>(); - match conn.initiate_nat_traversal(Vec::from_iter(local_candidates.iter().copied())) { + match conn.initiate_nat_traversal_round() { Ok(remote_candidates) => { trace!( ?local_candidates, From b059baba745356bbe48d59d55b4e272e42bdad76 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Thu, 20 Nov 2025 10:33:13 +0100 Subject: [PATCH 12/29] updage api --- iroh/src/magicsock/endpoint_map/endpoint_state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh/src/magicsock/endpoint_map/endpoint_state.rs b/iroh/src/magicsock/endpoint_map/endpoint_state.rs index 1e75eb1c5f..93ca97bd79 100644 --- a/iroh/src/magicsock/endpoint_map/endpoint_state.rs +++ b/iroh/src/magicsock/endpoint_map/endpoint_state.rs @@ -532,7 +532,7 @@ impl EndpointStateActor { }; // TODO: these are the local addresses, so this will be very sad. let Ok(remote_candidates) = conn - .get_nat_traversal_addresses() + .get_remote_nat_traversal_addresses() .and_then(|addrs| Ok(BTreeSet::from_iter(addrs))) else { warn!("boo"); From a93db87f6713ad01f1d07fc4d2ed390f8dbab9a3 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Thu, 20 Nov 2025 15:53:26 +0100 Subject: [PATCH 13/29] some intermediate stuff --- iroh/src/endpoint.rs | 4 +++- iroh/src/magicsock.rs | 2 +- iroh/src/magicsock/remote_map/remote_state.rs | 18 ++++++++---------- .../remote_map/remote_state/path_state.rs | 5 ----- 4 files changed, 12 insertions(+), 17 deletions(-) diff --git a/iroh/src/endpoint.rs b/iroh/src/endpoint.rs index b608091411..0bff432fc1 100644 --- a/iroh/src/endpoint.rs +++ b/iroh/src/endpoint.rs @@ -174,7 +174,9 @@ impl Builder { self.transport_config .default_path_max_idle_timeout(Some(PATH_MAX_IDLE_TIMEOUT)); self.transport_config - .max_concurrent_multipath_paths(MAX_MULTIPATH_PATHS); + .max_concurrent_multipath_paths(MAX_MULTIPATH_PATHS + 1); + self.transport_config + .set_max_remote_nat_traversal_addresses(MAX_MULTIPATH_PATHS as u8); let static_config = StaticConfig { transport_config: Arc::new(self.transport_config), diff --git a/iroh/src/magicsock.rs b/iroh/src/magicsock.rs index 8546bdfb1f..ad5c5c0c8a 100644 --- a/iroh/src/magicsock.rs +++ b/iroh/src/magicsock.rs @@ -94,7 +94,7 @@ pub(crate) const PATH_MAX_IDLE_TIMEOUT: Duration = Duration::from_millis(6500); /// Maximum number of concurrent QUIC multipath paths per connection. /// /// Pretty arbitrary and high right now. -pub(crate) const MAX_MULTIPATH_PATHS: u32 = 16; +pub(crate) const MAX_MULTIPATH_PATHS: u32 = 12; /// Error returned when the endpoint state actor stopped while waiting for a reply. #[stack_error(add_meta, derive)] diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index 301c5d8b9c..74135d6440 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -587,13 +587,12 @@ impl RemoteStateActor { trace!("not holepunching: no client connection"); return; }; - // TODO: these are the local addresses, so this will be very sad. - let Ok(remote_candidates) = conn - .get_remote_nat_traversal_addresses() - .and_then(|addrs| Ok(BTreeSet::from_iter(addrs))) - else { - warn!("boo"); - return; + let remote_candidates = match conn.get_remote_nat_traversal_addresses() { + Ok(addrs) => BTreeSet::from_iter(addrs), + Err(err) => { + warn!("failed to get nat candidate addresses: {err:#}"); + return; + } }; // let remote_candidates = BTreeSet::from_iter(conn.get_nat_traversal_addresses()); let local_candidates: BTreeSet = self @@ -656,9 +655,8 @@ impl RemoteStateActor { remote_candidates: BTreeSet::from_iter(remote_candidates), }); } - Err(_) => { - // TODO: log error - warn!("failed to initiate NAT traversal"); + Err(err) => { + warn!("failed to initiate NAT traversal {err:#}"); } } } diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index d75c6f0f38..4b94835b9b 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -81,11 +81,6 @@ impl RemotePathState { self.emit_pending_resolve_requests(result.err()); } - /// Returns an iterator over all paths and their state. - pub(super) fn iter(&self) -> impl Iterator { - self.paths.iter() - } - /// Returns an iterator over the addresses of all paths. pub(super) fn addrs(&self) -> impl Iterator { self.paths.keys() From 1fc0714bb15328e6afe63e52278423d1a86ed6fd Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Fri, 21 Nov 2025 16:33:16 +0100 Subject: [PATCH 14/29] hook up adding addresses --- iroh/src/magicsock/remote_map/remote_state.rs | 50 ++++++++++++++++++- 1 file changed, 48 insertions(+), 2 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index 74135d6440..5907519b9c 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -298,6 +298,7 @@ impl RemoteStateActor { } } _ = self.local_addrs.updated() => { + self.local_addrs_updated(); trace!("local addrs updated, triggering holepunching"); self.trigger_holepunching().await; } @@ -429,6 +430,15 @@ impl RemoteStateActor { )); self.connections_close.push(OnClosed::new(&conn)); + // Add local addrs to the connection + let local_addrs = self + .local_addrs + .get() + .iter() + .map(|d| d.addr) + .collect::>(); + Self::set_local_addrs(&conn, &local_addrs); + // Store the connection let conn_state = self .connections @@ -559,6 +569,43 @@ impl RemoteStateActor { } } + /// Sets the current local addresses to QNT's state to all connections + fn local_addrs_updated(&mut self) { + let local_addrs = self + .local_addrs + .get() + .iter() + .map(|d| d.addr) + .collect::>(); + + for conn in self.connections.values().filter_map(|s| s.handle.upgrade()) { + Self::set_local_addrs(&conn, &local_addrs); + } + // todo: trace + } + + /// Sets the current local addresses to QNT's state + fn set_local_addrs(conn: &quinn::Connection, local_addrs: &BTreeSet) { + let quinn_local_addrs = match conn.get_local_nat_traversal_addresses() { + Ok(addrs) => BTreeSet::from_iter(addrs), + Err(err) => { + warn!("failed to get local nat candidates: {err:#}"); + return; + } + }; + for addr in local_addrs.difference(&quinn_local_addrs) { + if let Err(err) = conn.add_nat_traversal_address(*addr) { + warn!("failed adding local addr: {err:#}",); + } + } + for addr in quinn_local_addrs.difference(&local_addrs) { + if let Err(err) = conn.remove_nat_traversal_address(*addr) { + warn!("failed removing local addr: {err:#}"); + } + } + trace!(?local_addrs, "updated local QNT addresses"); + } + /// Triggers holepunching to the remote endpoint. /// /// This will manage the entire process of holepunching with the remote endpoint. @@ -594,7 +641,6 @@ impl RemoteStateActor { return; } }; - // let remote_candidates = BTreeSet::from_iter(conn.get_nat_traversal_addresses()); let local_candidates: BTreeSet = self .local_addrs .get() @@ -656,7 +702,7 @@ impl RemoteStateActor { }); } Err(err) => { - warn!("failed to initiate NAT traversal {err:#}"); + warn!("failed to initiate NAT traversal: {err:#}"); } } } From e24b41d721586fca49ef9ef2e0cd9f9044ffdca0 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Sun, 23 Nov 2025 10:58:51 +0100 Subject: [PATCH 15/29] properly patch in quinn git dependency and clippy --- Cargo.lock | 13 +++++++----- Cargo.toml | 20 +++++++++---------- iroh-relay/Cargo.toml | 4 ++-- iroh/Cargo.toml | 8 ++++---- iroh/bench/Cargo.toml | 2 +- iroh/src/magicsock/remote_map.rs | 3 +-- iroh/src/magicsock/remote_map/remote_state.rs | 2 +- 7 files changed, 27 insertions(+), 25 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9c3d7463d6..2a9f7aca06 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1914,7 +1914,7 @@ dependencies = [ "libc", "percent-encoding", "pin-project-lite", - "socket2 0.6.0", + "socket2 0.5.10", "tokio", "tower-service", "tracing", @@ -2363,6 +2363,7 @@ dependencies = [ [[package]] name = "iroh-quinn" version = "0.14.0" +source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#fe22a2bf3744353b411df26abeca1c2dcd8a4c5f" dependencies = [ "bytes", "cfg_aliases", @@ -2371,7 +2372,7 @@ dependencies = [ "pin-project-lite", "rustc-hash", "rustls", - "socket2 0.6.0", + "socket2 0.5.10", "thiserror 2.0.17", "tokio", "tokio-stream", @@ -2382,6 +2383,7 @@ dependencies = [ [[package]] name = "iroh-quinn-proto" version = "0.13.0" +source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#fe22a2bf3744353b411df26abeca1c2dcd8a4c5f" dependencies = [ "bytes", "fastbloom", @@ -2404,11 +2406,12 @@ dependencies = [ [[package]] name = "iroh-quinn-udp" version = "0.5.12" +source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#fe22a2bf3744353b411df26abeca1c2dcd8a4c5f" dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.6.0", + "socket2 0.5.10", "tracing", "windows-sys 0.60.2", ] @@ -3413,7 +3416,7 @@ dependencies = [ "quinn-udp", "rustc-hash", "rustls", - "socket2 0.6.0", + "socket2 0.5.10", "thiserror 2.0.17", "tokio", "tracing", @@ -3450,7 +3453,7 @@ dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.6.0", + "socket2 0.5.10", "tracing", "windows-sys 0.60.2", ] diff --git a/Cargo.toml b/Cargo.toml index 20f1861aef..e7859a7576 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -43,17 +43,17 @@ unused-async = "warn" [patch.crates-io] -# iroh-quinn = { git = "https://github.com/n0-computer/quinn", branch = "main-iroh" } -# iroh-quinn-proto = { git = "https://github.com/n0-computer/quinn", branch = "main-iroh" } -# iroh-quinn-udp = { git = "https://github.com/n0-computer/quinn", branch = "main-iroh" } +iroh-quinn = { git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification" } +iroh-quinn-proto = { git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification" } +iroh-quinn-udp = { git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification" } netwatch = { git = "https://github.com/n0-computer/net-tools", branch = "main" } -iroh-quinn = { path = "../quinn/quinn" } -iroh-quinn-proto = { path = "../quinn/quinn-proto" } -iroh-quinn-udp = { path = "../quinn/quinn-udp" } +# iroh-quinn = { path = "../quinn/quinn" } +# iroh-quinn-proto = { path = "../quinn/quinn-proto" } +# iroh-quinn-udp = { path = "../quinn/quinn-udp" } -[patch."https://github.com/n0-computer/quinn"] -iroh-quinn = { path = "../quinn/quinn" } -iroh-quinn-proto = { path = "../quinn/quinn-proto" } -iroh-quinn-udp = { path = "../quinn/quinn-udp" } +# [patch."https://github.com/n0-computer/quinn"] +# iroh-quinn = { path = "../quinn/quinn" } +# iroh-quinn-proto = { path = "../quinn/quinn-proto" } +# iroh-quinn-udp = { path = "../quinn/quinn-udp" } diff --git a/iroh-relay/Cargo.toml b/iroh-relay/Cargo.toml index 7a69c4a112..3b03936601 100644 --- a/iroh-relay/Cargo.toml +++ b/iroh-relay/Cargo.toml @@ -42,8 +42,8 @@ postcard = { version = "1", default-features = false, features = [ "use-std", "experimental-derive", ] } -quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "main-iroh", default-features = false, features = ["rustls-ring"] } -quinn-proto = { package = "iroh-quinn-proto", git = "https://github.com/n0-computer/quinn", branch = "main-iroh" } +quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification", default-features = false, features = ["rustls-ring"] } +quinn-proto = { package = "iroh-quinn-proto", git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification" } rand = "0.9.2" reqwest = { version = "0.12", default-features = false, features = [ "rustls-tls", diff --git a/iroh/Cargo.toml b/iroh/Cargo.toml index 7e63103341..79db7270d7 100644 --- a/iroh/Cargo.toml +++ b/iroh/Cargo.toml @@ -37,9 +37,9 @@ n0-watcher = "0.6" netwatch = { version = "0.12" } pin-project = "1" pkarr = { version = "5", default-features = false, features = ["relays"] } -quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "main-iroh", default-features = false, features = ["rustls-ring"] } -quinn-proto = { package = "iroh-quinn-proto", git = "https://github.com/n0-computer/quinn", branch = "main-iroh" } -quinn-udp = { package = "iroh-quinn-udp", git = "https://github.com/n0-computer/quinn", branch = "main-iroh" } +quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification", default-features = false, features = ["rustls-ring"] } +quinn-proto = { package = "iroh-quinn-proto", git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification" } +quinn-udp = { package = "iroh-quinn-udp", git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification" } rand = "0.9.2" reqwest = { version = "0.12", default-features = false, features = [ "rustls-tls", @@ -84,7 +84,7 @@ hickory-resolver = "0.25.1" igd-next = { version = "0.16", features = ["aio_tokio"] } netdev = { version = "0.39.0" } portmapper = { version = "0.12", default-features = false } -quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "main-iroh", default-features = false, features = ["runtime-tokio", "rustls-ring"] } +quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification", default-features = false, features = ["runtime-tokio", "rustls-ring"] } tokio = { version = "1", features = [ "io-util", "macros", diff --git a/iroh/bench/Cargo.toml b/iroh/bench/Cargo.toml index 086d1ebc31..8108a22c5e 100644 --- a/iroh/bench/Cargo.toml +++ b/iroh/bench/Cargo.toml @@ -12,7 +12,7 @@ iroh = { path = "..", default-features = false } iroh-metrics = { version = "0.37", optional = true } n0-future = "0.3.0" n0-error = "0.1.0" -quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "main-iroh" } +quinn = { package = "iroh-quinn", git = "https://github.com/n0-computer/quinn", branch = "protocol-simplification" } rand = "0.9.2" rcgen = "0.14" rustls = { version = "0.23.33", default-features = false, features = ["ring"] } diff --git a/iroh/src/magicsock/remote_map.rs b/iroh/src/magicsock/remote_map.rs index 866e6aca5f..cace484c6c 100644 --- a/iroh/src/magicsock/remote_map.rs +++ b/iroh/src/magicsock/remote_map.rs @@ -11,8 +11,6 @@ use rustc_hash::FxHashMap; use serde::{Deserialize, Serialize}; use tokio::sync::mpsc; -use crate::discovery::ConcurrentDiscovery; - pub(crate) use self::remote_state::PathsWatcher; pub(super) use self::remote_state::RemoteStateMessage; pub use self::remote_state::{PathInfo, PathInfoList}; @@ -22,6 +20,7 @@ use super::{ mapped_addrs::{AddrMap, EndpointIdMappedAddr, RelayMappedAddr}, transports::TransportsSender, }; +use crate::discovery::ConcurrentDiscovery; mod remote_state; diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index 5907519b9c..27cebc8eb0 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -598,7 +598,7 @@ impl RemoteStateActor { warn!("failed adding local addr: {err:#}",); } } - for addr in quinn_local_addrs.difference(&local_addrs) { + for addr in quinn_local_addrs.difference(local_addrs) { if let Err(err) = conn.remove_nat_traversal_address(*addr) { warn!("failed removing local addr: {err:#}"); } From 08274f1d6d9072845356e070c61c4d3c6d5d2f53 Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Sun, 23 Nov 2025 13:02:52 +0100 Subject: [PATCH 16/29] Make sure to use canonical addrs when comparing hp rounds --- iroh/src/magicsock/remote_map/remote_state.rs | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index 415fa5e0ac..ee01511fa5 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -285,8 +285,8 @@ impl RemoteStateActor { Some((id, evt)) = self.path_events.next() => { self.handle_path_event(id, evt); } - Some((id, _)) = self.addr_events.next() => { - trace!(?id, "remote addrs updated, triggering holepunching"); + Some((id, evt)) = self.addr_events.next() => { + trace!(?id, ?evt, "remote addrs updated, triggering holepunching"); self.trigger_holepunching().await; } Some(conn_id) = self.connections_close.next(), if !self.connections_close.is_empty() => { @@ -700,6 +700,10 @@ impl RemoteStateActor { .collect::>(); match conn.initiate_nat_traversal_round() { Ok(remote_candidates) => { + let remote_candidates = remote_candidates + .iter() + .map(|addr| SocketAddr::new(addr.ip().to_canonical(), addr.port())) + .collect(); trace!( ?local_candidates, ?remote_candidates, @@ -708,7 +712,7 @@ impl RemoteStateActor { self.last_holepunch = Some(HolepunchAttempt { when: Instant::now(), local_candidates, - remote_candidates: BTreeSet::from_iter(remote_candidates), + remote_candidates, }); } Err(err) => { @@ -1025,6 +1029,8 @@ pub(super) struct RemoteStateHandle { } /// Information about a holepunch attempt. +/// +/// Addresses are always stored in canonical form. #[derive(Debug)] struct HolepunchAttempt { when: Instant, From f12341d1b3c614dccaf3396af108d9241b9db42b Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Sun, 23 Nov 2025 13:06:07 +0100 Subject: [PATCH 17/29] bump quinn --- Cargo.lock | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2a9f7aca06..9e5abcb14f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1914,7 +1914,7 @@ dependencies = [ "libc", "percent-encoding", "pin-project-lite", - "socket2 0.5.10", + "socket2 0.6.0", "tokio", "tower-service", "tracing", @@ -2363,7 +2363,7 @@ dependencies = [ [[package]] name = "iroh-quinn" version = "0.14.0" -source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#fe22a2bf3744353b411df26abeca1c2dcd8a4c5f" +source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#7a094bd12f5ee643be3fd873d6d6eef3fafbc0e7" dependencies = [ "bytes", "cfg_aliases", @@ -2372,7 +2372,7 @@ dependencies = [ "pin-project-lite", "rustc-hash", "rustls", - "socket2 0.5.10", + "socket2 0.6.0", "thiserror 2.0.17", "tokio", "tokio-stream", @@ -2383,7 +2383,7 @@ dependencies = [ [[package]] name = "iroh-quinn-proto" version = "0.13.0" -source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#fe22a2bf3744353b411df26abeca1c2dcd8a4c5f" +source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#7a094bd12f5ee643be3fd873d6d6eef3fafbc0e7" dependencies = [ "bytes", "fastbloom", @@ -2406,12 +2406,12 @@ dependencies = [ [[package]] name = "iroh-quinn-udp" version = "0.5.12" -source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#fe22a2bf3744353b411df26abeca1c2dcd8a4c5f" +source = "git+https://github.com/n0-computer/quinn?branch=protocol-simplification#7a094bd12f5ee643be3fd873d6d6eef3fafbc0e7" dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.5.10", + "socket2 0.6.0", "tracing", "windows-sys 0.60.2", ] @@ -3416,7 +3416,7 @@ dependencies = [ "quinn-udp", "rustc-hash", "rustls", - "socket2 0.5.10", + "socket2 0.6.0", "thiserror 2.0.17", "tokio", "tracing", @@ -3453,7 +3453,7 @@ dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.5.10", + "socket2 0.6.0", "tracing", "windows-sys 0.60.2", ] @@ -5233,7 +5233,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.1", + "windows-sys 0.48.0", ] [[package]] From f0cdffe4c5f66a14985fb35e47ed50a10d43008a Mon Sep 17 00:00:00 2001 From: Floris Bruynooghe Date: Sun, 23 Nov 2025 14:13:30 +0100 Subject: [PATCH 18/29] replace trace log with event --- iroh/src/magicsock/remote_map/remote_state.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index ee01511fa5..8a656d223c 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -704,10 +704,12 @@ impl RemoteStateActor { .iter() .map(|addr| SocketAddr::new(addr.ip().to_canonical(), addr.port())) .collect(); - trace!( + event!( + target: "iroh::_events::qnt::init", + Level::DEBUG, + remote = %self.endpoint_id.fmt_short(), ?local_candidates, ?remote_candidates, - "nat traversal initiated" ); self.last_holepunch = Some(HolepunchAttempt { when: Instant::now(), From e92d8574746f1b1e13b1e58bbb7f83f600975c6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Fri, 14 Nov 2025 14:06:30 -0500 Subject: [PATCH 19/29] feat(iroh): prune old, inactive paths --- iroh/src/magicsock/endpoint_map/path_state.rs | 26 +++++ iroh/src/magicsock/remote_map/remote_state.rs | 98 +++++++++++++++++-- 2 files changed, 115 insertions(+), 9 deletions(-) create mode 100644 iroh/src/magicsock/endpoint_map/path_state.rs diff --git a/iroh/src/magicsock/endpoint_map/path_state.rs b/iroh/src/magicsock/endpoint_map/path_state.rs new file mode 100644 index 0000000000..dc7a2dce60 --- /dev/null +++ b/iroh/src/magicsock/endpoint_map/path_state.rs @@ -0,0 +1,26 @@ +//! The state kept for each network path to a remote endpoint. + +use std::collections::HashMap; + +use n0_future::time::Instant; + +use super::Source; +use crate::disco::TransactionId; + +/// The state of a single path to the remote endpoint. +/// +/// Each path is identified by the destination [`transports::Addr`] and they are stored in +/// the [`EndpointStateActor::paths`] map. +/// +/// [`transports::Addr`]: super::transports::Addr +/// [`EndpointStateActor::paths`]: super::endpoint_state::EndpointStateActor +#[derive(Debug, Default, Clone)] +pub(super) struct PathState { + /// How we learned about this path, and when. + /// + /// We keep track of only the latest [`Instant`] for each [`Source`], keeping the size + /// of the map of sources down to one entry per type of source. + pub(super) sources: HashMap, + /// The last ping sent on this path. + pub(super) ping_sent: Option, +} diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index 8a656d223c..d60a836b92 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -26,7 +26,7 @@ use tracing::{Instrument, Level, debug, error, event, info_span, instrument, tra use self::{ guarded_channel::{GuardedReceiver, GuardedSender, guarded_channel}, - path_state::RemotePathState, + path_state::{PathState, RemotePathState}, }; use super::Source; use crate::{ @@ -50,15 +50,15 @@ const HOLEPUNCH_ATTEMPTS_INTERVAL: Duration = Duration::from_secs(5); mod guarded_channel; mod path_state; -// TODO: use this -// /// Number of addresses that are not active that we keep around per endpoint. -// /// -// /// See [`RemoteState::prune_direct_addresses`]. -// pub(super) const MAX_INACTIVE_DIRECT_ADDRESSES: usize = 20; +/// Number of addresses that are not active that we keep around per endpoint. +/// +/// See [`EndpointState::prune_ip_addresses`]. +pub(super) const MAX_INACTIVE_IP_ADDRESSES: usize = 20; -// TODO: use this -// /// How long since an endpoint path was last alive before it might be pruned. -// const LAST_ALIVE_PRUNE_DURATION: Duration = Duration::from_secs(120); +/// Max duration of how long ago we learned about this source before we are willing +/// to prune it, if the path for this ip address is inactive. +/// TODO(ramfox): fix this comment it's not clear enough +const LAST_SOURCE_PRUNE_DURATION: Duration = Duration::from_secs(120); // TODO: use this // /// The latency at or under which we don't try to upgrade to a better path. @@ -1348,3 +1348,83 @@ fn to_transports_addr( } }) } + +fn prune_paths( + paths: &mut FxHashMap, + pending: &VecDeque, + selected_path: &Option, + open_paths: &Vec, +) { + let ip_paths: BTreeSet<_> = paths + .keys() + .filter(|p| { + if p.is_ip() { + return true; + } + return false; + }) + .cloned() + .collect(); + // if the total number of ip paths is less than the allowed number of inactive + // paths, just return early; + if ip_paths.len() < MAX_INACTIVE_IP_ADDRESSES { + return; + } + + let mut protected_paths = std::collections::BTreeSet::new(); + for addr in pending { + protected_paths.insert(addr.clone()); + } + if let Some(path) = selected_path { + protected_paths.insert(path.clone()); + } + for path in open_paths { + protected_paths.insert(path.clone()); + } + + let inactive_paths: Vec<_> = ip_paths.difference(&protected_paths).collect(); + + if inactive_paths.len() < MAX_INACTIVE_IP_ADDRESSES { + return; + } + + let mut keep_paths = Vec::new(); + let now = Instant::now(); + // if the last instance in the source was CONST time ago, it can be pruned + for (addr, state) in paths { + if inactive_paths.contains(&&addr) { + let mut is_expired = true; + for (_source, instant) in &state.sources { + // it's been less than LAST_SOURCE_PRUNE_DURATION since we + // last learned about this source + if *instant + LAST_SOURCE_PRUNE_DURATION < now { + is_expired = false; + break; + } + } + if !is_expired { + keep_paths.push(addr); + } + continue; + } else { + keep_paths.push(addr); + } + } + + *paths = paths + .iter() + .to_owned() + .filter(|(addr, _)| keep_paths.contains(addr)) + .map(|(addr, state)| (addr.clone(), state.clone())) + .collect(); +} + +#[cfg(test)] +mod tests { + use n0_error::Result; + + #[test] + fn test_prune_paths() -> Result { + todo!(); + } +} From 55303df419e3762fffa5c4a17b2cb91f0184ed90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Mon, 17 Nov 2025 18:40:43 +0000 Subject: [PATCH 20/29] added tests for `prune_paths`, added `EndpointActorState::prune_paths` method, and called it whenever new paths are added --- iroh/src/magicsock/remote_map.rs | 5 - iroh/src/magicsock/remote_map/remote_state.rs | 283 +++++++++++++++--- 2 files changed, 237 insertions(+), 51 deletions(-) diff --git a/iroh/src/magicsock/remote_map.rs b/iroh/src/magicsock/remote_map.rs index 583b491681..007aaeef42 100644 --- a/iroh/src/magicsock/remote_map.rs +++ b/iroh/src/magicsock/remote_map.rs @@ -190,11 +190,6 @@ pub enum Source { _0: Private, }, /// We established a connection on this address. - /// - /// Currently this means the path was in uses as [`PathId::ZERO`] when the a connection - /// was added to the `RemoteStateActor`. - /// - /// [`PathId::ZERO`]: quinn_proto::PathId::ZERO #[strum(serialize = "Connection")] Connection { /// private marker diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index d60a836b92..e813aedae8 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -1,5 +1,5 @@ use std::{ - collections::{BTreeSet, VecDeque}, + collections::{BTreeSet, HashSet, VecDeque}, net::SocketAddr, pin::Pin, sync::Arc, @@ -479,6 +479,8 @@ impl RemoteStateActor { self.paths .insert(path_remote, Source::Connection { _0: Private }); self.select_path(); + // TODO(ramfox): do we need to prune paths here? + self.prune_paths(); if path_remote_is_ip { // We may have raced this with a relay address. Try and add any @@ -822,6 +824,7 @@ impl RemoteStateActor { } self.select_path(); + self.prune_paths(); } PathEvent::Abandoned { id, path_stats } => { trace!(?path_stats, "path abandoned"); @@ -973,6 +976,25 @@ impl RemoteStateActor { } } } + + fn prune_paths(&mut self) { + // if the total number of paths, relay or ip, is less + // than the max inactive ip addrs we allow, bail early + if self.paths.len() < MAX_INACTIVE_IP_ADDRESSES { + return; + } + let open_paths = self + .connections + .values() + .map(|state| state.open_paths.values()) + .flatten(); + prune_paths( + &mut self.paths, + &self.pending_open_paths, + &self.selected_path.get(), + open_paths, + ); + } } /// Messages to send to the [`RemoteStateActor`]. @@ -1349,82 +1371,251 @@ fn to_transports_addr( }) } -fn prune_paths( +fn prune_paths<'a>( paths: &mut FxHashMap, pending: &VecDeque, selected_path: &Option, - open_paths: &Vec, + open_paths: impl Iterator, ) { - let ip_paths: BTreeSet<_> = paths - .keys() - .filter(|p| { - if p.is_ip() { - return true; - } - return false; - }) - .cloned() - .collect(); + let ip_count = paths.keys().filter(|p| p.is_ip()).count(); // if the total number of ip paths is less than the allowed number of inactive // paths, just return early; - if ip_paths.len() < MAX_INACTIVE_IP_ADDRESSES { + if ip_count < MAX_INACTIVE_IP_ADDRESSES { return; } - let mut protected_paths = std::collections::BTreeSet::new(); + let ip_paths: HashSet<_> = paths.keys().filter(|p| p.is_ip()).collect(); + + let mut protected_paths = HashSet::new(); for addr in pending { - protected_paths.insert(addr.clone()); + protected_paths.insert(addr); } if let Some(path) = selected_path { - protected_paths.insert(path.clone()); + protected_paths.insert(path); } for path in open_paths { - protected_paths.insert(path.clone()); + protected_paths.insert(path); } - let inactive_paths: Vec<_> = ip_paths.difference(&protected_paths).collect(); + let inactive_paths: HashSet<_> = ip_paths + .difference(&protected_paths) + // cloned here so we can use `paths.retain` later + .map(|&addr| addr.clone()) + .collect(); if inactive_paths.len() < MAX_INACTIVE_IP_ADDRESSES { return; } - let mut keep_paths = Vec::new(); let now = Instant::now(); - // if the last instance in the source was CONST time ago, it can be pruned - for (addr, state) in paths { - if inactive_paths.contains(&&addr) { - let mut is_expired = true; - for (_source, instant) in &state.sources { - // it's been less than LAST_SOURCE_PRUNE_DURATION since we - // last learned about this source - if *instant + LAST_SOURCE_PRUNE_DURATION < now { - is_expired = false; - break; - } - } - if !is_expired { - keep_paths.push(addr); - } - continue; + + paths.retain(|addr, state| { + if inactive_paths.contains(addr) { + keep_path(state, &now) } else { - keep_paths.push(addr); + // keep all active paths + true } - } + }); +} - *paths = paths - .iter() - .to_owned() - .filter(|(addr, _)| keep_paths.contains(addr)) - .map(|(addr, state)| (addr.clone(), state.clone())) - .collect(); +/// Based on the [`PathState`], returns true if we should keep this path. +/// +/// Currently we have two criteria: +/// 1) This path has sent a Ping +/// 2) The last time we learned about this address was greater than LAST_SOURCE_PRUNE_DURATION +fn keep_path(state: &PathState, now: &Instant) -> bool { + // if we have never sent a ping, don't remove it + state.ping_sent.is_none() + || state + .sources + .values() + // only keep it if this path contains recent sources + .any(|instant| *instant + LAST_SOURCE_PRUNE_DURATION > *now) } #[cfg(test)] mod tests { + use super::super::Source; + use super::{PathState, prune_paths}; + use crate::disco::TransactionId; + use crate::magicsock::{endpoint_map::Private, transports}; use n0_error::Result; + use n0_future::time::{Duration, Instant}; + use rustc_hash::FxHashMap; + use std::collections::VecDeque; + use std::net::{Ipv4Addr, SocketAddr}; + + /// Create a test IP address with specific port + fn test_ip_addr(port: u16) -> transports::Addr { + transports::Addr::Ip(SocketAddr::new( + std::net::IpAddr::V4(Ipv4Addr::LOCALHOST), + port, + )) + } + + /// Create a PathState with sources at a specific time offset + fn test_path_state(time_offset: Duration, sent_ping: bool) -> PathState { + let mut state = PathState::default(); + if sent_ping { + state.ping_sent = Some(TransactionId::default()); + } + state.sources.insert( + Source::Connection { _0: Private }, + Instant::now() - time_offset, + ); + state + } + + #[test] + fn test_prune_paths_too_few_total_paths() -> Result { + // create fewer than MAX_INACTIVE_IP_ADDRESSES paths + let mut paths = FxHashMap::default(); + for i in 0..15 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(0), false), + ); + } + + let pending = VecDeque::new(); + let selected_path = None; + let open_paths = Vec::new(); + + let initial_len = paths.len(); + // should not prune because we have fewer than MAX_INACTIVE_IP_ADDRESSES paths + prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + assert_eq!( + paths.len(), + initial_len, + "Expected no paths to be pruned when total IP paths < MAX_INACTIVE_IP_ADDRESSES" + ); + + Ok(()) + } #[test] - fn test_prune_paths() -> Result { - todo!(); + fn test_prune_paths_too_few_inactive_paths() -> Result { + // create MAX_INACTIVE_IP_ADDRESSES + 5 paths + let mut paths = FxHashMap::default(); + for i in 0..25 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(0), false), + ); + } + + // mark 10 of them as "active" by adding them to open_paths + let open_paths: Vec = (0..10).map(|i| test_ip_addr(i)).collect(); + + let pending = VecDeque::new(); + let selected_path = None; + + let initial_len = paths.len(); + // now we have 25 total paths, but only 15 inactive paths (25 - 10 = 15) + // which is less than MAX_INACTIVE_IP_ADDRESSES (20) + prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + assert_eq!( + paths.len(), + initial_len, + "Expected no paths to be pruned when inactive paths < MAX_INACTIVE_IP_ADDRESSES" + ); + + Ok(()) + } + + #[test] + fn test_prune_paths_prunes_old_inactive_paths() -> Result { + // create MAX_INACTIVE_IP_ADDRESSES + 10 paths + let mut paths = FxHashMap::default(); + + // add 20 paths with recent sources (within 2 minutes) + for i in 0..20 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(60), true), // 1 minute ago + ); + } + + // add 10 paths with old sources (more than 2 minutes ago) + for i in 20..30 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(180), true), // 3 minutes ago + ); + } + + let pending = VecDeque::new(); + let selected_path = None; + let open_paths = Vec::new(); + + // we have 30 total paths, all inactive + // paths with sources older than LAST_SOURCE_PRUNE_DURATION should be pruned + prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + + // we should have kept the 20 recent paths + assert_eq!( + paths.len(), + 20, + "Expected to keep 20 paths with recent sources" + ); + + // verify that the kept paths are the ones with recent sources + for i in 0..20 { + let addr = test_ip_addr(i); + assert!( + paths.contains_key(&addr), + "Expected to keep path with recent source: {:?}", + addr + ); + } + + // verify that the old paths were removed + for i in 20..30 { + let addr = test_ip_addr(i); + assert!( + !paths.contains_key(&addr), + "Expected to prune path with old source: {:?}", + addr + ); + } + + Ok(()) + } + + #[test] + fn test_prune_paths_protects_selected_and_open_paths() -> Result { + // create MAX_INACTIVE_IP_ADDRESSES + 10 paths, all with old sources + let mut paths = FxHashMap::default(); + for i in 0..30 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(180), true), // 3 minutes ago + ); + } + + let pending = VecDeque::new(); + // mark one path as selected + let selected_path = Some(test_ip_addr(0)); + // mark a few paths as open + let open_paths = vec![test_ip_addr(1), test_ip_addr(2)]; + + prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + + // protected paths should still be in the result even though they have old sources + assert!( + paths.contains_key(&test_ip_addr(0)), + "Expected to keep selected path even with old source" + ); + assert!( + paths.contains_key(&test_ip_addr(1)), + "Expected to keep open path even with old source" + ); + assert!( + paths.contains_key(&test_ip_addr(2)), + "Expected to keep open path even with old source" + ); + + Ok(()) } } From 57d8a780dc3150f26bdc7b8b8fbb8ebce5224f60 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Mon, 17 Nov 2025 18:57:02 +0000 Subject: [PATCH 21/29] rename `prune_paths` to `prune_ip_paths`, clippy, and fmt --- iroh/src/magicsock/remote_map/remote_state.rs | 279 +---------------- .../remote_map/remote_state/path_state.rs | 289 +++++++++++++++++- 2 files changed, 298 insertions(+), 270 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index e813aedae8..305d2c2338 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -1,5 +1,5 @@ use std::{ - collections::{BTreeSet, HashSet, VecDeque}, + collections::{BTreeSet, VecDeque}, net::SocketAddr, pin::Pin, sync::Arc, @@ -26,7 +26,7 @@ use tracing::{Instrument, Level, debug, error, event, info_span, instrument, tra use self::{ guarded_channel::{GuardedReceiver, GuardedSender, guarded_channel}, - path_state::{PathState, RemotePathState}, + path_state::RemotePathState, }; use super::Source; use crate::{ @@ -50,16 +50,6 @@ const HOLEPUNCH_ATTEMPTS_INTERVAL: Duration = Duration::from_secs(5); mod guarded_channel; mod path_state; -/// Number of addresses that are not active that we keep around per endpoint. -/// -/// See [`EndpointState::prune_ip_addresses`]. -pub(super) const MAX_INACTIVE_IP_ADDRESSES: usize = 20; - -/// Max duration of how long ago we learned about this source before we are willing -/// to prune it, if the path for this ip address is inactive. -/// TODO(ramfox): fix this comment it's not clear enough -const LAST_SOURCE_PRUNE_DURATION: Duration = Duration::from_secs(120); - // TODO: use this // /// The latency at or under which we don't try to upgrade to a better path. // const GOOD_ENOUGH_LATENCY: Duration = Duration::from_millis(5); @@ -480,7 +470,7 @@ impl RemoteStateActor { .insert(path_remote, Source::Connection { _0: Private }); self.select_path(); // TODO(ramfox): do we need to prune paths here? - self.prune_paths(); + self.prune_ip_paths(); if path_remote_is_ip { // We may have raced this with a relay address. Try and add any @@ -824,7 +814,7 @@ impl RemoteStateActor { } self.select_path(); - self.prune_paths(); + self.prune_ip_paths(); } PathEvent::Abandoned { id, path_stats } => { trace!(?path_stats, "path abandoned"); @@ -977,19 +967,19 @@ impl RemoteStateActor { } } - fn prune_paths(&mut self) { + /// TODO: fix up docs once review indicates this is actually + /// the criteria for pruning. + fn prune_ip_paths(&mut self) { // if the total number of paths, relay or ip, is less // than the max inactive ip addrs we allow, bail early - if self.paths.len() < MAX_INACTIVE_IP_ADDRESSES { + if self.paths.len() < path_state::MAX_INACTIVE_IP_ADDRESSES { return; } let open_paths = self .connections .values() - .map(|state| state.open_paths.values()) - .flatten(); - prune_paths( - &mut self.paths, + .flat_map(|state| state.open_paths.values()); + self.paths.prune_ip_paths( &self.pending_open_paths, &self.selected_path.get(), open_paths, @@ -1370,252 +1360,3 @@ fn to_transports_addr( } }) } - -fn prune_paths<'a>( - paths: &mut FxHashMap, - pending: &VecDeque, - selected_path: &Option, - open_paths: impl Iterator, -) { - let ip_count = paths.keys().filter(|p| p.is_ip()).count(); - // if the total number of ip paths is less than the allowed number of inactive - // paths, just return early; - if ip_count < MAX_INACTIVE_IP_ADDRESSES { - return; - } - - let ip_paths: HashSet<_> = paths.keys().filter(|p| p.is_ip()).collect(); - - let mut protected_paths = HashSet::new(); - for addr in pending { - protected_paths.insert(addr); - } - if let Some(path) = selected_path { - protected_paths.insert(path); - } - for path in open_paths { - protected_paths.insert(path); - } - - let inactive_paths: HashSet<_> = ip_paths - .difference(&protected_paths) - // cloned here so we can use `paths.retain` later - .map(|&addr| addr.clone()) - .collect(); - - if inactive_paths.len() < MAX_INACTIVE_IP_ADDRESSES { - return; - } - - let now = Instant::now(); - - paths.retain(|addr, state| { - if inactive_paths.contains(addr) { - keep_path(state, &now) - } else { - // keep all active paths - true - } - }); -} - -/// Based on the [`PathState`], returns true if we should keep this path. -/// -/// Currently we have two criteria: -/// 1) This path has sent a Ping -/// 2) The last time we learned about this address was greater than LAST_SOURCE_PRUNE_DURATION -fn keep_path(state: &PathState, now: &Instant) -> bool { - // if we have never sent a ping, don't remove it - state.ping_sent.is_none() - || state - .sources - .values() - // only keep it if this path contains recent sources - .any(|instant| *instant + LAST_SOURCE_PRUNE_DURATION > *now) -} - -#[cfg(test)] -mod tests { - use super::super::Source; - use super::{PathState, prune_paths}; - use crate::disco::TransactionId; - use crate::magicsock::{endpoint_map::Private, transports}; - use n0_error::Result; - use n0_future::time::{Duration, Instant}; - use rustc_hash::FxHashMap; - use std::collections::VecDeque; - use std::net::{Ipv4Addr, SocketAddr}; - - /// Create a test IP address with specific port - fn test_ip_addr(port: u16) -> transports::Addr { - transports::Addr::Ip(SocketAddr::new( - std::net::IpAddr::V4(Ipv4Addr::LOCALHOST), - port, - )) - } - - /// Create a PathState with sources at a specific time offset - fn test_path_state(time_offset: Duration, sent_ping: bool) -> PathState { - let mut state = PathState::default(); - if sent_ping { - state.ping_sent = Some(TransactionId::default()); - } - state.sources.insert( - Source::Connection { _0: Private }, - Instant::now() - time_offset, - ); - state - } - - #[test] - fn test_prune_paths_too_few_total_paths() -> Result { - // create fewer than MAX_INACTIVE_IP_ADDRESSES paths - let mut paths = FxHashMap::default(); - for i in 0..15 { - paths.insert( - test_ip_addr(i), - test_path_state(Duration::from_secs(0), false), - ); - } - - let pending = VecDeque::new(); - let selected_path = None; - let open_paths = Vec::new(); - - let initial_len = paths.len(); - // should not prune because we have fewer than MAX_INACTIVE_IP_ADDRESSES paths - prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); - assert_eq!( - paths.len(), - initial_len, - "Expected no paths to be pruned when total IP paths < MAX_INACTIVE_IP_ADDRESSES" - ); - - Ok(()) - } - - #[test] - fn test_prune_paths_too_few_inactive_paths() -> Result { - // create MAX_INACTIVE_IP_ADDRESSES + 5 paths - let mut paths = FxHashMap::default(); - for i in 0..25 { - paths.insert( - test_ip_addr(i), - test_path_state(Duration::from_secs(0), false), - ); - } - - // mark 10 of them as "active" by adding them to open_paths - let open_paths: Vec = (0..10).map(|i| test_ip_addr(i)).collect(); - - let pending = VecDeque::new(); - let selected_path = None; - - let initial_len = paths.len(); - // now we have 25 total paths, but only 15 inactive paths (25 - 10 = 15) - // which is less than MAX_INACTIVE_IP_ADDRESSES (20) - prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); - assert_eq!( - paths.len(), - initial_len, - "Expected no paths to be pruned when inactive paths < MAX_INACTIVE_IP_ADDRESSES" - ); - - Ok(()) - } - - #[test] - fn test_prune_paths_prunes_old_inactive_paths() -> Result { - // create MAX_INACTIVE_IP_ADDRESSES + 10 paths - let mut paths = FxHashMap::default(); - - // add 20 paths with recent sources (within 2 minutes) - for i in 0..20 { - paths.insert( - test_ip_addr(i), - test_path_state(Duration::from_secs(60), true), // 1 minute ago - ); - } - - // add 10 paths with old sources (more than 2 minutes ago) - for i in 20..30 { - paths.insert( - test_ip_addr(i), - test_path_state(Duration::from_secs(180), true), // 3 minutes ago - ); - } - - let pending = VecDeque::new(); - let selected_path = None; - let open_paths = Vec::new(); - - // we have 30 total paths, all inactive - // paths with sources older than LAST_SOURCE_PRUNE_DURATION should be pruned - prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); - - // we should have kept the 20 recent paths - assert_eq!( - paths.len(), - 20, - "Expected to keep 20 paths with recent sources" - ); - - // verify that the kept paths are the ones with recent sources - for i in 0..20 { - let addr = test_ip_addr(i); - assert!( - paths.contains_key(&addr), - "Expected to keep path with recent source: {:?}", - addr - ); - } - - // verify that the old paths were removed - for i in 20..30 { - let addr = test_ip_addr(i); - assert!( - !paths.contains_key(&addr), - "Expected to prune path with old source: {:?}", - addr - ); - } - - Ok(()) - } - - #[test] - fn test_prune_paths_protects_selected_and_open_paths() -> Result { - // create MAX_INACTIVE_IP_ADDRESSES + 10 paths, all with old sources - let mut paths = FxHashMap::default(); - for i in 0..30 { - paths.insert( - test_ip_addr(i), - test_path_state(Duration::from_secs(180), true), // 3 minutes ago - ); - } - - let pending = VecDeque::new(); - // mark one path as selected - let selected_path = Some(test_ip_addr(0)); - // mark a few paths as open - let open_paths = vec![test_ip_addr(1), test_ip_addr(2)]; - - prune_paths(&mut paths, &pending, &selected_path, open_paths.iter()); - - // protected paths should still be in the result even though they have old sources - assert!( - paths.contains_key(&test_ip_addr(0)), - "Expected to keep selected path even with old source" - ); - assert!( - paths.contains_key(&test_ip_addr(1)), - "Expected to keep open path even with old source" - ); - assert!( - paths.contains_key(&test_ip_addr(2)), - "Expected to keep open path even with old source" - ); - - Ok(()) - } -} diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index ae3656a2a8..2117b53b0e 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -1,6 +1,9 @@ //! The state kept for each network path to a remote endpoint. -use std::collections::{HashMap, VecDeque}; +use std::{ + collections::{HashMap, HashSet, VecDeque}, + time::Duration, +}; use n0_error::e; use n0_future::time::Instant; @@ -11,6 +14,16 @@ use tracing::trace; use super::Source; use crate::{discovery::DiscoveryError, magicsock::transports}; +/// Number of addresses that are not active that we keep around per endpoint. +/// +/// See [`EndpointState::prune_ip_paths`]. +pub(super) const MAX_INACTIVE_IP_ADDRESSES: usize = 20; + +/// Max duration of how long ago we learned about this source before we are willing +/// to prune it, if the path for this ip address is inactive. +/// TODO(ramfox): fix this comment it's not clear enough +const LAST_SOURCE_PRUNE_DURATION: Duration = Duration::from_secs(120); + /// Map of all paths that we are aware of for a remote endpoint. /// /// Also stores a list of resolve requests which are triggered once at least one path is known, @@ -108,6 +121,26 @@ impl RemotePathState { tx.send(result.clone()).ok(); } } + + pub(super) fn len(&self) -> usize { + self.paths.len() + } + + /// TODO: fix up docs once review indicates this is actually + /// the criteria for pruning. + pub(super) fn prune_ip_paths<'a>( + &mut self, + pending: &VecDeque, + selected_path: &Option, + open_paths: impl Iterator, + ) { + // if the total number of paths, relay or ip, is less + // than the max inactive ip addrs we allow, bail early + if self.paths.len() < MAX_INACTIVE_IP_ADDRESSES { + return; + } + prune_ip_paths(&mut self.paths, pending, selected_path, open_paths); + } } /// The state of a single path to the remote endpoint. @@ -124,3 +157,257 @@ pub(super) struct PathState { /// of the map of sources down to one entry per type of source. pub(super) sources: HashMap, } + +fn prune_ip_paths<'a>( + paths: &mut FxHashMap, + pending: &VecDeque, + selected_path: &Option, + open_paths: impl Iterator, +) { + let ip_count = paths.keys().filter(|p| p.is_ip()).count(); + // if the total number of ip paths is less than the allowed number of inactive + // paths, just return early; + if ip_count < MAX_INACTIVE_IP_ADDRESSES { + return; + } + + let ip_paths: HashSet<_> = paths.keys().filter(|p| p.is_ip()).collect(); + + let mut protected_paths = HashSet::new(); + for addr in pending { + protected_paths.insert(addr); + } + if let Some(path) = selected_path { + protected_paths.insert(path); + } + for path in open_paths { + protected_paths.insert(path); + } + + let inactive_paths: HashSet<_> = ip_paths + .difference(&protected_paths) + // cloned here so we can use `paths.retain` later + .map(|&addr| addr.clone()) + .collect(); + + if inactive_paths.len() < MAX_INACTIVE_IP_ADDRESSES { + return; + } + + let now = Instant::now(); + + paths.retain(|addr, state| { + if inactive_paths.contains(addr) { + keep_path(state, &now) + } else { + // keep all active paths + true + } + }); +} + +/// Based on the [`PathState`], returns true if we should keep this path. +/// +/// Currently we have two criteria: +/// 1) This path has sent a Ping +/// 2) The last time we learned about this address was greater than LAST_SOURCE_PRUNE_DURATION +fn keep_path(state: &PathState, now: &Instant) -> bool { + // if we have never sent a ping, don't remove it + state.ping_sent.is_none() + || state + .sources + .values() + // only keep it if this path contains recent sources + .any(|instant| *instant + LAST_SOURCE_PRUNE_DURATION > *now) +} + +#[cfg(test)] +mod tests { + use std::{ + collections::VecDeque, + net::{Ipv4Addr, SocketAddr}, + }; + + use n0_error::Result; + use n0_future::time::{Duration, Instant}; + use rustc_hash::FxHashMap; + + use super::*; + use crate::{ + disco::TransactionId, + magicsock::{remote_map::Private, transports}, + }; + + /// Create a test IP address with specific port + fn test_ip_addr(port: u16) -> transports::Addr { + transports::Addr::Ip(SocketAddr::new( + std::net::IpAddr::V4(Ipv4Addr::LOCALHOST), + port, + )) + } + + /// Create a PathState with sources at a specific time offset + fn test_path_state(time_offset: Duration, sent_ping: bool) -> PathState { + let mut state = PathState::default(); + if sent_ping { + state.ping_sent = Some(TransactionId::default()); + } + state.sources.insert( + Source::Connection { _0: Private }, + Instant::now() - time_offset, + ); + state + } + + #[test] + fn test_prune_ip_paths_too_few_total_paths() -> Result { + // create fewer than MAX_INACTIVE_IP_ADDRESSES paths + let mut paths = FxHashMap::default(); + for i in 0..15 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(0), false), + ); + } + + let pending = VecDeque::new(); + let selected_path = None; + let open_paths = Vec::new(); + + let initial_len = paths.len(); + // should not prune because we have fewer than MAX_INACTIVE_IP_ADDRESSES paths + prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + assert_eq!( + paths.len(), + initial_len, + "Expected no paths to be pruned when total IP paths < MAX_INACTIVE_IP_ADDRESSES" + ); + + Ok(()) + } + + #[test] + fn test_prune_ip_paths_too_few_inactive_paths() -> Result { + // create MAX_INACTIVE_IP_ADDRESSES + 5 paths + let mut paths = FxHashMap::default(); + for i in 0..25 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(0), false), + ); + } + + // mark 10 of them as "active" by adding them to open_paths + let open_paths: Vec = (0..10).map(test_ip_addr).collect(); + + let pending = VecDeque::new(); + let selected_path = None; + + let initial_len = paths.len(); + // now we have 25 total paths, but only 15 inactive paths (25 - 10 = 15) + // which is less than MAX_INACTIVE_IP_ADDRESSES (20) + prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + assert_eq!( + paths.len(), + initial_len, + "Expected no paths to be pruned when inactive paths < MAX_INACTIVE_IP_ADDRESSES" + ); + + Ok(()) + } + + #[test] + fn test_prune_ip_paths_prunes_old_inactive_paths() -> Result { + // create MAX_INACTIVE_IP_ADDRESSES + 10 paths + let mut paths = FxHashMap::default(); + + // add 20 paths with recent sources (within 2 minutes) + for i in 0..20 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(60), true), // 1 minute ago + ); + } + + // add 10 paths with old sources (more than 2 minutes ago) + for i in 20..30 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(180), true), // 3 minutes ago + ); + } + + let pending = VecDeque::new(); + let selected_path = None; + let open_paths = Vec::new(); + + // we have 30 total paths, all inactive + // paths with sources older than LAST_SOURCE_PRUNE_DURATION should be pruned + prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + + // we should have kept the 20 recent paths + assert_eq!( + paths.len(), + 20, + "Expected to keep 20 paths with recent sources" + ); + + // verify that the kept paths are the ones with recent sources + for i in 0..20 { + let addr = test_ip_addr(i); + assert!( + paths.contains_key(&addr), + "Expected to keep path with recent source: {:?}", + addr + ); + } + + // verify that the old paths were removed + for i in 20..30 { + let addr = test_ip_addr(i); + assert!( + !paths.contains_key(&addr), + "Expected to prune path with old source: {:?}", + addr + ); + } + + Ok(()) + } + + #[test] + fn test_prune_ip_paths_protects_selected_and_open_paths() -> Result { + // create MAX_INACTIVE_IP_ADDRESSES + 10 paths, all with old sources + let mut paths = FxHashMap::default(); + for i in 0..30 { + paths.insert( + test_ip_addr(i), + test_path_state(Duration::from_secs(180), true), // 3 minutes ago + ); + } + + let pending = VecDeque::new(); + // mark one path as selected + let selected_path = Some(test_ip_addr(0)); + // mark a few paths as open + let open_paths = [test_ip_addr(1), test_ip_addr(2)]; + + prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + + // protected paths should still be in the result even though they have old sources + assert!( + paths.contains_key(&test_ip_addr(0)), + "Expected to keep selected path even with old source" + ); + assert!( + paths.contains_key(&test_ip_addr(1)), + "Expected to keep open path even with old source" + ); + assert!( + paths.contains_key(&test_ip_addr(2)), + "Expected to keep open path even with old source" + ); + + Ok(()) + } +} From eab21a6135da943640fdfe7e6925675af9e960ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Thu, 20 Nov 2025 13:01:50 +0100 Subject: [PATCH 22/29] docs: broken doc comment --- iroh/src/magicsock/remote_map/remote_state/path_state.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index 2117b53b0e..b1b93b72da 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -15,8 +15,6 @@ use super::Source; use crate::{discovery::DiscoveryError, magicsock::transports}; /// Number of addresses that are not active that we keep around per endpoint. -/// -/// See [`EndpointState::prune_ip_paths`]. pub(super) const MAX_INACTIVE_IP_ADDRESSES: usize = 20; /// Max duration of how long ago we learned about this source before we are willing From 017ad73f57b6680d8abfbd17fda30f14145f8a0b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Thu, 20 Nov 2025 18:08:16 +0100 Subject: [PATCH 23/29] add additional pruning constraint: do not prune if we have ever holepunched this connection --- iroh/src/magicsock/endpoint_map/path_state.rs | 26 ------- iroh/src/magicsock/remote_map/remote_state.rs | 9 +-- .../remote_map/remote_state/path_state.rs | 77 +++++++++++++++---- 3 files changed, 62 insertions(+), 50 deletions(-) delete mode 100644 iroh/src/magicsock/endpoint_map/path_state.rs diff --git a/iroh/src/magicsock/endpoint_map/path_state.rs b/iroh/src/magicsock/endpoint_map/path_state.rs deleted file mode 100644 index dc7a2dce60..0000000000 --- a/iroh/src/magicsock/endpoint_map/path_state.rs +++ /dev/null @@ -1,26 +0,0 @@ -//! The state kept for each network path to a remote endpoint. - -use std::collections::HashMap; - -use n0_future::time::Instant; - -use super::Source; -use crate::disco::TransactionId; - -/// The state of a single path to the remote endpoint. -/// -/// Each path is identified by the destination [`transports::Addr`] and they are stored in -/// the [`EndpointStateActor::paths`] map. -/// -/// [`transports::Addr`]: super::transports::Addr -/// [`EndpointStateActor::paths`]: super::endpoint_state::EndpointStateActor -#[derive(Debug, Default, Clone)] -pub(super) struct PathState { - /// How we learned about this path, and when. - /// - /// We keep track of only the latest [`Instant`] for each [`Source`], keeping the size - /// of the map of sources down to one entry per type of source. - pub(super) sources: HashMap, - /// The last ping sent on this path. - pub(super) ping_sent: Option, -} diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index 305d2c2338..e63fcb30b7 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -469,7 +469,6 @@ impl RemoteStateActor { self.paths .insert(path_remote, Source::Connection { _0: Private }); self.select_path(); - // TODO(ramfox): do we need to prune paths here? self.prune_ip_paths(); if path_remote_is_ip { @@ -810,7 +809,8 @@ impl RemoteStateActor { ); conn_state.add_open_path(path_remote.clone(), path_id); self.paths - .insert(path_remote, Source::Connection { _0: Private }); + .insert(path_remote.clone(), Source::Connection { _0: Private }); + self.paths.holepunched(&path_remote); } self.select_path(); @@ -970,11 +970,6 @@ impl RemoteStateActor { /// TODO: fix up docs once review indicates this is actually /// the criteria for pruning. fn prune_ip_paths(&mut self) { - // if the total number of paths, relay or ip, is less - // than the max inactive ip addrs we allow, bail early - if self.paths.len() < path_state::MAX_INACTIVE_IP_ADDRESSES { - return; - } let open_paths = self .connections .values() diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index b1b93b72da..7af84f70e0 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -22,6 +22,9 @@ pub(super) const MAX_INACTIVE_IP_ADDRESSES: usize = 20; /// TODO(ramfox): fix this comment it's not clear enough const LAST_SOURCE_PRUNE_DURATION: Duration = Duration::from_secs(120); +/// Duration after sending a ping in which we assume holepunching failed. +const PING_TIMEOUT: Duration = Duration::from_secs(30); + /// Map of all paths that we are aware of for a remote endpoint. /// /// Also stores a list of resolve requests which are triggered once at least one path is known, @@ -50,6 +53,18 @@ impl RemotePathState { self.emit_pending_resolve_requests(None); } + /// Record in the [`PathState`] for the given [`Addr`], that we have + /// successfully holepunched. + /// + /// If this path does exist, no information is added. + /// + /// [`Addr`]: transports::Addr + pub(super) fn holepunched(&mut self, addr: &transports::Addr) { + if let Some(path) = self.paths.get_mut(addr) { + path.holepunched = true; + } + } + /// Inserts multiple addresses into our list of potential paths. /// /// This will emit pending resolve requests. @@ -120,10 +135,6 @@ impl RemotePathState { } } - pub(super) fn len(&self) -> usize { - self.paths.len() - } - /// TODO: fix up docs once review indicates this is actually /// the criteria for pruning. pub(super) fn prune_ip_paths<'a>( @@ -154,6 +165,22 @@ pub(super) struct PathState { /// We keep track of only the latest [`Instant`] for each [`Source`], keeping the size /// of the map of sources down to one entry per type of source. pub(super) sources: HashMap, + /// The last ping sent on this path. + pub(super) ping_sent: Option<(TransactionId, Instant)>, + /// Last time we successfully holepunched. + pub(super) holepunched: bool, +} + +impl PathState { + /// Returns true if a ping was sent in the last [`PING_DURATION`] amount of time. + fn ping_in_process(&self, now: &Instant) -> bool { + if let Some((_, ping_sent)) = self.ping_sent { + if ping_sent + PING_TIMEOUT > *now { + return true; + } + } + false + } } fn prune_ip_paths<'a>( @@ -206,12 +233,19 @@ fn prune_ip_paths<'a>( /// Based on the [`PathState`], returns true if we should keep this path. /// -/// Currently we have two criteria: -/// 1) This path has sent a Ping -/// 2) The last time we learned about this address was greater than LAST_SOURCE_PRUNE_DURATION +/// Currently we have four criteria: +/// 1) This path has successfully holepunched, ever +/// 2) We have never attempted to holepunch on this path +/// 3) We are in the process of holepunching +/// 4) The last time we learned about this address was greater than LAST_SOURCE_PRUNE_DURATION +/// +/// In other words, paths that have never successfully holepunched, that we learned about +/// over `LAST_SOURCE_PRUNE_DURATION` ago, should not be kept. fn keep_path(state: &PathState, now: &Instant) -> bool { - // if we have never sent a ping, don't remove it - state.ping_sent.is_none() + // if we have never successfully holepunched + state.holepunched + || state.ping_sent.is_none() + || state.ping_in_process(now) || state .sources .values() @@ -245,10 +279,10 @@ mod tests { } /// Create a PathState with sources at a specific time offset - fn test_path_state(time_offset: Duration, sent_ping: bool) -> PathState { + fn test_path_state(time_offset: Duration, sent_ping: Option) -> PathState { let mut state = PathState::default(); - if sent_ping { - state.ping_sent = Some(TransactionId::default()); + if let Some(sent_ping_ago) = sent_ping { + state.ping_sent = Some((TransactionId::default(), Instant::now() - sent_ping_ago)); } state.sources.insert( Source::Connection { _0: Private }, @@ -264,7 +298,7 @@ mod tests { for i in 0..15 { paths.insert( test_ip_addr(i), - test_path_state(Duration::from_secs(0), false), + test_path_state(Duration::from_secs(0), None), ); } @@ -291,7 +325,7 @@ mod tests { for i in 0..25 { paths.insert( test_ip_addr(i), - test_path_state(Duration::from_secs(0), false), + test_path_state(Duration::from_secs(0), None), ); } @@ -323,7 +357,10 @@ mod tests { for i in 0..20 { paths.insert( test_ip_addr(i), - test_path_state(Duration::from_secs(60), true), // 1 minute ago + test_path_state( + Duration::from_secs(60), // learn about this path 1 min ago + Some(Duration::from_secs(60)), // sent ping 1 min ago + ), ); } @@ -331,7 +368,10 @@ mod tests { for i in 20..30 { paths.insert( test_ip_addr(i), - test_path_state(Duration::from_secs(180), true), // 3 minutes ago + test_path_state( + Duration::from_secs(180), // learned abou this path 3 mins ago + Some(Duration::from_secs(60)), + ), // sent ping 1 min ago ); } @@ -380,7 +420,10 @@ mod tests { for i in 0..30 { paths.insert( test_ip_addr(i), - test_path_state(Duration::from_secs(180), true), // 3 minutes ago + test_path_state( + Duration::from_secs(180), // learned about this path 3 mins ago + Some(Duration::from_secs(60)), // sent ping 1 min ago + ), ); } From 3d3e492c398dbafcac9eca8101667d9c57fb5bc1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Fri, 21 Nov 2025 13:14:19 +0100 Subject: [PATCH 24/29] review fixes --- .../remote_map/remote_state/path_state.rs | 29 +++++++------------ 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index 7af84f70e0..623e4e8dfd 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -143,11 +143,6 @@ impl RemotePathState { selected_path: &Option, open_paths: impl Iterator, ) { - // if the total number of paths, relay or ip, is less - // than the max inactive ip addrs we allow, bail early - if self.paths.len() < MAX_INACTIVE_IP_ADDRESSES { - return; - } prune_ip_paths(&mut self.paths, pending, selected_path, open_paths); } } @@ -166,15 +161,15 @@ pub(super) struct PathState { /// of the map of sources down to one entry per type of source. pub(super) sources: HashMap, /// The last ping sent on this path. - pub(super) ping_sent: Option<(TransactionId, Instant)>, + pub(super) ping_sent: Option, /// Last time we successfully holepunched. pub(super) holepunched: bool, } impl PathState { - /// Returns true if a ping was sent in the last [`PING_DURATION`] amount of time. + /// Returns true if a ping was sent in the last [`PING_TIMEOUT`] amount of time. fn ping_in_process(&self, now: &Instant) -> bool { - if let Some((_, ping_sent)) = self.ping_sent { + if let Some(ping_sent) = self.ping_sent { if ping_sent + PING_TIMEOUT > *now { return true; } @@ -189,12 +184,11 @@ fn prune_ip_paths<'a>( selected_path: &Option, open_paths: impl Iterator, ) { - let ip_count = paths.keys().filter(|p| p.is_ip()).count(); - // if the total number of ip paths is less than the allowed number of inactive - // paths, just return early; - if ip_count < MAX_INACTIVE_IP_ADDRESSES { + // if the total number of paths, relay or ip, is less + // than the max inactive ip addrs we allow, bail early + if paths.len() < MAX_INACTIVE_IP_ADDRESSES { return; - } + }; let ip_paths: HashSet<_> = paths.keys().filter(|p| p.is_ip()).collect(); @@ -265,10 +259,7 @@ mod tests { use rustc_hash::FxHashMap; use super::*; - use crate::{ - disco::TransactionId, - magicsock::{remote_map::Private, transports}, - }; + use crate::magicsock::{remote_map::Private, transports}; /// Create a test IP address with specific port fn test_ip_addr(port: u16) -> transports::Addr { @@ -282,7 +273,7 @@ mod tests { fn test_path_state(time_offset: Duration, sent_ping: Option) -> PathState { let mut state = PathState::default(); if let Some(sent_ping_ago) = sent_ping { - state.ping_sent = Some((TransactionId::default(), Instant::now() - sent_ping_ago)); + state.ping_sent = Some(Instant::now() - sent_ping_ago); } state.sources.insert( Source::Connection { _0: Private }, @@ -369,7 +360,7 @@ mod tests { paths.insert( test_ip_addr(i), test_path_state( - Duration::from_secs(180), // learned abou this path 3 mins ago + Duration::from_secs(180), // learned about this path 3 mins ago Some(Duration::from_secs(60)), ), // sent ping 1 min ago ); From c6de9e72faa377470d7c8f02d41f15fc025e483f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Fri, 21 Nov 2025 15:35:42 +0100 Subject: [PATCH 25/29] test: test more stuff --- .../remote_map/remote_state/path_state.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index 623e4e8dfd..cb54198d1b 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -418,6 +418,13 @@ mod tests { ); } + // mark path 3 as holepunched + paths.get_mut(&test_ip_addr(3)).unwrap().holepunched = true; + + // mark path 4 as having a recent ping (ping in process) + paths.get_mut(&test_ip_addr(4)).unwrap().ping_sent = + Some((TransactionId::default(), Instant::now() - Duration::from_secs(5))); + let pending = VecDeque::new(); // mark one path as selected let selected_path = Some(test_ip_addr(0)); @@ -439,6 +446,14 @@ mod tests { paths.contains_key(&test_ip_addr(2)), "Expected to keep open path even with old source" ); + assert!( + paths.contains_key(&test_ip_addr(3)), + "Expected to keep holepunched path even with old source" + ); + assert!( + paths.contains_key(&test_ip_addr(4)), + "Expected to keep path with ping in process even with old source" + ); Ok(()) } From becbcb639fedac68a3a2b524d3481ecada046b4e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Fri, 21 Nov 2025 18:15:33 +0100 Subject: [PATCH 26/29] fmt --- iroh/src/magicsock/remote_map/remote_state/path_state.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index cb54198d1b..d1cb8e1a08 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -422,8 +422,10 @@ mod tests { paths.get_mut(&test_ip_addr(3)).unwrap().holepunched = true; // mark path 4 as having a recent ping (ping in process) - paths.get_mut(&test_ip_addr(4)).unwrap().ping_sent = - Some((TransactionId::default(), Instant::now() - Duration::from_secs(5))); + paths.get_mut(&test_ip_addr(4)).unwrap().ping_sent = Some(( + TransactionId::default(), + Instant::now() - Duration::from_secs(5), + )); let pending = VecDeque::new(); // mark one path as selected From b954009585f90d9cbcf52b2c3e492aaa6c29838e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Sun, 23 Nov 2025 12:46:08 +0100 Subject: [PATCH 27/29] change prune ip logic --- iroh/src/magicsock/remote_map/remote_state.rs | 35 +- .../remote_map/remote_state/path_state.rs | 528 +++++++++--------- 2 files changed, 285 insertions(+), 278 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state.rs b/iroh/src/magicsock/remote_map/remote_state.rs index e63fcb30b7..1521853118 100644 --- a/iroh/src/magicsock/remote_map/remote_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state.rs @@ -467,9 +467,10 @@ impl RemoteStateActor { path.set_status(status).ok(); conn_state.add_open_path(path_remote.clone(), PathId::ZERO); self.paths - .insert(path_remote, Source::Connection { _0: Private }); + .insert(path_remote.clone(), Source::Connection { _0: Private }); + self.paths.opened_path(&path_remote); self.select_path(); - self.prune_ip_paths(); + self.prune_paths(); if path_remote_is_ip { // We may have raced this with a relay address. Try and add any @@ -506,6 +507,7 @@ impl RemoteStateActor { self.paths.resolve_remote(tx); // Start discovery if we have no selected path. self.trigger_discovery(); + self.prune_paths(); } /// Handles [`RemoteStateMessage::Latency`]. @@ -552,6 +554,7 @@ impl RemoteStateActor { let addrs = to_transports_addr(self.endpoint_id, item.into_endpoint_addr().addrs); self.paths.insert_multiple(addrs, source); + self.prune_paths(); } } } @@ -810,16 +813,18 @@ impl RemoteStateActor { conn_state.add_open_path(path_remote.clone(), path_id); self.paths .insert(path_remote.clone(), Source::Connection { _0: Private }); - self.paths.holepunched(&path_remote); + self.paths.opened_path(&path_remote); } self.select_path(); - self.prune_ip_paths(); + self.prune_paths(); } PathEvent::Abandoned { id, path_stats } => { trace!(?path_stats, "path abandoned"); // This is the last event for this path. - conn_state.remove_path(&id); + if let Some(addr) = conn_state.remove_path(&id) { + self.paths.abandoned_path(&addr); + } } PathEvent::Closed { id, .. } | PathEvent::LocallyClosed { id, .. } => { let Some(path_remote) = conn_state.paths.get(&id).cloned() else { @@ -969,16 +974,8 @@ impl RemoteStateActor { /// TODO: fix up docs once review indicates this is actually /// the criteria for pruning. - fn prune_ip_paths(&mut self) { - let open_paths = self - .connections - .values() - .flat_map(|state| state.open_paths.values()); - self.paths.prune_ip_paths( - &self.pending_open_paths, - &self.selected_path.get(), - open_paths, - ); + fn prune_paths(&mut self) { + self.paths.prune_paths(); } } @@ -1101,11 +1098,13 @@ impl ConnectionState { } /// Completely removes a path from this connection. - fn remove_path(&mut self, path_id: &PathId) { - if let Some(addr) = self.paths.remove(path_id) { - self.path_ids.remove(&addr); + fn remove_path(&mut self, path_id: &PathId) -> Option { + let addr = self.paths.remove(path_id); + if let Some(ref addr) = addr { + self.path_ids.remove(addr); } self.open_paths.remove(path_id); + addr } /// Removes the path from the open paths. diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index d1cb8e1a08..6de7c6f0ed 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -1,9 +1,6 @@ //! The state kept for each network path to a remote endpoint. -use std::{ - collections::{HashMap, HashSet, VecDeque}, - time::Duration, -}; +use std::collections::{HashMap, HashSet, VecDeque}; use n0_error::e; use n0_future::time::Instant; @@ -14,16 +11,13 @@ use tracing::trace; use super::Source; use crate::{discovery::DiscoveryError, magicsock::transports}; -/// Number of addresses that are not active that we keep around per endpoint. -pub(super) const MAX_INACTIVE_IP_ADDRESSES: usize = 20; +/// Maximum number of IP paths we keep around per endpoint. +pub(super) const MAX_IP_PATHS: usize = 30; -/// Max duration of how long ago we learned about this source before we are willing -/// to prune it, if the path for this ip address is inactive. -/// TODO(ramfox): fix this comment it's not clear enough -const LAST_SOURCE_PRUNE_DURATION: Duration = Duration::from_secs(120); - -/// Duration after sending a ping in which we assume holepunching failed. -const PING_TIMEOUT: Duration = Duration::from_secs(30); +/// Maximum number of inactive IP paths we keep around per endpoint. +/// +/// These are paths that at one point been opened and are now closed. +pub(super) const MAX_INACTIVE_IP_PATHS: usize = 10; /// Map of all paths that we are aware of for a remote endpoint. /// @@ -53,15 +47,24 @@ impl RemotePathState { self.emit_pending_resolve_requests(None); } - /// Record in the [`PathState`] for the given [`Addr`], that we have - /// successfully holepunched. + /// Mark a path as abandoned. /// - /// If this path does exist, no information is added. + /// If this path does not exist, it does nothing to the + /// `RemotePathState` + pub(super) fn abandoned_path(&mut self, addr: &transports::Addr) { + if let Some(state) = self.paths.get_mut(addr) { + state.abandoned = Some(Instant::now()); + } + } + + /// Mark a path as opened. /// - /// [`Addr`]: transports::Addr - pub(super) fn holepunched(&mut self, addr: &transports::Addr) { - if let Some(path) = self.paths.get_mut(addr) { - path.holepunched = true; + /// If this path does not exist, it does nothing to the + /// `RemotePathState` + pub(super) fn opened_path(&mut self, addr: &transports::Addr) { + if let Some(state) = self.paths.get_mut(addr) { + state.usable = true; + state.abandoned = None; } } @@ -137,13 +140,9 @@ impl RemotePathState { /// TODO: fix up docs once review indicates this is actually /// the criteria for pruning. - pub(super) fn prune_ip_paths<'a>( - &mut self, - pending: &VecDeque, - selected_path: &Option, - open_paths: impl Iterator, - ) { - prune_ip_paths(&mut self.paths, pending, selected_path, open_paths); + pub(super) fn prune_paths(&mut self) { + // right now we only prune IP paths + prune_ip_paths(&mut self.paths); } } @@ -160,303 +159,312 @@ pub(super) struct PathState { /// We keep track of only the latest [`Instant`] for each [`Source`], keeping the size /// of the map of sources down to one entry per type of source. pub(super) sources: HashMap, - /// The last ping sent on this path. - pub(super) ping_sent: Option, - /// Last time we successfully holepunched. - pub(super) holepunched: bool, -} - -impl PathState { - /// Returns true if a ping was sent in the last [`PING_TIMEOUT`] amount of time. - fn ping_in_process(&self, now: &Instant) -> bool { - if let Some(ping_sent) = self.ping_sent { - if ping_sent + PING_TIMEOUT > *now { - return true; - } - } - false - } + /// The last time this path was proven usable. + /// + /// If this is `false` and closed is `Some`, than we attempted to open this path, but + /// it did not work. + /// + /// If this is `false` and closed is `None`, than we do not know yet if this path is + /// usable. + pub(super) usable: bool, + /// The last time a path with this addr was abandoned. + /// + /// If this is `Some` and usable is `None`, than we attempted to use this path and it + /// did not work. + pub(super) abandoned: Option, } -fn prune_ip_paths<'a>( - paths: &mut FxHashMap, - pending: &VecDeque, - selected_path: &Option, - open_paths: impl Iterator, -) { - // if the total number of paths, relay or ip, is less - // than the max inactive ip addrs we allow, bail early - if paths.len() < MAX_INACTIVE_IP_ADDRESSES { +/// Prunes the IP paths in the paths HashMap. +/// +/// Only prunes if the number of IP paths is above [`MAX_IP_PATHS`]. +/// +/// Keeps paths that are active or have never been holepunched. +/// +/// Always prunes paths that have unsuccessfully holepunched. +/// +/// Keeps [`MAX_INACTIVE_PATHS`] of the most recently closed paths +/// that are not currently being used but have successfully been +/// holepunched previously. +/// +/// This all ensures that: +/// +/// - We do not have unbounded growth of paths. +/// - If we have many paths for this remote, we prune the paths that cannot hole punch. +/// - We do not prune holepunched paths that are currently not in use too quickly. For example, if a large number of untested paths are added at once, we will not immediately prune all of the unused, but valid, paths at once. +fn prune_ip_paths(paths: &mut FxHashMap) { + // if the total number of paths is less than the max, bail early + if paths.len() < MAX_IP_PATHS { return; - }; + } - let ip_paths: HashSet<_> = paths.keys().filter(|p| p.is_ip()).collect(); + let ip_paths: Vec<_> = paths.iter().filter(|(addr, _)| addr.is_ip()).collect(); - let mut protected_paths = HashSet::new(); - for addr in pending { - protected_paths.insert(addr); - } - if let Some(path) = selected_path { - protected_paths.insert(path); + // if the total number of ip paths is less than the max, bail early + if ip_paths.len() < MAX_IP_PATHS { + return; } - for path in open_paths { - protected_paths.insert(path); + + // paths that were opened at one point but have previously been closed + let mut can_prune = Vec::new(); + // paths where we attempted hole punching but it not successful + let mut must_prune = Vec::new(); + + for (addr, state) in ip_paths { + match state.abandoned { + // If a path has never been abandoned, it is either + // open currently or has never been dialed. + // Keep it. + None => {} + Some(abandoned) => { + if state.usable { + // These are paths where hole punching succeeded at one point, but the path was closed. + can_prune.push((addr.clone(), abandoned)); + } else { + // These are paths where hole punching has been attempted and failed. + must_prune.push(addr.clone()); + } + } + } } - let inactive_paths: HashSet<_> = ip_paths - .difference(&protected_paths) - // cloned here so we can use `paths.retain` later - .map(|&addr| addr.clone()) - .collect(); + // sort the potentially prunable from most recently closed to least recently closed + can_prune.sort_by(|a, b| b.1.cmp(&a.1)); - if inactive_paths.len() < MAX_INACTIVE_IP_ADDRESSES { - return; - } + // Don't prune any potentially usable but inactive paths if we don't need to. + let prunable_slots = MAX_INACTIVE_IP_PATHS.saturating_sub(must_prune.len()); - let now = Instant::now(); + // Prune the "oldest" closed paths. + let prune = can_prune.split_off(can_prune.len().saturating_sub(prunable_slots)); - paths.retain(|addr, state| { - if inactive_paths.contains(addr) { - keep_path(state, &now) - } else { - // keep all active paths - true - } - }); -} + // collect all the paths that should be pruned + let must_prune: HashSet<_> = must_prune + .into_iter() + .chain(prune.into_iter().map(|(addr, _)| addr)) + .collect(); -/// Based on the [`PathState`], returns true if we should keep this path. -/// -/// Currently we have four criteria: -/// 1) This path has successfully holepunched, ever -/// 2) We have never attempted to holepunch on this path -/// 3) We are in the process of holepunching -/// 4) The last time we learned about this address was greater than LAST_SOURCE_PRUNE_DURATION -/// -/// In other words, paths that have never successfully holepunched, that we learned about -/// over `LAST_SOURCE_PRUNE_DURATION` ago, should not be kept. -fn keep_path(state: &PathState, now: &Instant) -> bool { - // if we have never successfully holepunched - state.holepunched - || state.ping_sent.is_none() - || state.ping_in_process(now) - || state - .sources - .values() - // only keep it if this path contains recent sources - .any(|instant| *instant + LAST_SOURCE_PRUNE_DURATION > *now) + paths.retain(|addr, _| !must_prune.contains(addr)); } #[cfg(test)] mod tests { - use std::{ - collections::VecDeque, - net::{Ipv4Addr, SocketAddr}, - }; + use std::net::{Ipv4Addr, SocketAddrV4}; + use std::time::Duration; - use n0_error::Result; - use n0_future::time::{Duration, Instant}; - use rustc_hash::FxHashMap; + use iroh_base::{RelayUrl, SecretKey}; + use rand::SeedableRng; use super::*; - use crate::magicsock::{remote_map::Private, transports}; - - /// Create a test IP address with specific port - fn test_ip_addr(port: u16) -> transports::Addr { - transports::Addr::Ip(SocketAddr::new( - std::net::IpAddr::V4(Ipv4Addr::LOCALHOST), - port, - )) + + fn ip_addr(port: u16) -> transports::Addr { + transports::Addr::Ip(SocketAddrV4::new(Ipv4Addr::LOCALHOST, port).into()) } - /// Create a PathState with sources at a specific time offset - fn test_path_state(time_offset: Duration, sent_ping: Option) -> PathState { - let mut state = PathState::default(); - if let Some(sent_ping_ago) = sent_ping { - state.ping_sent = Some(Instant::now() - sent_ping_ago); + fn path_state_usable_abandoned(abandoned: Instant) -> PathState { + PathState { + sources: HashMap::new(), + usable: true, + abandoned: Some(abandoned), + } + } + + fn path_state_failed_abandoned(abandoned: Instant) -> PathState { + PathState { + sources: HashMap::new(), + usable: false, + abandoned: Some(abandoned), } - state.sources.insert( - Source::Connection { _0: Private }, - Instant::now() - time_offset, - ); - state } #[test] - fn test_prune_ip_paths_too_few_total_paths() -> Result { - // create fewer than MAX_INACTIVE_IP_ADDRESSES paths + fn test_prune_under_max_paths() { let mut paths = FxHashMap::default(); - for i in 0..15 { - paths.insert( - test_ip_addr(i), - test_path_state(Duration::from_secs(0), None), - ); + for i in 0..20 { + paths.insert(ip_addr(i), PathState::default()); } - let pending = VecDeque::new(); - let selected_path = None; - let open_paths = Vec::new(); + prune_ip_paths(&mut paths); + assert_eq!(paths.len(), 20, "should not prune when under MAX_IP_PATHS"); + } - let initial_len = paths.len(); - // should not prune because we have fewer than MAX_INACTIVE_IP_ADDRESSES paths - prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); - assert_eq!( - paths.len(), - initial_len, - "Expected no paths to be pruned when total IP paths < MAX_INACTIVE_IP_ADDRESSES" - ); + #[test] + fn test_prune_at_max_paths_no_prunable() { + let mut paths = FxHashMap::default(); + // All paths are active (never abandoned), so none should be pruned + for i in 0..MAX_IP_PATHS { + paths.insert(ip_addr(i as u16), PathState::default()); + } - Ok(()) + prune_ip_paths(&mut paths); + assert_eq!(paths.len(), MAX_IP_PATHS, "should not prune active paths"); } #[test] - fn test_prune_ip_paths_too_few_inactive_paths() -> Result { - // create MAX_INACTIVE_IP_ADDRESSES + 5 paths + fn test_prune_failed_holepunch() { let mut paths = FxHashMap::default(); - for i in 0..25 { - paths.insert( - test_ip_addr(i), - test_path_state(Duration::from_secs(0), None), - ); - } + let now = Instant::now(); - // mark 10 of them as "active" by adding them to open_paths - let open_paths: Vec = (0..10).map(test_ip_addr).collect(); + // Add 20 active paths + for i in 0..20 { + paths.insert(ip_addr(i), PathState::default()); + } - let pending = VecDeque::new(); - let selected_path = None; + // Add 15 failed holepunch paths (must_prune) + for i in 20..35 { + paths.insert(ip_addr(i), path_state_failed_abandoned(now)); + } - let initial_len = paths.len(); - // now we have 25 total paths, but only 15 inactive paths (25 - 10 = 15) - // which is less than MAX_INACTIVE_IP_ADDRESSES (20) - prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); - assert_eq!( - paths.len(), - initial_len, - "Expected no paths to be pruned when inactive paths < MAX_INACTIVE_IP_ADDRESSES" - ); + prune_ip_paths(&mut paths); - Ok(()) + // All failed holepunch paths should be pruned + assert_eq!(paths.len(), 20); + for i in 0..20 { + assert!(paths.contains_key(&ip_addr(i))); + } + for i in 20..35 { + assert!(!paths.contains_key(&ip_addr(i))); + } } #[test] - fn test_prune_ip_paths_prunes_old_inactive_paths() -> Result { - // create MAX_INACTIVE_IP_ADDRESSES + 10 paths + fn test_prune_keeps_most_recent_inactive() { let mut paths = FxHashMap::default(); + let now = Instant::now(); - // add 20 paths with recent sources (within 2 minutes) - for i in 0..20 { - paths.insert( - test_ip_addr(i), - test_path_state( - Duration::from_secs(60), // learn about this path 1 min ago - Some(Duration::from_secs(60)), // sent ping 1 min ago - ), - ); + // Add 15 active paths + for i in 0..15 { + paths.insert(ip_addr(i), PathState::default()); } - // add 10 paths with old sources (more than 2 minutes ago) - for i in 20..30 { + // Add 20 usable but abandoned paths with different abandon times + // Ports 15-34, with port 34 being most recently abandoned + for i in 0..20 { + let abandoned_time = now - Duration::from_secs((20 - i) as u64); paths.insert( - test_ip_addr(i), - test_path_state( - Duration::from_secs(180), // learned about this path 3 mins ago - Some(Duration::from_secs(60)), - ), // sent ping 1 min ago + ip_addr(15 + i as u16), + path_state_usable_abandoned(abandoned_time), ); } - let pending = VecDeque::new(); - let selected_path = None; - let open_paths = Vec::new(); + assert_eq!(paths.len(), 35); + prune_ip_paths(&mut paths); - // we have 30 total paths, all inactive - // paths with sources older than LAST_SOURCE_PRUNE_DURATION should be pruned - prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); + // Should keep 15 active + 10 most recently abandoned + assert_eq!(paths.len(), 25); - // we should have kept the 20 recent paths - assert_eq!( - paths.len(), - 20, - "Expected to keep 20 paths with recent sources" - ); + // Active paths should remain + for i in 0..15 { + assert!(paths.contains_key(&ip_addr(i))); + } - // verify that the kept paths are the ones with recent sources - for i in 0..20 { - let addr = test_ip_addr(i); - assert!( - paths.contains_key(&addr), - "Expected to keep path with recent source: {:?}", - addr - ); + // Most recently abandoned (ports 25-34) should remain + for i in 25..35 { + assert!(paths.contains_key(&ip_addr(i)), "port {} should be kept", i); } - // verify that the old paths were removed - for i in 20..30 { - let addr = test_ip_addr(i); + // Oldest abandoned (ports 15-24) should be pruned + for i in 15..25 { assert!( - !paths.contains_key(&addr), - "Expected to prune path with old source: {:?}", - addr + !paths.contains_key(&ip_addr(i)), + "port {} should be pruned", + i ); } - - Ok(()) } #[test] - fn test_prune_ip_paths_protects_selected_and_open_paths() -> Result { - // create MAX_INACTIVE_IP_ADDRESSES + 10 paths, all with old sources + fn test_prune_mixed_must_and_can_prune() { let mut paths = FxHashMap::default(); - for i in 0..30 { + let now = Instant::now(); + + // Add 15 active paths + for i in 0..15 { + paths.insert(ip_addr(i), PathState::default()); + } + + // Add 5 failed holepunch paths + for i in 15..20 { + paths.insert(ip_addr(i), path_state_failed_abandoned(now)); + } + + // Add 15 usable but abandoned paths + for i in 0..15 { + let abandoned_time = now - Duration::from_secs((15 - i) as u64); paths.insert( - test_ip_addr(i), - test_path_state( - Duration::from_secs(180), // learned about this path 3 mins ago - Some(Duration::from_secs(60)), // sent ping 1 min ago - ), + ip_addr(20 + i as u16), + path_state_usable_abandoned(abandoned_time), ); } - // mark path 3 as holepunched - paths.get_mut(&test_ip_addr(3)).unwrap().holepunched = true; - - // mark path 4 as having a recent ping (ping in process) - paths.get_mut(&test_ip_addr(4)).unwrap().ping_sent = Some(( - TransactionId::default(), - Instant::now() - Duration::from_secs(5), - )); - - let pending = VecDeque::new(); - // mark one path as selected - let selected_path = Some(test_ip_addr(0)); - // mark a few paths as open - let open_paths = [test_ip_addr(1), test_ip_addr(2)]; - - prune_ip_paths(&mut paths, &pending, &selected_path, open_paths.iter()); - - // protected paths should still be in the result even though they have old sources - assert!( - paths.contains_key(&test_ip_addr(0)), - "Expected to keep selected path even with old source" - ); - assert!( - paths.contains_key(&test_ip_addr(1)), - "Expected to keep open path even with old source" - ); - assert!( - paths.contains_key(&test_ip_addr(2)), - "Expected to keep open path even with old source" - ); - assert!( - paths.contains_key(&test_ip_addr(3)), - "Expected to keep holepunched path even with old source" - ); - assert!( - paths.contains_key(&test_ip_addr(4)), - "Expected to keep path with ping in process even with old source" - ); - - Ok(()) + assert_eq!(paths.len(), 35); + prune_ip_paths(&mut paths); + + // Total: 15 active + 5 most recent can_prune = 20 + assert_eq!(paths.len(), 20); + + // Active paths should remain + for i in 0..15 { + assert!(paths.contains_key(&ip_addr(i))); + } + + // Failed holepunch should be pruned + for i in 15..20 { + assert!(!paths.contains_key(&ip_addr(i))); + } + + // Most recently abandoned (ports 30-34) should remain + for i in 30..35 { + assert!(paths.contains_key(&ip_addr(i)), "port {} should be kept", i); + } + } + + #[test] + fn test_prune_non_ip_paths_not_counted() { + let mut paths = FxHashMap::default(); + let now = Instant::now(); + + // Add 25 IP paths (under MAX_IP_PATHS) + for i in 0..25 { + paths.insert(ip_addr(i), path_state_failed_abandoned(now)); + } + + let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64); + let relay_url: RelayUrl = url::Url::parse("https://localhost") + .expect("should be valid url") + .into(); + // Add 10 relay addresses + for _ in 0..10 { + let id = SecretKey::generate(&mut rng).public(); + let relay_addr = transports::Addr::Relay(relay_url.clone().into(), id); + paths.insert(relay_addr, PathState::default()); + } + + assert_eq!(paths.len(), 35); // 25 IP + 10 relay + prune_ip_paths(&mut paths); + + // Should not prune since IP paths < MAX_IP_PATHS + assert_eq!(paths.len(), 35); + } + + #[test] + fn test_prune_preserves_never_dialed() { + let mut paths = FxHashMap::default(); + let now = Instant::now(); + + // Add 20 never-dialed paths (abandoned = None, usable = false) + for i in 0..20 { + paths.insert(ip_addr(i), PathState::default()); + } + + // Add 15 failed paths to trigger pruning + for i in 20..35 { + paths.insert(ip_addr(i), path_state_failed_abandoned(now)); + } + + prune_ip_paths(&mut paths); + + // Never-dialed paths should be preserved + for i in 0..20 { + assert!(paths.contains_key(&ip_addr(i))); + } } } From e1e2e4dc590c3bcb8c1178e4e4b7acb87d1aaf90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Sun, 23 Nov 2025 13:20:30 +0100 Subject: [PATCH 28/29] doc and clippy fixes --- .../magicsock/remote_map/remote_state/path_state.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index 6de7c6f0ed..a5d2416fac 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -182,7 +182,7 @@ pub(super) struct PathState { /// /// Always prunes paths that have unsuccessfully holepunched. /// -/// Keeps [`MAX_INACTIVE_PATHS`] of the most recently closed paths +/// Keeps [`MAX_INACTIVE_IP_PATHS`] of the most recently closed paths /// that are not currently being used but have successfully been /// holepunched previously. /// @@ -247,8 +247,10 @@ fn prune_ip_paths(paths: &mut FxHashMap) { #[cfg(test)] mod tests { - use std::net::{Ipv4Addr, SocketAddrV4}; - use std::time::Duration; + use std::{ + net::{Ipv4Addr, SocketAddrV4}, + time::Duration, + }; use iroh_base::{RelayUrl, SecretKey}; use rand::SeedableRng; @@ -434,7 +436,7 @@ mod tests { // Add 10 relay addresses for _ in 0..10 { let id = SecretKey::generate(&mut rng).public(); - let relay_addr = transports::Addr::Relay(relay_url.clone().into(), id); + let relay_addr = transports::Addr::Relay(relay_url.clone(), id); paths.insert(relay_addr, PathState::default()); } From 6a02094e3b8adeec5eb11be3e1eadd72e458aa2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cramfox=E2=80=9D?= <“kasey@n0.computer”> Date: Sun, 23 Nov 2025 13:46:48 +0100 Subject: [PATCH 29/29] fix test --- .../remote_map/remote_state/path_state.rs | 47 ++++++++++--------- 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/iroh/src/magicsock/remote_map/remote_state/path_state.rs b/iroh/src/magicsock/remote_map/remote_state/path_state.rs index a5d2416fac..c44fdd8dcf 100644 --- a/iroh/src/magicsock/remote_map/remote_state/path_state.rs +++ b/iroh/src/magicsock/remote_map/remote_state/path_state.rs @@ -205,9 +205,9 @@ fn prune_ip_paths(paths: &mut FxHashMap) { } // paths that were opened at one point but have previously been closed - let mut can_prune = Vec::new(); + let mut inactive = Vec::new(); // paths where we attempted hole punching but it not successful - let mut must_prune = Vec::new(); + let mut failed = Vec::new(); for (addr, state) in ip_paths { match state.abandoned { @@ -218,28 +218,32 @@ fn prune_ip_paths(paths: &mut FxHashMap) { Some(abandoned) => { if state.usable { // These are paths where hole punching succeeded at one point, but the path was closed. - can_prune.push((addr.clone(), abandoned)); + inactive.push((addr.clone(), abandoned)); } else { // These are paths where hole punching has been attempted and failed. - must_prune.push(addr.clone()); + failed.push(addr.clone()); } } } } - // sort the potentially prunable from most recently closed to least recently closed - can_prune.sort_by(|a, b| b.1.cmp(&a.1)); + // All paths are bad, don't prune all of them. + // + // This implies that `inactive` is empty. + if failed.len() == paths.len() { + failed.truncate(MAX_IP_PATHS); + } - // Don't prune any potentially usable but inactive paths if we don't need to. - let prunable_slots = MAX_INACTIVE_IP_PATHS.saturating_sub(must_prune.len()); + // sort the potentially prunable from most recently closed to least recently closed + inactive.sort_by(|a, b| b.1.cmp(&a.1)); // Prune the "oldest" closed paths. - let prune = can_prune.split_off(can_prune.len().saturating_sub(prunable_slots)); + let old_inactive = inactive.split_off(inactive.len().saturating_sub(MAX_INACTIVE_IP_PATHS)); // collect all the paths that should be pruned - let must_prune: HashSet<_> = must_prune + let must_prune: HashSet<_> = failed .into_iter() - .chain(prune.into_iter().map(|(addr, _)| addr)) + .chain(old_inactive.into_iter().map(|(addr, _)| addr)) .collect(); paths.retain(|addr, _| !must_prune.contains(addr)); @@ -285,7 +289,7 @@ mod tests { } prune_ip_paths(&mut paths); - assert_eq!(paths.len(), 20, "should not prune when under MAX_IP_PATHS"); + assert_eq!(20, paths.len(), "should not prune when under MAX_IP_PATHS"); } #[test] @@ -297,7 +301,7 @@ mod tests { } prune_ip_paths(&mut paths); - assert_eq!(paths.len(), MAX_IP_PATHS, "should not prune active paths"); + assert_eq!(MAX_IP_PATHS, paths.len(), "should not prune active paths"); } #[test] @@ -318,7 +322,7 @@ mod tests { prune_ip_paths(&mut paths); // All failed holepunch paths should be pruned - assert_eq!(paths.len(), 20); + assert_eq!(20, paths.len()); for i in 0..20 { assert!(paths.contains_key(&ip_addr(i))); } @@ -347,11 +351,11 @@ mod tests { ); } - assert_eq!(paths.len(), 35); + assert_eq!(35, paths.len()); prune_ip_paths(&mut paths); // Should keep 15 active + 10 most recently abandoned - assert_eq!(paths.len(), 25); + assert_eq!(25, paths.len()); // Active paths should remain for i in 0..15 { @@ -397,11 +401,12 @@ mod tests { ); } - assert_eq!(paths.len(), 35); + assert_eq!(35, paths.len()); prune_ip_paths(&mut paths); - // Total: 15 active + 5 most recent can_prune = 20 - assert_eq!(paths.len(), 20); + // Remove all failed paths -> down to 30 + // Keep MAX_INACTIVE_IP_PATHS, eg remove 5 usable but abandoned paths -> down to 20 + assert_eq!(20, paths.len()); // Active paths should remain for i in 0..15 { @@ -440,11 +445,11 @@ mod tests { paths.insert(relay_addr, PathState::default()); } - assert_eq!(paths.len(), 35); // 25 IP + 10 relay + assert_eq!(35, paths.len()); // 25 IP + 10 relay prune_ip_paths(&mut paths); // Should not prune since IP paths < MAX_IP_PATHS - assert_eq!(paths.len(), 35); + assert_eq!(35, paths.len()); } #[test]