Skip to content
Open
Show file tree
Hide file tree
Changes from 4 commits
Commits
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
61 changes: 35 additions & 26 deletions iroh/src/endpoint.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,31 +23,6 @@ use n0_watcher::Watcher;
use tracing::{debug, instrument, trace, warn};
use url::Url;

use self::hooks::EndpointHooksList;
pub use super::magicsock::{
DirectAddr, DirectAddrType, PathInfo,
remote_map::{PathInfoList, Source},
};
#[cfg(wasm_browser)]
use crate::discovery::pkarr::PkarrResolver;
#[cfg(not(wasm_browser))]
use crate::dns::DnsResolver;
use crate::{
discovery::{ConcurrentDiscovery, DiscoveryError, DynIntoDiscovery, IntoDiscovery, UserData},
endpoint::presets::Preset,
magicsock::{self, Handle, RemoteStateActorStoppedError, mapped_addrs::MappedAddr},
metrics::EndpointMetrics,
net_report::Report,
tls::{self, DEFAULT_MAX_TLS_TICKETS},
};

mod connection;
pub(crate) mod hooks;
pub mod presets;
mod quic;

pub use hooks::{AfterHandshakeOutcome, BeforeConnectOutcome, EndpointHooks};

#[cfg(feature = "qlog")]
pub use self::quic::{QlogConfig, QlogFactory, QlogFileFactory};
pub use self::{
Expand All @@ -57,6 +32,7 @@ pub use self::{
IncomingZeroRttConnection, OutgoingZeroRtt, OutgoingZeroRttConnection,
RemoteEndpointIdError, ZeroRttStatus,
},
hooks::{AfterHandshakeOutcome, BeforeConnectOutcome, EndpointHooks},
quic::{
AcceptBi, AcceptUni, AckFrequencyConfig, AeadKey, ApplicationClose, Chunk, ClosedStream,
ConnectionClose, ConnectionError, ConnectionStats, Controller, ControllerFactory,
Expand All @@ -68,7 +44,28 @@ pub use self::{
VarIntBoundsExceeded, WeakConnectionHandle, WriteError, Written,
},
};
pub use crate::magicsock::transports::TransportConfig;
use self::{hooks::EndpointHooksList, presets::Preset};
#[cfg(wasm_browser)]
use crate::discovery::pkarr::PkarrResolver;
#[cfg(not(wasm_browser))]
use crate::dns::DnsResolver;
pub use crate::magicsock::{
DirectAddr, DirectAddrType, PathInfo,
remote_map::{PathInfoList, RemoteInfo, Source, TransportAddrInfo, TransportAddrUsage},
transports::TransportConfig,
};
use crate::{
discovery::{ConcurrentDiscovery, DiscoveryError, DynIntoDiscovery, IntoDiscovery, UserData},
magicsock::{self, Handle, RemoteStateActorStoppedError, mapped_addrs::MappedAddr},
metrics::EndpointMetrics,
net_report::Report,
tls::{self, DEFAULT_MAX_TLS_TICKETS},
};

mod connection;
pub(crate) mod hooks;
pub mod presets;
mod quic;

/// Builder for [`Endpoint`].
///
Expand Down Expand Up @@ -1060,6 +1057,18 @@ impl Endpoint {
&self.msock.metrics
}

/// Returns addressing information about a recently used remote endpoint.
///
/// The returned [`RemoteInfo`] contains a list of all transport addresses for the remote
/// that we know about. This is a snapshot in time and not a watcher.
///
/// Returns `None` if the endpoint doesn't have information about the remote.
/// When remote endpoints are no longer used, our endpoint will keep information around
/// for a little while, and then drop it. Afterwards, this will return `None`.
pub async fn remote_info(&self, endpoint_id: EndpointId) -> Option<RemoteInfo> {
self.msock.remote_info(endpoint_id).await
}

// # Methods for less common state updates.

/// Notifies the system of potential network changes.
Expand Down
12 changes: 11 additions & 1 deletion iroh/src/magicsock.rs
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ use crate::{
defaults::timeouts::NET_REPORT_TIMEOUT,
discovery::{ConcurrentDiscovery, Discovery, DiscoveryError, EndpointData, UserData},
endpoint::hooks::EndpointHooksList,
magicsock::remote_map::PathsWatcher,
magicsock::remote_map::{PathsWatcher, RemoteInfo},
metrics::EndpointMetrics,
net_report::{self, IfStateDetails, Report},
};
Expand Down Expand Up @@ -313,6 +313,16 @@ impl MagicSock {
}
}

/// Fetches the [`RemoteInfo`] about a remote from the `RemoteStateActor`.
///
/// Returns `None` if no actor is running for the remote.
pub(crate) async fn remote_info(&self, id: EndpointId) -> Option<RemoteInfo> {
let actor = self.remote_map.remote_state_actor_if_exists(id)?;
let (tx, rx) = oneshot::channel();
actor.send(RemoteStateMessage::RemoteInfo(tx)).await.ok()?;
rx.await.ok()
}

pub(crate) async fn insert_relay(
&self,
relay: RelayUrl,
Expand Down
15 changes: 14 additions & 1 deletion iroh/src/magicsock/remote_map.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,9 @@ use tokio::sync::mpsc;

pub(crate) use self::remote_state::PathsWatcher;
pub(super) use self::remote_state::RemoteStateMessage;
pub use self::remote_state::{PathInfo, PathInfoList};
pub use self::remote_state::{
PathInfo, PathInfoList, RemoteInfo, TransportAddrInfo, TransportAddrUsage,
};
use self::remote_state::{RemoteStateActor, RemoteStateHandle};
use super::{
DirectAddr, MagicsockMetrics,
Expand Down Expand Up @@ -118,6 +120,17 @@ impl RemoteMap {
}
}

pub(super) fn remote_state_actor_if_exists(
&self,
eid: EndpointId,
) -> Option<mpsc::Sender<RemoteStateMessage>> {
self.actor_handles
.lock()
.expect("poisoned")
.get(&eid)
.and_then(|handle| handle.sender.get())
}

/// Starts a new remote state actor and returns a handle and a sender.
///
/// The handle is not inserted into the endpoint map, this must be done by the caller of this function.
Expand Down
14 changes: 14 additions & 0 deletions iroh/src/magicsock/remote_map/remote_state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ use tokio::sync::oneshot;
use tokio_stream::wrappers::{BroadcastStream, errors::BroadcastStreamRecvError};
use tracing::{Instrument, Level, debug, error, event, info_span, instrument, trace, warn};

pub use self::remote_info::{RemoteInfo, TransportAddrInfo, TransportAddrUsage};
use self::{
guarded_channel::{GuardedReceiver, GuardedSender, guarded_channel},
path_state::RemotePathState,
Expand All @@ -49,6 +50,7 @@ const HOLEPUNCH_ATTEMPTS_INTERVAL: Duration = Duration::from_secs(5);

mod guarded_channel;
mod path_state;
mod remote_info;

// TODO: use this
// /// The latency at or under which we don't try to upgrade to a better path.
Expand Down Expand Up @@ -336,6 +338,14 @@ impl RemoteStateActor {
RemoteStateMessage::ResolveRemote(addrs, tx) => {
self.handle_msg_resolve_remote(addrs, tx);
}
RemoteStateMessage::RemoteInfo(tx) => {
let addrs = self.paths.to_remote_addrs();
let info = RemoteInfo {
endpoint_id: self.endpoint_id,
addrs,
};
tx.send(info).ok();
}
}
}

Expand Down Expand Up @@ -1090,6 +1100,10 @@ pub(crate) enum RemoteStateMessage {
BTreeSet<TransportAddr>,
oneshot::Sender<Result<(), DiscoveryError>>,
),
/// Returns information about the remote.
///
/// This currently only includes a list of all known transport addresses for the remote.
RemoteInfo(oneshot::Sender<RemoteInfo>),
}

/// A handle to a [`RemoteStateActor`].
Expand Down
23 changes: 22 additions & 1 deletion iroh/src/magicsock/remote_map/remote_state/path_state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ use rustc_hash::FxHashMap;
use tokio::sync::oneshot;
use tracing::trace;

use super::Source;
use super::{Source, TransportAddrInfo, TransportAddrUsage};
use crate::{discovery::DiscoveryError, magicsock::transports, metrics::MagicsockMetrics};

/// Maximum number of IP paths we keep around per endpoint.
Expand Down Expand Up @@ -62,6 +62,27 @@ impl RemotePathState {
metrics,
}
}
pub(super) fn to_remote_addrs(&self) -> Vec<TransportAddrInfo> {
self.paths
.iter()
.flat_map(|(addr, state)| {
let usage = match state.status {
PathStatus::Open => TransportAddrUsage::Active,
PathStatus::Inactive(instant) => {
TransportAddrUsage::Inactive { last_used: instant }
}
PathStatus::Unusable => TransportAddrUsage::Unusable,
PathStatus::Unknown => TransportAddrUsage::Unknown,
};
let last_source = state.sources.iter().max_by(|a, b| a.1.cmp(b.1))?;
Some(TransportAddrInfo {
addr: addr.clone().into(),
usage,
most_recent_source: last_source.0.clone(),
})
})
.collect()
}
/// Insert a new address of an open path into our list of paths.
///
/// This will emit pending resolve requests and trigger pruning paths.
Expand Down
97 changes: 97 additions & 0 deletions iroh/src/magicsock/remote_map/remote_state/remote_info.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
use std::collections::BTreeSet;

use iroh_base::{EndpointAddr, EndpointId, TransportAddr};
use n0_future::time::Instant;

use crate::endpoint::Source;

/// Information about a remote endpoint.
#[derive(Debug, Clone)]
pub struct RemoteInfo {
pub(super) endpoint_id: EndpointId,
pub(super) addrs: Vec<TransportAddrInfo>,
}

impl RemoteInfo {
/// Returns the remote's endpoint id.
pub fn id(&self) -> EndpointId {
self.endpoint_id
}

/// Returns an iterator over known all addresses for this remote.
///
/// Note that this may include outdated or unusable addresses.
pub fn addrs(&self) -> impl Iterator<Item = &TransportAddrInfo> {
self.addrs.iter()
}

/// Converts into an iterator over known all addresses for this remote.
///
/// Note that this may include outdated or unusable addresses.
pub fn into_addrs(self) -> impl Iterator<Item = TransportAddrInfo> {
self.addrs.into_iter()
}

/// Returns a [`EndpointAddr`] that includes all addresses that are not [`TransportAddrUsage::Unusable`].
pub fn into_endpoint_addr(self) -> EndpointAddr {
let addrs = self
.addrs
.into_iter()
.filter(|a| !matches!(a.usage(), TransportAddrUsage::Unusable))
.map(|a| a.addr);

EndpointAddr {
id: self.endpoint_id,
addrs: BTreeSet::from_iter(addrs),
}
}
}

/// Address of a remote with some metadata
#[derive(Debug, Clone)]
pub struct TransportAddrInfo {
pub(super) addr: TransportAddr,
pub(super) usage: TransportAddrUsage,
pub(super) most_recent_source: Source,
}

impl TransportAddrInfo {
/// Returns the [`TransportAddr`].
pub fn addr(&self) -> &TransportAddr {
&self.addr
}

/// Converts into [`TransportAddr`].
pub fn into_addr(self) -> TransportAddr {
self.addr
}

/// Returns information how this address is used.
pub fn usage(&self) -> TransportAddrUsage {
self.usage
}

/// Returns the most recent source of this address.
///
/// We may learn about new addresses from multiple sources. This returns the most recent source
/// that told us about this address.
pub fn most_recent_source(&self) -> &Source {
&self.most_recent_source
}
}

/// Information how a transport address is used.
#[derive(Debug, Copy, Clone)]
pub enum TransportAddrUsage {
/// The address is in active use.
Active,
/// The address was used, but is not currently.
Inactive {
/// Time when this address was last used.
last_used: Instant,
},
/// We tried to use this address, but failed.
Unusable,
/// We have not tried to use this address.
Unknown,
}
Loading