Skip to content
Open
Show file tree
Hide file tree
Changes from 3 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;
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The import ordering was off, and when I changed a line, cargo make format picked it up again. The only actual change here is that I added the re-exports from the new remote_info module.

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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe mention that this is a snapshot in time, or sth, to indicate that this can have already changed by the moment it is used

///
/// The returned [`RemoteInfo`] contains a list of all transport addresses for the remote
/// that we know about.
///
/// 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())
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you think this is wroth having? The actor would be started and shut down again. I guess that's not ideal. I'm just sad whenever there are two functions like this, one that creates-on-demand and one that returns an Option. I can live with it I guess.


/// 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> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

missing blank line? (how does rustfmt not handle this?)

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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this was already mentioned, but this should explain that this is a snapshot at the time it was created.

#[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
}
Comment on lines +78 to +80
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm also not comfortable exposing this yet. Source is a number of states that I don't know we'll always want to keep and want to keep around as such. Also, there is not a single source for an addr, certainly not the way it is used currently. Which makes this messy and indicates that it's not a great thing to expose.

Source is public right now, but only because of discovery methods. I had an issue originally to figure something out about that. This was solved in 50fdda3 in a way that seemed to solve the immediate issue. But I do now think it was probably the wrong approach, given the variants on that enum. I don't think this is suitable for a 1.0 API.

}

/// 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,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe this is still problematic? I was hopeing to keep postponing having to figure this out to be fair.

},
/// We tried to use this address, but failed.
Unusable,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not comfortable exposing this to be fair. It forces us to keep this status around and I don't think I want to do that.

I think for now I'm at most comfortable with exposing two items on this enum:

  • Active
  • Inactive

No timestamps, nothing else.

/// We have not tried to use this address.
Unknown,
}
Loading