Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
26 commits
Select commit Hold shift + click to select a range
8f2aba2
Basic using of QUIC NAT Traversal for holepunching
flub Oct 31, 2025
bbf947a
Do not receive DISCO messages, we no longer need them
flub Oct 31, 2025
697583e
remove disco from endpoint_state.rs
flub Oct 31, 2025
343a533
remove disco from endpoint_map.rs
flub Oct 31, 2025
22b80f1
remove disco from magicsock.rs
flub Oct 31, 2025
ff13fa0
delete disco and key modules, all unused now
flub Oct 31, 2025
77f63b9
delete disco metrics
flub Oct 31, 2025
5c4851a
emit a single metric, so metrics are used
flub Oct 31, 2025
a8d9748
Don't store remote NAT candidates
flub Nov 3, 2025
68e673b
Clean up connections at an interval
flub Nov 3, 2025
b8fc574
Merge branch 'feat-multipath' into feat-multipath-quic-nat
flub Nov 17, 2025
ce227f5
compile against protocol-simplification branch
flub Nov 18, 2025
b059bab
updage api
flub Nov 20, 2025
11d9731
Merge branch 'feat-multipath' into feat-multipath-quic-nat
flub Nov 20, 2025
a93db87
some intermediate stuff
flub Nov 20, 2025
1fc0714
hook up adding addresses
flub Nov 21, 2025
e24b41d
properly patch in quinn git dependency and clippy
dignifiedquire Nov 23, 2025
75fe396
Merge remote-tracking branch 'origin/feat-multipath' into feat-qnt-me…
dignifiedquire Nov 23, 2025
e3bb795
Merge pull request #3697 from n0-computer/feat-qnt-merge-main
dignifiedquire Nov 23, 2025
08274f1
Make sure to use canonical addrs when comparing hp rounds
flub Nov 23, 2025
f12341d
bump quinn
flub Nov 23, 2025
f0cdffe
replace trace log with event
flub Nov 23, 2025
9097dc8
improve select path logic
flub Nov 23, 2025
709b80f
bump quinn
flub Nov 23, 2025
fffa4af
deps: cargo update
Frando Nov 23, 2025
7101d19
cleanup unused deps
dignifiedquire Nov 23, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
Basic using of QUIC NAT Traversal for holepunching
  • Loading branch information
flub committed Oct 31, 2025
commit 8f2aba2ea777f3dba8e9decbfc82aeff0ab8b3e3
228 changes: 100 additions & 128 deletions iroh/src/magicsock/endpoint_map/endpoint_state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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.
// ///
Expand Down Expand Up @@ -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<Pin<Box<dyn Stream<Item = (ConnId, Vec<SocketAddr>)> + 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
Expand Down Expand Up @@ -109,6 +121,8 @@ pub(super) struct EndpointStateActor {
connections: FxHashMap<ConnId, ConnectionState>,
/// 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.
//
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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(),
Expand Down Expand Up @@ -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<SocketAddr> = self.remote_hp_addrs();
let local_addrs: BTreeSet<SocketAddr> = self
};
let remote_candidates = BTreeSet::from_iter(conn.nat_candidates());
let local_candidates: BTreeSet<SocketAddr> = 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();
Expand All @@ -592,101 +613,36 @@ impl EndpointStateActor {
}
}

self.do_holepunching().await;
}

/// Returns the remote addresses to holepunch against.
fn remote_hp_addrs(&self) -> BTreeSet<SocketAddr> {
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<SocketAddr> = 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<SocketAddr> = 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::<BTreeSet<_>>();
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.
Expand Down Expand Up @@ -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<SocketAddr>) {
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) {
Expand Down Expand Up @@ -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<SocketAddr>,
local_candidates: BTreeSet<SocketAddr>,
/// The set of remote addresses which could take part in holepunching.
///
/// Like `local_addrs` we may not have used them.
remote_addrs: BTreeSet<SocketAddr>,
/// Like [`Self::local_candidates`] we may not have used them.
remote_candidates: BTreeSet<SocketAddr>,
}

/// The type of connection we have to the endpoint.
Expand Down Expand Up @@ -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.
Expand All @@ -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<HashMap<TransportAddr, PathInfo>>,
/// NAT candidate addresses we received from the server.
///
/// If we are the server for this connection, this will remain empty.
nat_candidates: Vec<SocketAddr>,
/// The paths that exist on this connection.
///
/// This could be in any state, e.g. while still validating the path or already closed
Expand Down
2 changes: 1 addition & 1 deletion iroh/src/magicsock/endpoint_map/path_state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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<Source, Instant>,
/// The last ping sent on this path.
/// The last DISCO ping sent on this path.
pub(super) ping_sent: Option<TransactionId>,
}