iroh/magicsock/node_map/
node_state.rs

1use std::{
2    collections::{BTreeSet, HashMap, btree_map::Entry},
3    hash::Hash,
4    net::{IpAddr, SocketAddr},
5    sync::atomic::AtomicBool,
6};
7
8use data_encoding::HEXLOWER;
9use iroh_base::{NodeAddr, NodeId, PublicKey, RelayUrl};
10use n0_future::{
11    task::{self, AbortOnDropHandle},
12    time::{self, Duration, Instant},
13};
14use n0_watcher::Watchable;
15use serde::{Deserialize, Serialize};
16use tokio::sync::mpsc;
17use tracing::{Level, debug, event, info, instrument, trace, warn};
18
19use super::{
20    IpPort, Source,
21    path_state::{PathState, summarize_node_paths},
22    udp_paths::{NodeUdpPaths, UdpSendAddr},
23};
24#[cfg(any(test, feature = "test-utils"))]
25use crate::endpoint::PathSelection;
26use crate::{
27    disco::{self, SendAddr},
28    magicsock::{
29        ActorMessage, HEARTBEAT_INTERVAL, MagicsockMetrics, NodeIdMappedAddr,
30        node_map::path_validity::PathValidity,
31    },
32};
33
34/// Number of addresses that are not active that we keep around per node.
35///
36/// See [`NodeState::prune_direct_addresses`].
37pub(super) const MAX_INACTIVE_DIRECT_ADDRESSES: usize = 20;
38
39/// How long since an endpoint path was last alive before it might be pruned.
40const LAST_ALIVE_PRUNE_DURATION: Duration = Duration::from_secs(120);
41
42/// How long we wait for a pong reply before assuming it's never coming.
43const PING_TIMEOUT_DURATION: Duration = Duration::from_secs(5);
44
45/// The latency at or under which we don't try to upgrade to a better path.
46const GOOD_ENOUGH_LATENCY: Duration = Duration::from_millis(5);
47
48/// How long since the last activity we try to keep an established endpoint peering alive.
49/// It's also the idle time at which we stop doing QAD queries to keep NAT mappings alive.
50pub(super) const SESSION_ACTIVE_TIMEOUT: Duration = Duration::from_secs(45);
51
52/// How often we try to upgrade to a better patheven if we have some non-relay route that works.
53const UPGRADE_INTERVAL: Duration = Duration::from_secs(60);
54
55/// How long until we send a stayin alive ping
56const STAYIN_ALIVE_MIN_ELAPSED: Duration = Duration::from_secs(2);
57
58#[derive(Debug)]
59pub(in crate::magicsock) enum PingAction {
60    SendCallMeMaybe {
61        relay_url: RelayUrl,
62        dst_node: NodeId,
63    },
64    SendPing(SendPing),
65}
66
67#[derive(Debug)]
68pub(in crate::magicsock) struct SendPing {
69    pub id: usize,
70    pub dst: SendAddr,
71    pub dst_node: NodeId,
72    pub tx_id: stun_rs::TransactionId,
73    pub purpose: DiscoPingPurpose,
74}
75
76/// Indicating an [`NodeState`] has handled a ping.
77#[derive(Debug)]
78pub struct PingHandled {
79    /// What this ping did to the [`NodeState`].
80    pub role: PingRole,
81    /// Whether the sender path should also be pinged.
82    ///
83    /// This is the case if an [`NodeState`] does not yet have a direct path, i.e. it has no
84    /// best_addr.  In this case we want to ping right back to open the direct path in this
85    /// direction as well.
86    pub needs_ping_back: Option<SendPing>,
87}
88
89#[derive(Debug)]
90pub enum PingRole {
91    Duplicate,
92    NewPath,
93    LikelyHeartbeat,
94    Activate,
95}
96
97/// An iroh node, which we can have connections with.
98///
99/// The whole point of the magicsock is that we can have multiple **paths** to a particular
100/// node.  One of these paths is via the endpoint's home relay node but as we establish a
101/// connection we'll hopefully discover more direct paths.
102#[derive(Debug)]
103pub(super) struct NodeState {
104    /// The ID used as index in the [`NodeMap`].
105    ///
106    /// [`NodeMap`]: super::NodeMap
107    id: usize,
108    /// The UDP address used on the QUIC-layer to address this node.
109    quic_mapped_addr: NodeIdMappedAddr,
110    /// The global identifier for this endpoint.
111    node_id: NodeId,
112    /// The last time we pinged all endpoints.
113    last_full_ping: Option<Instant>,
114    /// The url of relay node that we can relay over to communicate.
115    ///
116    /// The fallback/bootstrap path, if non-zero (non-zero for well-behaved clients).
117    relay_url: Option<(RelayUrl, PathState)>,
118    udp_paths: NodeUdpPaths,
119    sent_pings: HashMap<stun_rs::TransactionId, SentPing>,
120    /// Last time this node was used.
121    ///
122    /// A node is marked as in use when sending datagrams to them, or when having received
123    /// datagrams from it. Regardless of whether the datagrams are payload or DISCO, and whether
124    /// they go via UDP or the relay.
125    ///
126    /// Note that sending datagrams to a node does not mean the node receives them.
127    last_used: Option<Instant>,
128    /// Last time we sent a call-me-maybe.
129    ///
130    /// When we do not have a direct connection and we try to send some data, we will try to
131    /// do a full ping + call-me-maybe.  Usually each side only needs to send one
132    /// call-me-maybe to the other for holes to be punched in both directions however.  So
133    /// we only try and send one per [`HEARTBEAT_INTERVAL`].  Each [`HEARTBEAT_INTERVAL`]
134    /// the [`NodeState::stayin_alive`] function is called, which will trigger new
135    /// call-me-maybe messages as backup.
136    last_call_me_maybe: Option<Instant>,
137    /// The type of connection we have to the node, either direct, relay, mixed, or none.
138    conn_type: Watchable<ConnectionType>,
139    /// Whether the conn_type was ever observed to be `Direct` at some point.
140    ///
141    /// Used for metric reporting.
142    has_been_direct: AtomicBool,
143    /// Configuration for what path selection to use
144    #[cfg(any(test, feature = "test-utils"))]
145    path_selection: PathSelection,
146}
147
148/// Options for creating a new [`NodeState`].
149#[derive(Debug)]
150pub(super) struct Options {
151    pub(super) node_id: NodeId,
152    pub(super) relay_url: Option<RelayUrl>,
153    /// Is this endpoint currently active (sending data)?
154    pub(super) active: bool,
155    pub(super) source: super::Source,
156    #[cfg(any(test, feature = "test-utils"))]
157    pub(super) path_selection: PathSelection,
158}
159
160impl NodeState {
161    pub(super) fn new(id: usize, options: Options) -> Self {
162        let quic_mapped_addr = NodeIdMappedAddr::generate();
163
164        // TODO(frando): I don't think we need to track the `num_relay_conns_added`
165        // metric here. We do so in `Self::addr_for_send`.
166        // if options.relay_url.is_some() {
167        //     // we potentially have a relay connection to the node
168        //     inc!(MagicsockMetrics, num_relay_conns_added);
169        // }
170
171        let now = Instant::now();
172
173        NodeState {
174            id,
175            quic_mapped_addr,
176            node_id: options.node_id,
177            last_full_ping: None,
178            relay_url: options.relay_url.map(|url| {
179                (
180                    url.clone(),
181                    PathState::new(options.node_id, SendAddr::Relay(url), options.source, now),
182                )
183            }),
184            udp_paths: NodeUdpPaths::new(),
185            sent_pings: HashMap::new(),
186            last_used: options.active.then(Instant::now),
187            last_call_me_maybe: None,
188            conn_type: Watchable::new(ConnectionType::None),
189            has_been_direct: AtomicBool::new(false),
190            #[cfg(any(test, feature = "test-utils"))]
191            path_selection: options.path_selection,
192        }
193    }
194
195    pub(super) fn public_key(&self) -> &PublicKey {
196        &self.node_id
197    }
198
199    pub(super) fn quic_mapped_addr(&self) -> &NodeIdMappedAddr {
200        &self.quic_mapped_addr
201    }
202
203    pub(super) fn id(&self) -> usize {
204        self.id
205    }
206
207    pub(super) fn conn_type(&self) -> n0_watcher::Direct<ConnectionType> {
208        self.conn_type.watch()
209    }
210
211    pub(super) fn latency(&self) -> Option<Duration> {
212        match self.conn_type.get() {
213            ConnectionType::Direct(addr) => self
214                .udp_paths
215                .paths()
216                .get(&addr.into())
217                .and_then(|state| state.latency()),
218            ConnectionType::Relay(ref url) => self
219                .relay_url
220                .as_ref()
221                .filter(|(relay_url, _)| relay_url == url)
222                .and_then(|(_, state)| state.latency()),
223            ConnectionType::Mixed(addr, ref url) => {
224                let addr_latency = self
225                    .udp_paths
226                    .paths()
227                    .get(&addr.into())
228                    .and_then(|state| state.latency());
229                let relay_latency = self
230                    .relay_url
231                    .as_ref()
232                    .filter(|(relay_url, _)| relay_url == url)
233                    .and_then(|(_, state)| state.latency());
234                addr_latency.min(relay_latency)
235            }
236            ConnectionType::None => None,
237        }
238    }
239
240    /// Returns info about this node.
241    pub(super) fn info(&self, now: Instant) -> RemoteInfo {
242        let conn_type = self.conn_type.get();
243        let latency = self.latency();
244
245        let addrs = self
246            .udp_paths
247            .paths()
248            .iter()
249            .map(|(addr, path_state)| DirectAddrInfo {
250                addr: SocketAddr::from(*addr),
251                latency: path_state.validity.latency(),
252                last_control: path_state.last_control_msg(now),
253                last_payload: path_state
254                    .last_payload_msg
255                    .as_ref()
256                    .map(|instant| now.duration_since(*instant)),
257                last_alive: path_state
258                    .last_alive()
259                    .map(|instant| now.duration_since(instant)),
260                sources: path_state
261                    .sources
262                    .iter()
263                    .map(|(source, instant)| (source.clone(), now.duration_since(*instant)))
264                    .collect(),
265            })
266            .collect();
267
268        RemoteInfo {
269            node_id: self.node_id,
270            relay_url: self.relay_url.clone().map(|r| r.into()),
271            addrs,
272            conn_type,
273            latency,
274            last_used: self.last_used.map(|instant| now.duration_since(instant)),
275        }
276    }
277
278    /// Returns the relay url of this endpoint
279    pub(super) fn relay_url(&self) -> Option<RelayUrl> {
280        self.relay_url.as_ref().map(|(url, _state)| url.clone())
281    }
282
283    /// Returns the address(es) that should be used for sending the next packet.
284    ///
285    /// This may return to send on one, both or no paths.
286    fn addr_for_send(
287        &self,
288        have_ipv6: bool,
289        metrics: &MagicsockMetrics,
290    ) -> (Option<SocketAddr>, Option<RelayUrl>) {
291        #[cfg(any(test, feature = "test-utils"))]
292        if self.path_selection == PathSelection::RelayOnly {
293            debug!(
294                "in `RelayOnly` mode, giving the relay address as the only viable address for this endpoint"
295            );
296            return (None, self.relay_url());
297        }
298        let (best_addr, relay_url) = match self.udp_paths.send_addr(have_ipv6) {
299            UdpSendAddr::Valid(addr) => {
300                // If we have a valid address we use it.
301                trace!(%addr, ?have_ipv6, "UdpSendAddr is valid, use it");
302                (Some(*addr), None)
303            }
304            UdpSendAddr::Outdated(addr) => {
305                // If the address is outdated we use it, but send via relay at the same time.
306                // We also send disco pings so that it will become valid again if it still
307                // works (i.e. we don't need to holepunch again).
308                trace!(%addr, ?have_ipv6, "UdpSendAddr is outdated, use it together with relay");
309                (Some(*addr), self.relay_url())
310            }
311            UdpSendAddr::Unconfirmed(addr) => {
312                trace!(%addr, ?have_ipv6, "UdpSendAddr is unconfirmed, use it together with relay");
313                (Some(*addr), self.relay_url())
314            }
315            UdpSendAddr::None => {
316                trace!(?have_ipv6, "No UdpSendAddr, use relay");
317                (None, self.relay_url())
318            }
319        };
320        let typ = match (best_addr, relay_url.clone()) {
321            (Some(best_addr), Some(relay_url)) => ConnectionType::Mixed(best_addr, relay_url),
322            (Some(best_addr), None) => ConnectionType::Direct(best_addr),
323            (None, Some(relay_url)) => ConnectionType::Relay(relay_url),
324            (None, None) => ConnectionType::None,
325        };
326        if matches!(&typ, ConnectionType::Direct(_)) {
327            let before = self
328                .has_been_direct
329                .swap(true, std::sync::atomic::Ordering::Relaxed);
330            if !before {
331                metrics.nodes_contacted_directly.inc();
332            }
333        }
334        if let Ok(prev_typ) = self.conn_type.set(typ.clone()) {
335            // The connection type has changed.
336            event!(
337                target: "iroh::_events::conn_type::changed",
338                Level::DEBUG,
339                remote_node = %self.node_id.fmt_short(),
340                conn_type = ?typ,
341            );
342            info!(%typ, "new connection type");
343
344            // Update some metrics
345            match (prev_typ, typ) {
346                (ConnectionType::Relay(_), ConnectionType::Direct(_))
347                | (ConnectionType::Mixed(_, _), ConnectionType::Direct(_)) => {
348                    metrics.num_direct_conns_added.inc();
349                    metrics.num_relay_conns_removed.inc();
350                }
351                (ConnectionType::Direct(_), ConnectionType::Relay(_))
352                | (ConnectionType::Direct(_), ConnectionType::Mixed(_, _)) => {
353                    metrics.num_direct_conns_removed.inc();
354                    metrics.num_relay_conns_added.inc();
355                }
356                (ConnectionType::None, ConnectionType::Direct(_)) => {
357                    metrics.num_direct_conns_added.inc();
358                }
359                (ConnectionType::Direct(_), ConnectionType::None) => {
360                    metrics.num_direct_conns_removed.inc();
361                }
362                (ConnectionType::None, ConnectionType::Relay(_))
363                | (ConnectionType::None, ConnectionType::Mixed(_, _)) => {
364                    metrics.num_relay_conns_added.inc();
365                }
366                (ConnectionType::Relay(_), ConnectionType::None)
367                | (ConnectionType::Mixed(_, _), ConnectionType::None) => {
368                    metrics.num_relay_conns_removed.inc();
369                }
370                _ => (),
371            }
372        }
373        (best_addr, relay_url)
374    }
375
376    /// Removes a direct address for this node.
377    ///
378    /// If this is also the best address, it will be cleared as well.
379    pub(super) fn remove_direct_addr(&mut self, ip_port: &IpPort, now: Instant, why: &'static str) {
380        let Some(state) = self.udp_paths.access_mut(now).paths().remove(ip_port) else {
381            return;
382        };
383
384        match state.last_alive().map(|instant| instant.elapsed()) {
385            Some(last_alive) => debug!(%ip_port, ?last_alive, why, "pruning address"),
386            None => debug!(%ip_port, last_seen=%"never", why, "pruning address"),
387        }
388    }
389
390    /// Whether we need to send another call-me-maybe to the endpoint.
391    ///
392    /// Basically we need to send a call-me-maybe if we need to find a better path.  Maybe
393    /// we only have a relay path, or our path is expired.
394    ///
395    /// When a call-me-maybe message is sent we also need to send pings to all known paths
396    /// of the endpoint.  The [`NodeState::send_call_me_maybe`] function takes care of this.
397    #[cfg(not(wasm_browser))]
398    #[instrument("want_call_me_maybe", skip_all)]
399    fn want_call_me_maybe(&self, now: &Instant, have_ipv6: bool) -> bool {
400        trace!("full ping: wanted?");
401        let Some(last_full_ping) = self.last_full_ping else {
402            debug!("no previous full ping: need full ping");
403            return true;
404        };
405        match &self.udp_paths.send_addr(have_ipv6) {
406            UdpSendAddr::None | UdpSendAddr::Unconfirmed(_) => {
407                debug!("best addr not set: need full ping");
408                true
409            }
410            UdpSendAddr::Outdated(_) => {
411                debug!("best addr expired: need full ping");
412                true
413            }
414            UdpSendAddr::Valid(addr) => {
415                let latency = self
416                    .udp_paths
417                    .paths()
418                    .get(&(*addr).into())
419                    .expect("send path not tracked?")
420                    .latency()
421                    .expect("send_addr marked valid incorrectly");
422                if latency > GOOD_ENOUGH_LATENCY && *now - last_full_ping >= UPGRADE_INTERVAL {
423                    debug!(
424                        "full ping interval expired and latency is only {}ms: need full ping",
425                        latency.as_millis()
426                    );
427                    true
428                } else {
429                    trace!(?now, "best_addr valid: not needed");
430                    false
431                }
432            }
433        }
434    }
435
436    #[cfg(wasm_browser)]
437    fn want_call_me_maybe(&self, _now: &Instant, _have_ipv6: bool) -> bool {
438        trace!("full ping: skipped in browser");
439        false
440    }
441
442    /// Cleanup the expired ping for the passed in txid.
443    #[instrument("disco", skip_all, fields(node = %self.node_id.fmt_short()))]
444    pub(super) fn ping_timeout(
445        &mut self,
446        txid: stun_rs::TransactionId,
447        now: Instant,
448        metrics: &MagicsockMetrics,
449    ) {
450        if let Some(sp) = self.sent_pings.remove(&txid) {
451            debug!(tx = %HEXLOWER.encode(&txid), addr = %sp.to, "pong not received in timeout");
452            match sp.to {
453                SendAddr::Udp(addr) => {
454                    if let Some(path_state) =
455                        self.udp_paths.access_mut(now).paths().get_mut(&addr.into())
456                    {
457                        path_state.last_ping = None;
458                        let consider_alive = path_state
459                            .last_alive()
460                            .map(|last_alive| last_alive.elapsed() <= PING_TIMEOUT_DURATION)
461                            .unwrap_or(false);
462                        if !consider_alive {
463                            // If there was no sign of life from this path during the time
464                            // which we should have received the pong, clear best addr and
465                            // pong.  Both are used to select this path again, but we know
466                            // it's not a usable path now.
467                            path_state.validity = PathValidity::empty();
468                            metrics.path_ping_failures.inc();
469
470                            path_state.validity.record_metrics(metrics);
471                            metrics.path_marked_outdated.inc();
472                        }
473                    }
474                }
475                SendAddr::Relay(ref url) => {
476                    if let Some((home_relay, relay_state)) = self.relay_url.as_mut() {
477                        if home_relay == url {
478                            // lost connectivity via relay
479                            relay_state.last_ping = None;
480                        }
481                    }
482                }
483            }
484        }
485    }
486
487    #[must_use = "pings must be handled"]
488    fn start_ping(&self, dst: SendAddr, purpose: DiscoPingPurpose) -> Option<SendPing> {
489        #[cfg(any(test, feature = "test-utils"))]
490        if self.path_selection == PathSelection::RelayOnly && !dst.is_relay() {
491            // don't attempt any hole punching in relay only mode
492            warn!("in `RelayOnly` mode, ignoring request to start a hole punching attempt.");
493            return None;
494        }
495        #[cfg(wasm_browser)]
496        if !dst.is_relay() {
497            return None; // Similar to `RelayOnly` mode, we don't send UDP pings for hole-punching.
498        }
499
500        let tx_id = stun_rs::TransactionId::default();
501        trace!(tx = %HEXLOWER.encode(&tx_id), %dst, ?purpose,
502               dst = %self.node_id.fmt_short(), "start ping");
503        event!(
504            target: "iroh::_events::ping::sent",
505            Level::DEBUG,
506            remote_node = %self.node_id.fmt_short(),
507            ?dst,
508            txn = ?tx_id,
509            ?purpose,
510        );
511        Some(SendPing {
512            id: self.id,
513            dst,
514            dst_node: self.node_id,
515            tx_id,
516            purpose,
517        })
518    }
519
520    /// Record the fact that a ping has been sent out.
521    pub(super) fn ping_sent(
522        &mut self,
523        to: SendAddr,
524        tx_id: stun_rs::TransactionId,
525        purpose: DiscoPingPurpose,
526        sender: mpsc::Sender<ActorMessage>,
527    ) {
528        trace!(%to, tx = %HEXLOWER.encode(&tx_id), ?purpose, "record ping sent");
529
530        let now = Instant::now();
531        let mut path_found = false;
532        match to {
533            SendAddr::Udp(addr) => {
534                if let Some(st) = self.udp_paths.access_mut(now).paths().get_mut(&addr.into()) {
535                    st.last_ping.replace(now);
536                    st.validity.record_ping_sent();
537                    path_found = true
538                }
539            }
540            SendAddr::Relay(ref url) => {
541                if let Some((home_relay, relay_state)) = self.relay_url.as_mut() {
542                    if home_relay == url {
543                        relay_state.last_ping.replace(now);
544                        path_found = true
545                    }
546                }
547            }
548        }
549        if !path_found {
550            // Shouldn't happen. But don't ping an endpoint that's not active for us.
551            warn!(%to, ?purpose, "unexpected attempt to ping no longer live path");
552            return;
553        }
554
555        let id = self.id;
556        let _expiry_task = AbortOnDropHandle::new(task::spawn(async move {
557            time::sleep(PING_TIMEOUT_DURATION).await;
558            sender
559                .send(ActorMessage::EndpointPingExpired(id, tx_id))
560                .await
561                .ok();
562        }));
563        self.sent_pings.insert(
564            tx_id,
565            SentPing {
566                to,
567                at: now,
568                purpose,
569                _expiry_task,
570            },
571        );
572    }
573
574    /// Send a DISCO call-me-maybe message to the peer.
575    ///
576    /// This takes care of sending the needed pings beforehand.  This ensures that we open
577    /// our firewall's port so that when the receiver sends us DISCO pings in response to
578    /// our call-me-maybe they will reach us and the other side establishes a direct
579    /// connection upon our subsequent pong response.
580    ///
581    /// For [`SendCallMeMaybe::IfNoRecent`], **no** paths will be pinged if there already
582    /// was a recent call-me-maybe sent.
583    ///
584    /// The caller is responsible for sending the messages.
585    #[must_use = "actions must be handled"]
586    fn send_call_me_maybe(&mut self, now: Instant, always: SendCallMeMaybe) -> Vec<PingAction> {
587        match always {
588            SendCallMeMaybe::Always => (),
589            SendCallMeMaybe::IfNoRecent => {
590                let had_recent_call_me_maybe = self
591                    .last_call_me_maybe
592                    .map(|when| when.elapsed() < HEARTBEAT_INTERVAL)
593                    .unwrap_or(false);
594                if had_recent_call_me_maybe {
595                    trace!("skipping call-me-maybe, still recent");
596                    return Vec::new();
597                }
598            }
599        }
600        // We send pings regardless of whether we have a RelayUrl.  If we were given any
601        // direct address paths to contact but no RelayUrl, we still need to send a DISCO
602        // ping to the direct address paths so that the other node will learn about us and
603        // accepts the connection.
604        let mut msgs = self.send_pings(now);
605
606        if let Some(url) = self.relay_url() {
607            debug!(%url, "queue call-me-maybe");
608            msgs.push(PingAction::SendCallMeMaybe {
609                relay_url: url,
610                dst_node: self.node_id,
611            });
612            self.last_call_me_maybe = Some(now);
613        } else {
614            debug!("can not send call-me-maybe, no relay URL");
615        }
616
617        msgs
618    }
619
620    /// Send DISCO Pings to all the paths of this node.
621    ///
622    /// Any paths to the node which have not been recently pinged will be sent a disco
623    /// ping.
624    ///
625    /// The caller is responsible for sending the messages.
626    #[must_use = "actions must be handled"]
627    fn send_pings(&mut self, now: Instant) -> Vec<PingAction> {
628        // We allocate +1 in case the caller wants to add a call-me-maybe message.
629        let mut ping_msgs = Vec::with_capacity(self.udp_paths.paths().len() + 1);
630
631        if let Some((url, state)) = self.relay_url.as_ref() {
632            if state.needs_ping(&now) {
633                debug!(%url, "relay path needs ping");
634                if let Some(msg) =
635                    self.start_ping(SendAddr::Relay(url.clone()), DiscoPingPurpose::Discovery)
636                {
637                    ping_msgs.push(PingAction::SendPing(msg))
638                }
639            }
640        }
641
642        #[cfg(any(test, feature = "test-utils"))]
643        if self.path_selection == PathSelection::RelayOnly {
644            warn!("in `RelayOnly` mode, ignoring request to respond to a hole punching attempt.");
645            return ping_msgs;
646        }
647
648        self.prune_direct_addresses(now);
649        let mut ping_dsts = String::from("[");
650        self.udp_paths
651            .paths()
652            .iter()
653            .filter_map(|(ipp, state)| state.needs_ping(&now).then_some(*ipp))
654            .filter_map(|ipp| {
655                self.start_ping(SendAddr::Udp(ipp.into()), DiscoPingPurpose::Discovery)
656            })
657            .for_each(|msg| {
658                use std::fmt::Write;
659                write!(&mut ping_dsts, " {} ", msg.dst).ok();
660                ping_msgs.push(PingAction::SendPing(msg));
661            });
662        ping_dsts.push(']');
663        debug!(
664            %ping_dsts,
665            dst = %self.node_id.fmt_short(),
666            paths = %summarize_node_paths(self.udp_paths.paths()),
667            "sending pings to node",
668        );
669        self.last_full_ping.replace(now);
670        ping_msgs
671    }
672
673    pub(super) fn update_from_node_addr(
674        &mut self,
675        new_relay_url: Option<&RelayUrl>,
676        new_addrs: &BTreeSet<SocketAddr>,
677        source: super::Source,
678        have_ipv6: bool,
679        metrics: &MagicsockMetrics,
680    ) {
681        if matches!(
682            self.udp_paths.send_addr(have_ipv6),
683            UdpSendAddr::None | UdpSendAddr::Unconfirmed(_)
684        ) {
685            // we do not have a direct connection, so changing the relay information may
686            // have an effect on our connection status
687            if self.relay_url.is_none() && new_relay_url.is_some() {
688                // we did not have a relay connection before, but now we do
689                metrics.num_relay_conns_added.inc();
690            } else if self.relay_url.is_some() && new_relay_url.is_none() {
691                // we had a relay connection before but do not have one now
692                metrics.num_relay_conns_removed.inc();
693            }
694        }
695
696        let now = Instant::now();
697
698        if new_relay_url.is_some() && new_relay_url != self.relay_url().as_ref() {
699            debug!(
700                "Changing relay node from {:?} to {:?}",
701                self.relay_url, new_relay_url
702            );
703            self.relay_url = new_relay_url.map(|url| {
704                (
705                    url.clone(),
706                    PathState::new(self.node_id, url.clone().into(), source.clone(), now),
707                )
708            });
709        }
710
711        let mut access = self.udp_paths.access_mut(now);
712        for &addr in new_addrs.iter() {
713            access
714                .paths()
715                .entry(addr.into())
716                .and_modify(|path_state| {
717                    path_state.add_source(source.clone(), now);
718                })
719                .or_insert_with(|| {
720                    PathState::new(self.node_id, SendAddr::from(addr), source.clone(), now)
721                });
722        }
723        drop(access);
724        let paths = summarize_node_paths(self.udp_paths.paths());
725        debug!(new = ?new_addrs , %paths, "added new direct paths for endpoint");
726    }
727
728    /// Handle a received Disco Ping.
729    ///
730    /// - Ensures the paths the ping was received on is a known path for this endpoint.
731    ///
732    /// - If there is no best_addr for this endpoint yet, sends a ping itself to try and
733    ///   establish one.
734    ///
735    /// This is called once we've already verified that we got a valid discovery message
736    /// from `self` via ep.
737    pub(super) fn handle_ping(
738        &mut self,
739        path: SendAddr,
740        tx_id: stun_rs::TransactionId,
741    ) -> PingHandled {
742        let now = Instant::now();
743
744        let role = match path {
745            SendAddr::Udp(addr) => {
746                match self.udp_paths.access_mut(now).paths().entry(addr.into()) {
747                    Entry::Occupied(mut occupied) => occupied.get_mut().handle_ping(tx_id, now),
748                    Entry::Vacant(vacant) => {
749                        info!(%addr, "new direct addr for node");
750                        vacant.insert(PathState::with_ping(
751                            self.node_id,
752                            path.clone(),
753                            tx_id,
754                            Source::Udp,
755                            now,
756                        ));
757                        PingRole::NewPath
758                    }
759                }
760            }
761            SendAddr::Relay(ref url) => {
762                match self.relay_url.as_mut() {
763                    Some((home_url, _state)) if home_url != url => {
764                        // either the node changed relays or we didn't have a relay address for the
765                        // node. In both cases, trust the new confirmed url
766                        info!(%url, "new relay addr for node");
767                        self.relay_url = Some((
768                            url.clone(),
769                            PathState::with_ping(
770                                self.node_id,
771                                path.clone(),
772                                tx_id,
773                                Source::Relay,
774                                now,
775                            ),
776                        ));
777                        PingRole::NewPath
778                    }
779                    Some((_home_url, state)) => state.handle_ping(tx_id, now),
780                    None => {
781                        info!(%url, "new relay addr for node");
782                        self.relay_url = Some((
783                            url.clone(),
784                            PathState::with_ping(
785                                self.node_id,
786                                path.clone(),
787                                tx_id,
788                                Source::Relay,
789                                now,
790                            ),
791                        ));
792                        PingRole::NewPath
793                    }
794                }
795            }
796        };
797        event!(
798            target: "iroh::_events::ping::recv",
799            Level::DEBUG,
800            remote_node = %self.node_id.fmt_short(),
801            src = ?path,
802            txn = ?tx_id,
803            ?role,
804        );
805
806        if matches!(path, SendAddr::Udp(_)) && matches!(role, PingRole::NewPath) {
807            self.prune_direct_addresses(now);
808        }
809
810        // if the endpoint does not yet have a best_addr
811        let needs_ping_back = if matches!(path, SendAddr::Udp(_))
812            && matches!(
813                self.udp_paths.send_addr(true),
814                UdpSendAddr::None | UdpSendAddr::Unconfirmed(_) | UdpSendAddr::Outdated(_)
815            ) {
816            // We also need to send a ping to make this path available to us as well.  This
817            // is always sent together with a pong.  So in the worst case the pong gets lost
818            // and this ping does not.  In that case we ping-pong until both sides have
819            // received at least one pong.  Once both sides have received one pong they both
820            // have a best_addr and this ping will stop being sent.
821            self.start_ping(path, DiscoPingPurpose::PingBack)
822        } else {
823            None
824        };
825
826        debug!(
827            ?role,
828            needs_ping_back = ?needs_ping_back.is_some(),
829            paths = %summarize_node_paths(self.udp_paths.paths()),
830            "endpoint handled ping",
831        );
832        PingHandled {
833            role,
834            needs_ping_back,
835        }
836    }
837
838    /// Prune inactive paths.
839    ///
840    /// This trims the list of inactive paths for an endpoint.  At most
841    /// [`MAX_INACTIVE_DIRECT_ADDRESSES`] are kept.
842    pub(super) fn prune_direct_addresses(&mut self, now: Instant) {
843        // prune candidates are addresses that are not active
844        let mut prune_candidates: Vec<_> = self
845            .udp_paths
846            .paths()
847            .iter()
848            .filter(|(_ip_port, state)| !state.is_active())
849            .map(|(ip_port, state)| (*ip_port, state.last_alive()))
850            .filter(|(_ipp, last_alive)| match last_alive {
851                Some(last_seen) => last_seen.elapsed() > LAST_ALIVE_PRUNE_DURATION,
852                None => true,
853            })
854            .collect();
855        let prune_count = prune_candidates
856            .len()
857            .saturating_sub(MAX_INACTIVE_DIRECT_ADDRESSES);
858        if prune_count == 0 {
859            // nothing to do, within limits
860            debug!(
861                paths = %summarize_node_paths(self.udp_paths.paths()),
862                "prune addresses: {prune_count} pruned",
863            );
864            return;
865        }
866
867        // sort leaving the worst addresses first (never contacted) and better ones (most recently
868        // used ones) last
869        prune_candidates.sort_unstable_by_key(|(_ip_port, last_alive)| *last_alive);
870        prune_candidates.truncate(prune_count);
871        for (ip_port, _last_alive) in prune_candidates.into_iter() {
872            self.remove_direct_addr(&ip_port, now, "inactive");
873        }
874        debug!(
875            paths = %summarize_node_paths(self.udp_paths.paths()),
876            "prune addresses: {prune_count} pruned",
877        );
878    }
879
880    /// Called when connectivity changes enough that we should question our earlier
881    /// assumptions about which paths work.
882    #[instrument("disco", skip_all, fields(node = %self.node_id.fmt_short()))]
883    pub(super) fn note_connectivity_change(&mut self, now: Instant, metrics: &MagicsockMetrics) {
884        let mut guard = self.udp_paths.access_mut(now);
885        for es in guard.paths().values_mut() {
886            es.validity.record_metrics(metrics);
887            es.clear();
888        }
889    }
890
891    /// Handles a Pong message (a reply to an earlier ping).
892    ///
893    /// It reports the address and key that should be inserted for the endpoint if any.
894    #[instrument(skip(self, metrics))]
895    pub(super) fn handle_pong(
896        &mut self,
897        m: &disco::Pong,
898        src: SendAddr,
899        metrics: &MagicsockMetrics,
900    ) -> Option<(SocketAddr, PublicKey)> {
901        event!(
902            target: "iroh::_events::pong::recv",
903            Level::DEBUG,
904            remote_node = %self.node_id.fmt_short(),
905            ?src,
906            txn = ?m.tx_id,
907        );
908        let is_relay = src.is_relay();
909        match self.sent_pings.remove(&m.tx_id) {
910            None => {
911                // This is not a pong for a ping we sent.  In reality however we probably
912                // did send this ping but it has timed-out by the time we receive this pong
913                // so we removed the state already.
914                debug!(tx = %HEXLOWER.encode(&m.tx_id), "received unknown pong (did it timeout?)");
915                None
916            }
917            Some(sp) => {
918                let mut node_map_insert = None;
919
920                let now = Instant::now();
921                let latency = now - sp.at;
922
923                debug!(
924                    tx = %HEXLOWER.encode(&m.tx_id),
925                    src = %src,
926                    reported_ping_src = %m.ping_observed_addr,
927                    ping_dst = %sp.to,
928                    is_relay = %src.is_relay(),
929                    latency = %latency.as_millis(),
930                    "received pong",
931                );
932
933                match src {
934                    SendAddr::Udp(addr) => {
935                        match self.udp_paths.access_mut(now).paths().get_mut(&addr.into()) {
936                            None => {
937                                warn!("ignoring pong: no state for src addr");
938                                // This is no longer an endpoint we care about.
939                                return node_map_insert;
940                            }
941                            Some(st) => {
942                                node_map_insert = Some((addr, self.node_id));
943                                st.add_pong_reply(
944                                    PongReply {
945                                        latency,
946                                        pong_at: now,
947                                        from: src,
948                                        pong_src: m.ping_observed_addr.clone(),
949                                    },
950                                    metrics,
951                                );
952                            }
953                        }
954                        debug!(
955                            paths = %summarize_node_paths(self.udp_paths.paths()),
956                            "handled pong",
957                        );
958                    }
959                    SendAddr::Relay(ref url) => match self.relay_url.as_mut() {
960                        Some((home_url, state)) if home_url == url => {
961                            state.add_pong_reply(
962                                PongReply {
963                                    latency,
964                                    pong_at: now,
965                                    from: src,
966                                    pong_src: m.ping_observed_addr.clone(),
967                                },
968                                metrics,
969                            );
970                        }
971                        other => {
972                            // if we are here then we sent this ping, but the url changed
973                            // waiting for the response. It was either set to None or changed to
974                            // another relay. This should either never happen or be extremely
975                            // unlikely. Log and ignore for now
976                            warn!(
977                                stored=?other,
978                                received=?url,
979                                "ignoring pong via relay for different relay from last one",
980                            );
981                        }
982                    },
983                }
984
985                // Promote this pong response to our current best address if it's lower latency.
986                // TODO(bradfitz): decide how latency vs. preference order affects decision
987                if let SendAddr::Udp(_to) = sp.to {
988                    debug_assert!(!is_relay, "mismatching relay & udp");
989                }
990
991                node_map_insert
992            }
993        }
994    }
995
996    /// Handles a DISCO CallMeMaybe discovery message.
997    ///
998    /// The contract for use of this message is that the node has already pinged to us via
999    /// UDP, so their stateful firewall should be open. Now we can Ping back and make it
1000    /// through.
1001    ///
1002    /// However if the remote side has no direct path information to us, they would not have
1003    /// had any [`IpPort`]s to send pings to and our pings might end up blocked.  But at
1004    /// least open the firewalls on our side, giving the other side another change of making
1005    /// it through when it pings in response.
1006    pub(super) fn handle_call_me_maybe(
1007        &mut self,
1008        m: disco::CallMeMaybe,
1009        metrics: &MagicsockMetrics,
1010    ) -> Vec<PingAction> {
1011        let now = Instant::now();
1012        let mut call_me_maybe_ipps = BTreeSet::new();
1013
1014        let mut guard = self.udp_paths.access_mut(now);
1015
1016        for peer_sockaddr in &m.my_numbers {
1017            if let IpAddr::V6(ip) = peer_sockaddr.ip() {
1018                if netwatch::ip::is_unicast_link_local(ip) {
1019                    // We send these out, but ignore them for now.
1020                    // TODO: teach the ping code to ping on all interfaces for these.
1021                    continue;
1022                }
1023            }
1024            let ipp = IpPort::from(*peer_sockaddr);
1025            call_me_maybe_ipps.insert(ipp);
1026            guard
1027                .paths()
1028                .entry(ipp)
1029                .or_insert_with(|| {
1030                    PathState::new(
1031                        self.node_id,
1032                        SendAddr::from(*peer_sockaddr),
1033                        Source::Relay,
1034                        now,
1035                    )
1036                })
1037                .call_me_maybe_time
1038                .replace(now);
1039        }
1040
1041        // Zero out all the last_ping times to force send_pings to send new ones, even if
1042        // it's been less than 5 seconds ago.  Also clear pongs for direct addresses not
1043        // included in the updated set.
1044        for (ipp, st) in guard.paths().iter_mut() {
1045            st.last_ping = None;
1046            if !call_me_maybe_ipps.contains(ipp) {
1047                // TODO: This seems like a weird way to signal that the endpoint no longer
1048                // thinks it has this IpPort as an available path.
1049                if !st.validity.is_empty() {
1050                    debug!(path=?ipp ,"clearing recent pong");
1051                    st.validity.record_metrics(metrics);
1052                    st.validity = PathValidity::empty();
1053                }
1054            }
1055        }
1056        if guard.has_best_addr_changed() {
1057            // Clear the last call-me-maybe send time so we will send one again.
1058            self.last_call_me_maybe = None;
1059        }
1060        debug!(
1061            paths = %summarize_node_paths(self.udp_paths.paths()),
1062            "updated endpoint paths from call-me-maybe",
1063        );
1064        self.send_pings(now)
1065    }
1066
1067    /// Marks this node as having received a UDP payload message.
1068    #[cfg(not(wasm_browser))]
1069    pub(super) fn receive_udp(&mut self, addr: IpPort, now: Instant) {
1070        let mut guard = self.udp_paths.access_mut(now);
1071        let Some(state) = guard.paths().get_mut(&addr) else {
1072            debug_assert!(false, "node map inconsistency by_ip_port <-> direct addr");
1073            return;
1074        };
1075        state.receive_payload(now);
1076        self.last_used = Some(now);
1077    }
1078
1079    pub(super) fn receive_relay(&mut self, url: &RelayUrl, src: NodeId, now: Instant) {
1080        match self.relay_url.as_mut() {
1081            Some((current_home, state)) if current_home == url => {
1082                // We received on the expected url. update state.
1083                state.receive_payload(now);
1084            }
1085            Some((_current_home, _state)) => {
1086                // we have a different url. we only update on ping, not on receive_relay.
1087            }
1088            None => {
1089                self.relay_url = Some((
1090                    url.clone(),
1091                    PathState::with_last_payload(
1092                        src,
1093                        SendAddr::from(url.clone()),
1094                        Source::Relay,
1095                        now,
1096                    ),
1097                ));
1098            }
1099        }
1100        self.last_used = Some(now);
1101    }
1102
1103    pub(super) fn last_ping(&self, addr: &SendAddr) -> Option<Instant> {
1104        match addr {
1105            SendAddr::Udp(addr) => self
1106                .udp_paths
1107                .paths()
1108                .get(&(*addr).into())
1109                .and_then(|ep| ep.last_ping),
1110            SendAddr::Relay(url) => self
1111                .relay_url
1112                .as_ref()
1113                .filter(|(home_url, _state)| home_url == url)
1114                .and_then(|(_home_url, state)| state.last_ping),
1115        }
1116    }
1117
1118    /// Checks if this `Endpoint` is currently actively being used.
1119    pub(super) fn is_active(&self, now: &Instant) -> bool {
1120        match self.last_used {
1121            Some(last_active) => now.duration_since(last_active) <= SESSION_ACTIVE_TIMEOUT,
1122            None => false,
1123        }
1124    }
1125
1126    /// Send a heartbeat to the node to keep the connection alive, or trigger a full ping
1127    /// if necessary.
1128    #[instrument("stayin_alive", skip_all, fields(node = %self.node_id.fmt_short()))]
1129    pub(super) fn stayin_alive(&mut self, have_ipv6: bool) -> Vec<PingAction> {
1130        trace!("stayin_alive");
1131        let now = Instant::now();
1132        if !self.is_active(&now) {
1133            trace!("skipping stayin alive: session is inactive");
1134            return Vec::new();
1135        }
1136
1137        // If we do not have an optimal addr, send pings to all known places.
1138        if self.want_call_me_maybe(&now, have_ipv6) {
1139            debug!("sending a call-me-maybe");
1140            return self.send_call_me_maybe(now, SendCallMeMaybe::Always);
1141        }
1142
1143        // Send heartbeat ping to keep the current addr going as long as we need it.
1144        if let Some(udp_addr) = self.udp_paths.send_addr(have_ipv6).get_addr() {
1145            let elapsed = self.last_ping(&SendAddr::Udp(udp_addr)).map(|l| now - l);
1146            // Send a ping if the last ping is older than 2 seconds.
1147            let needs_ping = match elapsed {
1148                Some(e) => e >= STAYIN_ALIVE_MIN_ELAPSED,
1149                None => false,
1150            };
1151
1152            if needs_ping {
1153                debug!(
1154                    dst = %udp_addr,
1155                    since_last_ping=?elapsed,
1156                    "send stayin alive ping",
1157                );
1158                if let Some(msg) =
1159                    self.start_ping(SendAddr::Udp(udp_addr), DiscoPingPurpose::StayinAlive)
1160                {
1161                    return vec![PingAction::SendPing(msg)];
1162                }
1163            }
1164        }
1165
1166        Vec::new()
1167    }
1168
1169    /// Returns the addresses on which a payload should be sent right now.
1170    ///
1171    /// This is in the hot path of `.poll_send()`.
1172    // TODO(matheus23): Make this take &self. That's not quite possible yet due to `send_call_me_maybe`
1173    // eventually calling `prune_direct_addresses` (which needs &mut self)
1174    #[instrument("get_send_addrs", skip_all, fields(node = %self.node_id.fmt_short()))]
1175    pub(crate) fn get_send_addrs(
1176        &mut self,
1177        have_ipv6: bool,
1178        metrics: &MagicsockMetrics,
1179    ) -> (Option<SocketAddr>, Option<RelayUrl>, Vec<PingAction>) {
1180        let now = Instant::now();
1181        let prev = self.last_used.replace(now);
1182        if prev.is_none() {
1183            // this is the first time we are trying to connect to this node
1184            metrics.nodes_contacted.inc();
1185        }
1186        let (udp_addr, relay_url) = self.addr_for_send(have_ipv6, metrics);
1187
1188        let ping_msgs = if self.want_call_me_maybe(&now, have_ipv6) {
1189            self.send_call_me_maybe(now, SendCallMeMaybe::IfNoRecent)
1190        } else {
1191            Vec::new()
1192        };
1193        trace!(
1194            ?udp_addr,
1195            ?relay_url,
1196            pings = %ping_msgs.len(),
1197            "found send address",
1198        );
1199        (udp_addr, relay_url, ping_msgs)
1200    }
1201
1202    /// Get the direct addresses for this endpoint.
1203    pub(super) fn direct_addresses(&self) -> impl Iterator<Item = IpPort> + '_ {
1204        self.udp_paths.paths().keys().copied()
1205    }
1206
1207    #[cfg(test)]
1208    pub(super) fn direct_address_states(&self) -> impl Iterator<Item = (&IpPort, &PathState)> + '_ {
1209        self.udp_paths.paths().iter()
1210    }
1211
1212    pub(super) fn last_used(&self) -> Option<Instant> {
1213        self.last_used
1214    }
1215}
1216
1217impl From<RemoteInfo> for NodeAddr {
1218    fn from(info: RemoteInfo) -> Self {
1219        let direct_addresses = info
1220            .addrs
1221            .into_iter()
1222            .map(|info| info.addr)
1223            .collect::<BTreeSet<_>>();
1224
1225        NodeAddr {
1226            node_id: info.node_id,
1227            relay_url: info.relay_url.map(Into::into),
1228            direct_addresses,
1229        }
1230    }
1231}
1232
1233/// Whether to send a call-me-maybe message after sending pings to all known paths.
1234///
1235/// `IfNoRecent` will only send a call-me-maybe if no previous one was sent in the last
1236/// [`HEARTBEAT_INTERVAL`].
1237#[derive(Debug)]
1238enum SendCallMeMaybe {
1239    Always,
1240    IfNoRecent,
1241}
1242
1243#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1244pub(super) struct PongReply {
1245    pub(super) latency: Duration,
1246    /// When we received the pong.
1247    pub(super) pong_at: Instant,
1248    /// The pong's src (usually same as endpoint map key).
1249    pub(super) from: SendAddr,
1250    /// What they reported they heard.
1251    pub(super) pong_src: SendAddr,
1252}
1253
1254#[derive(Debug)]
1255pub(super) struct SentPing {
1256    pub(super) to: SendAddr,
1257    pub(super) at: Instant,
1258    #[allow(dead_code)]
1259    pub(super) purpose: DiscoPingPurpose,
1260    pub(super) _expiry_task: AbortOnDropHandle<()>,
1261}
1262
1263/// The reason why a discovery ping message was sent.
1264#[derive(Debug, Clone, Copy, PartialEq, Eq)]
1265pub enum DiscoPingPurpose {
1266    /// The purpose of a ping was to see if a path was valid.
1267    Discovery,
1268    /// Ping to ensure the current route is still valid.
1269    StayinAlive,
1270    /// When a ping was received and no direct connection exists yet.
1271    ///
1272    /// When a ping was received we suspect a direct connection is possible.  If we do not
1273    /// yet have one that triggers a ping, indicated with this reason.
1274    PingBack,
1275}
1276
1277/// The type of control message we have received.
1278#[derive(Debug, Clone, Copy, Eq, PartialEq, Serialize, Deserialize, derive_more::Display)]
1279pub enum ControlMsg {
1280    /// We received a Ping from the node.
1281    #[display("ping←")]
1282    Ping,
1283    /// We received a Pong from the node.
1284    #[display("pong←")]
1285    Pong,
1286    /// We received a CallMeMaybe.
1287    #[display("call me")]
1288    CallMeMaybe,
1289}
1290
1291/// Information about a *direct address*.
1292///
1293/// The *direct addresses* of an iroh node are those that could be used by other nodes to
1294/// establish direct connectivity, depending on the network situation. Due to NAT configurations,
1295/// for example, not all direct addresses of a node are usable by all peers.
1296#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
1297pub struct DirectAddrInfo {
1298    /// The UDP address reported by the remote node.
1299    pub addr: SocketAddr,
1300    /// The latency to the remote node over this network path.
1301    ///
1302    /// If there has never been any connectivity via this address no latency will be known.
1303    pub latency: Option<Duration>,
1304    /// Last control message received by this node about this address.
1305    ///
1306    /// This contains the elapsed duration since the control message was received and the
1307    /// kind of control message received at that time.  Only the most recent control message
1308    /// is returned.
1309    ///
1310    /// Note that [`ControlMsg::CallMeMaybe`] is received via a relay path, while
1311    /// [`ControlMsg::Ping`] and [`ControlMsg::Pong`] are received on the path to
1312    /// [`DirectAddrInfo::addr`] itself and thus convey very different information.
1313    pub last_control: Option<(Duration, ControlMsg)>,
1314    /// Elapsed time since the last payload message was received on this network path.
1315    ///
1316    /// This indicates how long ago a QUIC datagram was received from the remote node sent
1317    /// from this [`DirectAddrInfo::addr`].  It indicates the network path was in use to
1318    /// transport payload data.
1319    pub last_payload: Option<Duration>,
1320    /// Elapsed time since this network path was known to exist.
1321    ///
1322    /// A network path is considered to exist only because the remote node advertised it.
1323    /// It may not mean the path is usable.  However, if there was any communication with
1324    /// the remote node over this network path it also means the path exists.
1325    ///
1326    /// The elapsed time since *any* confirmation of the path's existence was received is
1327    /// returned.  If the remote node moved networks and no longer has this path, this could
1328    /// be a long duration.
1329    pub last_alive: Option<Duration>,
1330    /// A [`HashMap`] of [`Source`]s to [`Duration`]s.
1331    ///
1332    /// The [`Duration`] indicates the elapsed time since this source last
1333    /// recorded this address.
1334    ///
1335    /// The [`Duration`] will always indicate the most recent time the source
1336    /// recorded this address.
1337    pub sources: HashMap<Source, Duration>,
1338}
1339
1340/// Information about the network path to a remote node via a relay server.
1341#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
1342pub struct RelayUrlInfo {
1343    /// The relay URL.
1344    pub relay_url: RelayUrl,
1345    /// Elapsed time since this relay path last received payload or control data.
1346    pub last_alive: Option<Duration>,
1347    /// Latency to the remote node over this relayed network path.
1348    pub latency: Option<Duration>,
1349}
1350
1351impl From<(RelayUrl, PathState)> for RelayUrlInfo {
1352    fn from(value: (RelayUrl, PathState)) -> Self {
1353        RelayUrlInfo {
1354            relay_url: value.0,
1355            last_alive: value.1.last_alive().map(|i| i.elapsed()),
1356            latency: value.1.latency(),
1357        }
1358    }
1359}
1360
1361impl From<RelayUrlInfo> for RelayUrl {
1362    fn from(value: RelayUrlInfo) -> Self {
1363        value.relay_url
1364    }
1365}
1366
1367/// Details about a remote iroh node which is known to this node.
1368///
1369/// Having details of a node does not mean it can be connected to, nor that it has ever been
1370/// connected to in the past. There are various reasons a node might be known: it could have
1371/// been manually added via [`Endpoint::add_node_addr`], it could have been added by some
1372/// discovery mechanism, the node could have contacted this node, etc.
1373///
1374/// [`Endpoint::add_node_addr`]: crate::endpoint::Endpoint::add_node_addr
1375#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
1376pub(crate) struct RemoteInfo {
1377    /// The globally unique identifier for this node.
1378    pub node_id: NodeId,
1379    /// Relay server information, if available.
1380    pub relay_url: Option<RelayUrlInfo>,
1381    /// The addresses at which this node might be reachable.
1382    ///
1383    /// Some of these addresses might only be valid for networks we are not part of, but the remote
1384    /// node might be a part of.
1385    pub addrs: Vec<DirectAddrInfo>,
1386    /// The type of connection we have to the node, either direct or over relay.
1387    pub conn_type: ConnectionType,
1388    /// The latency of the current network path to the remote node.
1389    pub latency: Option<Duration>,
1390    /// Time elapsed time since last we have sent to or received from the node.
1391    ///
1392    /// This is the duration since *any* data (payload or control messages) was sent or receive
1393    /// from the remote node. Note that sending to the remote node does not imply
1394    /// the remote node received anything.
1395    pub last_used: Option<Duration>,
1396}
1397
1398impl RemoteInfo {
1399    /// Get the duration since the last activity we received from this endpoint
1400    /// on any of its direct addresses.
1401    pub(crate) fn last_received(&self) -> Option<Duration> {
1402        self.addrs
1403            .iter()
1404            .filter_map(|addr| addr.last_control.map(|x| x.0).min(addr.last_payload))
1405            .min()
1406    }
1407
1408    /// Whether there is a possible known network path to the remote node.
1409    ///
1410    /// Note that this does not provide any guarantees of whether any network path is
1411    /// usable.
1412    pub(crate) fn has_send_address(&self) -> bool {
1413        self.relay_url.is_some() || !self.addrs.is_empty()
1414    }
1415}
1416
1417/// The type of connection we have to the endpoint.
1418#[derive(derive_more::Display, Default, Debug, Clone, Eq, PartialEq, Serialize, Deserialize)]
1419pub enum ConnectionType {
1420    /// Direct UDP connection
1421    #[display("direct({_0})")]
1422    Direct(SocketAddr),
1423    /// Relay connection over relay
1424    #[display("relay({_0})")]
1425    Relay(RelayUrl),
1426    /// Both a UDP and a relay connection are used.
1427    ///
1428    /// This is the case if we do have a UDP address, but are missing a recent confirmation that
1429    /// the address works.
1430    #[display("mixed(udp: {_0}, relay: {_1})")]
1431    Mixed(SocketAddr, RelayUrl),
1432    /// We have no verified connection to this PublicKey
1433    #[default]
1434    #[display("none")]
1435    None,
1436}
1437
1438#[cfg(test)]
1439mod tests {
1440    use std::{collections::BTreeMap, net::Ipv4Addr};
1441
1442    use iroh_base::SecretKey;
1443    use rand::SeedableRng;
1444
1445    use super::*;
1446    use crate::magicsock::node_map::{NodeMap, NodeMapInner};
1447
1448    #[test]
1449    fn test_remote_infos() {
1450        let now = Instant::now();
1451        let elapsed = Duration::from_secs(3);
1452        let later = now + elapsed;
1453        let send_addr: RelayUrl = "https://my-relay.com".parse().unwrap();
1454        let pong_src = SendAddr::Udp("0.0.0.0:1".parse().unwrap());
1455        let latency = Duration::from_millis(50);
1456
1457        let relay_and_state = |node_id: NodeId, url: RelayUrl| {
1458            let relay_state = PathState::with_pong_reply(
1459                node_id,
1460                PongReply {
1461                    latency,
1462                    pong_at: now,
1463                    from: SendAddr::Relay(send_addr.clone()),
1464                    pong_src: pong_src.clone(),
1465                },
1466            );
1467            Some((url, relay_state))
1468        };
1469
1470        let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64);
1471
1472        // endpoint with a `best_addr` that has a latency but no relay
1473        let (a_endpoint, a_socket_addr) = {
1474            let key = SecretKey::generate(&mut rng);
1475            let node_id = key.public();
1476            let ip_port = IpPort {
1477                ip: Ipv4Addr::UNSPECIFIED.into(),
1478                port: 10,
1479            };
1480            let endpoint_state = BTreeMap::from([(
1481                ip_port,
1482                PathState::with_pong_reply(
1483                    node_id,
1484                    PongReply {
1485                        latency,
1486                        pong_at: now,
1487                        from: SendAddr::Udp(ip_port.into()),
1488                        pong_src: pong_src.clone(),
1489                    },
1490                ),
1491            )]);
1492            (
1493                NodeState {
1494                    id: 0,
1495                    quic_mapped_addr: NodeIdMappedAddr::generate(),
1496                    node_id: key.public(),
1497                    last_full_ping: None,
1498                    relay_url: None,
1499                    udp_paths: NodeUdpPaths::from_parts(
1500                        endpoint_state,
1501                        UdpSendAddr::Valid(ip_port.into()),
1502                    ),
1503                    sent_pings: HashMap::new(),
1504                    last_used: Some(now),
1505                    last_call_me_maybe: None,
1506                    conn_type: Watchable::new(ConnectionType::Direct(ip_port.into())),
1507                    has_been_direct: AtomicBool::new(true),
1508                    #[cfg(any(test, feature = "test-utils"))]
1509                    path_selection: PathSelection::default(),
1510                },
1511                ip_port.into(),
1512            )
1513        };
1514        // endpoint w/ no best addr but a relay w/ latency
1515        let b_endpoint = {
1516            // let socket_addr = "0.0.0.0:9".parse().unwrap();
1517            let key = SecretKey::generate(&mut rng);
1518            NodeState {
1519                id: 1,
1520                quic_mapped_addr: NodeIdMappedAddr::generate(),
1521                node_id: key.public(),
1522                last_full_ping: None,
1523                relay_url: relay_and_state(key.public(), send_addr.clone()),
1524                udp_paths: NodeUdpPaths::new(),
1525                sent_pings: HashMap::new(),
1526                last_used: Some(now),
1527                last_call_me_maybe: None,
1528                conn_type: Watchable::new(ConnectionType::Relay(send_addr.clone())),
1529                has_been_direct: AtomicBool::new(false),
1530                #[cfg(any(test, feature = "test-utils"))]
1531                path_selection: PathSelection::default(),
1532            }
1533        };
1534
1535        // endpoint w/ no best addr but a relay w/ no latency
1536        let c_endpoint = {
1537            // let socket_addr = "0.0.0.0:8".parse().unwrap();
1538            let key = SecretKey::generate(&mut rng);
1539            NodeState {
1540                id: 2,
1541                quic_mapped_addr: NodeIdMappedAddr::generate(),
1542                node_id: key.public(),
1543                last_full_ping: None,
1544                relay_url: Some((
1545                    send_addr.clone(),
1546                    PathState::new(
1547                        key.public(),
1548                        SendAddr::from(send_addr.clone()),
1549                        Source::App,
1550                        now,
1551                    ),
1552                )),
1553                udp_paths: NodeUdpPaths::new(),
1554                sent_pings: HashMap::new(),
1555                last_used: Some(now),
1556                last_call_me_maybe: None,
1557                conn_type: Watchable::new(ConnectionType::Relay(send_addr.clone())),
1558                has_been_direct: AtomicBool::new(false),
1559                #[cfg(any(test, feature = "test-utils"))]
1560                path_selection: PathSelection::default(),
1561            }
1562        };
1563
1564        // endpoint w/ expired best addr and relay w/ latency
1565        let (d_endpoint, d_socket_addr) = {
1566            let socket_addr: SocketAddr = "0.0.0.0:7".parse().unwrap();
1567            let key = SecretKey::generate(&mut rng);
1568            let node_id = key.public();
1569            let endpoint_state = BTreeMap::from([(
1570                IpPort::from(socket_addr),
1571                PathState::with_pong_reply(
1572                    node_id,
1573                    PongReply {
1574                        latency,
1575                        pong_at: now,
1576                        from: SendAddr::Udp(socket_addr),
1577                        pong_src: pong_src.clone(),
1578                    },
1579                ),
1580            )]);
1581            (
1582                NodeState {
1583                    id: 3,
1584                    quic_mapped_addr: NodeIdMappedAddr::generate(),
1585                    node_id: key.public(),
1586                    last_full_ping: None,
1587                    relay_url: relay_and_state(key.public(), send_addr.clone()),
1588                    udp_paths: NodeUdpPaths::from_parts(
1589                        endpoint_state,
1590                        UdpSendAddr::Outdated(socket_addr),
1591                    ),
1592                    sent_pings: HashMap::new(),
1593                    last_used: Some(now),
1594                    last_call_me_maybe: None,
1595                    conn_type: Watchable::new(ConnectionType::Mixed(
1596                        socket_addr,
1597                        send_addr.clone(),
1598                    )),
1599                    has_been_direct: AtomicBool::new(false),
1600                    #[cfg(any(test, feature = "test-utils"))]
1601                    path_selection: PathSelection::default(),
1602                },
1603                socket_addr,
1604            )
1605        };
1606
1607        let mut expect = Vec::from([
1608            RemoteInfo {
1609                node_id: a_endpoint.node_id,
1610                relay_url: None,
1611                addrs: Vec::from([DirectAddrInfo {
1612                    addr: a_socket_addr,
1613                    latency: Some(latency),
1614                    last_control: Some((elapsed, ControlMsg::Pong)),
1615                    last_payload: None,
1616                    last_alive: Some(elapsed),
1617                    sources: HashMap::new(),
1618                }]),
1619                conn_type: ConnectionType::Direct(a_socket_addr),
1620                latency: Some(latency),
1621                last_used: Some(elapsed),
1622            },
1623            RemoteInfo {
1624                node_id: b_endpoint.node_id,
1625                relay_url: Some(RelayUrlInfo {
1626                    relay_url: b_endpoint.relay_url.as_ref().unwrap().0.clone(),
1627                    last_alive: None,
1628                    latency: Some(latency),
1629                }),
1630                addrs: Vec::new(),
1631                conn_type: ConnectionType::Relay(send_addr.clone()),
1632                latency: Some(latency),
1633                last_used: Some(elapsed),
1634            },
1635            RemoteInfo {
1636                node_id: c_endpoint.node_id,
1637                relay_url: Some(RelayUrlInfo {
1638                    relay_url: c_endpoint.relay_url.as_ref().unwrap().0.clone(),
1639                    last_alive: None,
1640                    latency: None,
1641                }),
1642                addrs: Vec::new(),
1643                conn_type: ConnectionType::Relay(send_addr.clone()),
1644                latency: None,
1645                last_used: Some(elapsed),
1646            },
1647            RemoteInfo {
1648                node_id: d_endpoint.node_id,
1649                relay_url: Some(RelayUrlInfo {
1650                    relay_url: d_endpoint.relay_url.as_ref().unwrap().0.clone(),
1651                    last_alive: None,
1652                    latency: Some(latency),
1653                }),
1654                addrs: Vec::from([DirectAddrInfo {
1655                    addr: d_socket_addr,
1656                    latency: Some(latency),
1657                    last_control: Some((elapsed, ControlMsg::Pong)),
1658                    last_payload: None,
1659                    last_alive: Some(elapsed),
1660                    sources: HashMap::new(),
1661                }]),
1662                conn_type: ConnectionType::Mixed(d_socket_addr, send_addr.clone()),
1663                latency: Some(Duration::from_millis(50)),
1664                last_used: Some(elapsed),
1665            },
1666        ]);
1667
1668        let node_map = NodeMap::from_inner(NodeMapInner {
1669            by_node_key: HashMap::from([
1670                (a_endpoint.node_id, a_endpoint.id),
1671                (b_endpoint.node_id, b_endpoint.id),
1672                (c_endpoint.node_id, c_endpoint.id),
1673                (d_endpoint.node_id, d_endpoint.id),
1674            ]),
1675            by_ip_port: HashMap::from([
1676                (a_socket_addr.into(), a_endpoint.id),
1677                (d_socket_addr.into(), d_endpoint.id),
1678            ]),
1679            by_quic_mapped_addr: HashMap::from([
1680                (a_endpoint.quic_mapped_addr, a_endpoint.id),
1681                (b_endpoint.quic_mapped_addr, b_endpoint.id),
1682                (c_endpoint.quic_mapped_addr, c_endpoint.id),
1683                (d_endpoint.quic_mapped_addr, d_endpoint.id),
1684            ]),
1685            by_id: HashMap::from([
1686                (a_endpoint.id, a_endpoint),
1687                (b_endpoint.id, b_endpoint),
1688                (c_endpoint.id, c_endpoint),
1689                (d_endpoint.id, d_endpoint),
1690            ]),
1691            next_id: 5,
1692            path_selection: PathSelection::default(),
1693        });
1694        let mut got = node_map.list_remote_infos(later);
1695        got.sort_by_key(|p| p.node_id);
1696        expect.sort_by_key(|p| p.node_id);
1697        remove_non_deterministic_fields(&mut got);
1698        assert_eq!(expect, got);
1699    }
1700
1701    fn remove_non_deterministic_fields(infos: &mut [RemoteInfo]) {
1702        for info in infos.iter_mut() {
1703            if info.relay_url.is_some() {
1704                info.relay_url.as_mut().unwrap().last_alive = None;
1705            }
1706        }
1707    }
1708
1709    #[test]
1710    fn test_prune_direct_addresses() {
1711        // When we handle a call-me-maybe with more than MAX_INACTIVE_DIRECT_ADDRESSES we do
1712        // not want to prune them right away but send pings to all of them.
1713        let mut rng = rand_chacha::ChaCha8Rng::seed_from_u64(0u64);
1714
1715        let key = SecretKey::generate(&mut rng);
1716        let opts = Options {
1717            node_id: key.public(),
1718            relay_url: None,
1719            active: true,
1720            source: crate::magicsock::Source::NamedApp {
1721                name: "test".into(),
1722            },
1723            path_selection: PathSelection::default(),
1724        };
1725        let mut ep = NodeState::new(0, opts);
1726
1727        let my_numbers_count: u16 = (MAX_INACTIVE_DIRECT_ADDRESSES + 5).try_into().unwrap();
1728        let my_numbers = (0u16..my_numbers_count)
1729            .map(|i| SocketAddr::new(Ipv4Addr::LOCALHOST.into(), 1000 + i))
1730            .collect();
1731        let call_me_maybe = disco::CallMeMaybe { my_numbers };
1732
1733        let metrics = MagicsockMetrics::default();
1734        let ping_messages = ep.handle_call_me_maybe(call_me_maybe, &metrics);
1735
1736        // We have no relay server and no previous direct addresses, so we should get the same
1737        // number of pings as direct addresses in the call-me-maybe.
1738        assert_eq!(ping_messages.len(), my_numbers_count as usize);
1739    }
1740}