Skip to content
Open
Show file tree
Hide file tree
Changes from all 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
5 changes: 3 additions & 2 deletions iroh/examples/locally-discovered-nodes.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@
//! This is an async, non-determinate process, so the number of NodeIDs discovered each time may be different. If you have other iroh endpoints or iroh nodes with [`MdnsDiscovery`] enabled, it may discover those nodes as well.
use std::time::Duration;

use iroh::{Endpoint, NodeId, node_info::UserData};
use iroh::{Endpoint, NodeId, discovery::DiscoveryEvent, node_info::UserData};
use n0_future::StreamExt;
use n0_snafu::Result;
use tokio::task::JoinSet;
Expand All @@ -32,7 +32,7 @@ async fn main() -> Result<()> {
tracing::error!("{e}");
return;
}
Ok(item) => {
Ok(DiscoveryEvent::Discovered(item)) => {
// if there is no user data, or the user data
// does not indicate that the discovered node
// is a part of the example, ignore it
Expand All @@ -53,6 +53,7 @@ async fn main() -> Result<()> {
println!("Found node {}!", item.node_id().fmt_short());
}
}
Ok(DiscoveryEvent::Expired(_)) => {}
};
}
});
Expand Down
93 changes: 59 additions & 34 deletions iroh/src/discovery.rs
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ pub trait Discovery: std::fmt::Debug + Send + Sync + 'static {
fn resolve(
&self,
_node_id: NodeId,
) -> Option<BoxStream<Result<DiscoveryItem, DiscoveryError>>> {
) -> Option<BoxStream<Result<DiscoveryEvent, DiscoveryError>>> {
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 resolve is specifically to find a DiscoveryItem or not, would this conceivably ever need to return DiscoveryEvent::Expired (or whatever colour that bikesheds ends up having)?

Copy link
Author

Choose a reason for hiding this comment

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

My understanding was resolve is scoped to a single node where subscribe is for getting the events for every node.

If that is correct then personally I think DiscoveryEvent here makes sense, if not we could concider changing it back but personally I think that feels a little inconsistent.

I suppose the method's name being resolve doesn't fit the best with the new return type.

I'm happy to do whatever you think here, but personally I think we leave it or concider renaming resolve to something else.

Copy link
Member

@ramfox ramfox Aug 20, 2025

Choose a reason for hiding this comment

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

Hi hi! resolve should still return a DiscoveryItem. subscribe makes sense to return a stream of events, but resolve should never need to return an expired event. If a NodeId can no longer be resolved, it should return None.

I don't fully understand what would be inconsistent. If you mean, it seems cleaner to have the same return type in both resolve and subscribe methods, I think that it is not only fine to have different return types, but could be better in some ways. It would more clearly show, just by the function signature, that the methods have different purposes.

But let me know if you were referring to something else!

Copy link
Contributor

Choose a reason for hiding this comment

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

resolve is meant to be a request for a single node. The reason it returns a stream is because a discovery service might be using multiple queries behind the scenes and different results may come in at different times. iroh wants to start using the first result as soon as possible, but if later results are better still have them available. But this stream is not used for a "long" time.

Hence it makes more sense to leave this as a DiscoveryItem.

None
}

Expand All @@ -352,13 +352,23 @@ pub trait Discovery: std::fmt::Debug + Send + Sync + 'static {
/// The [`crate::endpoint::Endpoint`] will `subscribe` to the discovery system
/// and add the discovered addresses to the internal address book as they arrive
/// on this stream.
fn subscribe(&self) -> Option<BoxStream<DiscoveryItem>> {
fn subscribe(&self) -> Option<BoxStream<DiscoveryEvent>> {
None
}
}

impl<T: Discovery> Discovery for Arc<T> {}

/// An event emitted from [`Discovery`] services.
#[derive(Debug, Clone, Eq, PartialEq)]
pub enum DiscoveryEvent {
/// A peer was discovered or it's information was updated.
Discovered(DiscoveryItem),
/// A peer was expired due to being inactive, unreachable, or otherwise
/// unavailable.
Expired(NodeId),
}

/// Node discovery results from [`Discovery`] services.
///
/// This is the item in the streams returned from [`Discovery::resolve`] and
Expand All @@ -367,7 +377,7 @@ impl<T: Discovery> Discovery for Arc<T> {}
///
/// This struct derefs to [`NodeData`], so you can access the methods from [`NodeData`]
/// directly from [`DiscoveryItem`].
#[derive(Debug, Clone)]
#[derive(Debug, Clone, Eq, PartialEq)]
pub struct DiscoveryItem {
/// The node info for the node, as discovered by the the discovery service.
node_info: NodeInfo,
Expand Down Expand Up @@ -488,7 +498,10 @@ impl Discovery for ConcurrentDiscovery {
}
}

fn resolve(&self, node_id: NodeId) -> Option<BoxStream<Result<DiscoveryItem, DiscoveryError>>> {
fn resolve(
&self,
node_id: NodeId,
) -> Option<BoxStream<Result<DiscoveryEvent, DiscoveryError>>> {
let streams = self
.services
.iter()
Expand All @@ -498,7 +511,7 @@ impl Discovery for ConcurrentDiscovery {
Some(Box::pin(streams))
}

fn subscribe(&self) -> Option<BoxStream<DiscoveryItem>> {
fn subscribe(&self) -> Option<BoxStream<DiscoveryEvent>> {
let mut streams = vec![];
for service in self.services.iter() {
if let Some(stream) = service.subscribe() {
Expand Down Expand Up @@ -592,7 +605,7 @@ impl DiscoveryTask {
fn create_stream(
ep: &Endpoint,
node_id: NodeId,
) -> Result<BoxStream<Result<DiscoveryItem, DiscoveryError>>, DiscoveryError> {
) -> Result<BoxStream<Result<DiscoveryEvent, DiscoveryError>>, DiscoveryError> {
let discovery = ep.discovery().ok_or(NoServiceConfiguredSnafu.build())?;
let stream = discovery
.resolve(node_id)
Expand Down Expand Up @@ -641,14 +654,17 @@ impl DiscoveryTask {
loop {
match stream.next().await {
Some(Ok(r)) => {
let provenance = r.provenance;
let node_addr = r.to_node_addr();
if node_addr.is_empty() {
debug!(%provenance, "empty address found");
continue;
if let DiscoveryEvent::Discovered(r) = &r {
let provenance = r.provenance;
let node_addr = r.to_node_addr();
if node_addr.is_empty() {
debug!(%provenance, "empty address found");
continue;
}
debug!(%provenance, addr = ?node_addr, "new address found");
ep.add_node_addr_with_source(node_addr, provenance).ok();
}
debug!(%provenance, addr = ?node_addr, "new address found");
ep.add_node_addr_with_source(node_addr, provenance).ok();

if let Some(tx) = on_first_tx.take() {
tx.send(Ok(())).ok();
}
Expand Down Expand Up @@ -685,7 +701,7 @@ pub struct Lagged {

#[derive(Clone, Debug)]
pub(super) struct DiscoverySubscribers {
inner: tokio::sync::broadcast::Sender<DiscoveryItem>,
inner: tokio::sync::broadcast::Sender<DiscoveryEvent>,
}

impl DiscoverySubscribers {
Expand All @@ -699,13 +715,13 @@ impl DiscoverySubscribers {
}
}

pub(crate) fn subscribe(&self) -> impl Stream<Item = Result<DiscoveryItem, Lagged>> + use<> {
pub(crate) fn subscribe(&self) -> impl Stream<Item = Result<DiscoveryEvent, Lagged>> + use<> {
use tokio_stream::wrappers::{BroadcastStream, errors::BroadcastStreamRecvError};
let recv = self.inner.subscribe();
BroadcastStream::new(recv).map_err(|BroadcastStreamRecvError::Lagged(n)| Lagged { val: n })
}

pub(crate) fn send(&self, item: DiscoveryItem) {
pub(crate) fn send(&self, item: DiscoveryEvent) {
// `broadcast::Sender::send` returns an error if the channel has no subscribers,
// which we don't care about.
self.inner.send(item).ok();
Expand Down Expand Up @@ -737,7 +753,7 @@ mod tests {
#[derive(Debug, Clone)]
struct TestDiscoveryShared {
nodes: Arc<Mutex<InfoStore>>,
watchers: tokio::sync::broadcast::Sender<DiscoveryItem>,
watchers: tokio::sync::broadcast::Sender<DiscoveryEvent>,
}

impl Default for TestDiscoveryShared {
Expand Down Expand Up @@ -770,7 +786,7 @@ mod tests {
}
}

pub fn send_passive(&self, item: DiscoveryItem) {
pub fn send_passive(&self, item: DiscoveryEvent) {
self.watchers.send(item).ok();
}
}
Expand Down Expand Up @@ -800,7 +816,7 @@ mod tests {
fn resolve(
&self,
node_id: NodeId,
) -> Option<BoxStream<Result<DiscoveryItem, DiscoveryError>>> {
) -> Option<BoxStream<Result<DiscoveryEvent, DiscoveryError>>> {
let addr_info = if self.resolve_wrong {
let ts = system_time_now() - 100_000;
let port: u16 = rand::thread_rng().gen_range(10_000..20_000);
Expand All @@ -822,7 +838,7 @@ mod tests {
let fut = async move {
time::sleep(delay).await;
tracing::debug!("resolve: {} = {item:?}", node_id.fmt_short());
Ok(item)
Ok(DiscoveryEvent::Discovered(item))
};
n0_future::stream::once_future(fut).boxed()
}
Expand All @@ -831,7 +847,7 @@ mod tests {
Some(stream)
}

fn subscribe(&self) -> Option<BoxStream<DiscoveryItem>> {
fn subscribe(&self) -> Option<BoxStream<DiscoveryEvent>> {
let recv = self.shared.watchers.subscribe();
let stream =
tokio_stream::wrappers::BroadcastStream::new(recv).filter_map(|item| item.ok());
Expand All @@ -848,7 +864,7 @@ mod tests {
fn resolve(
&self,
_node_id: NodeId,
) -> Option<BoxStream<Result<DiscoveryItem, DiscoveryError>>> {
) -> Option<BoxStream<Result<DiscoveryEvent, DiscoveryError>>> {
Some(n0_future::stream::empty().boxed())
}
}
Expand Down Expand Up @@ -1016,25 +1032,34 @@ mod tests {
ep2.node_addr().initialized().await;
let _ = ep1.connect(ep2.node_id(), TEST_ALPN).await?;

let item = tokio::time::timeout(Duration::from_secs(1), stream.next())
.await
.expect("timeout")
.expect("stream closed")
.expect("stream lagged");
let DiscoveryEvent::Discovered(item) =
tokio::time::timeout(Duration::from_secs(1), stream.next())
.await
.expect("timeout")
.expect("stream closed")
.expect("stream lagged")
else {
panic!("Returned unexpected discovery event!");
};

assert_eq!(item.node_id(), ep2.node_id());
assert_eq!(item.provenance(), "test-disco");

// inject item into discovery passively
let passive_node_id = SecretKey::generate(rand::thread_rng()).public();
let node_info = NodeInfo::new(passive_node_id);
let passive_item = DiscoveryItem::new(node_info, "test-disco-passive", None);
disco_shared.send_passive(passive_item.clone());

let item = tokio::time::timeout(Duration::from_secs(1), stream.next())
.await
.expect("timeout")
.expect("stream closed")
.expect("stream lagged");
disco_shared.send_passive(DiscoveryEvent::Discovered(passive_item.clone()));

let DiscoveryEvent::Discovered(item) =
tokio::time::timeout(Duration::from_secs(1), stream.next())
.await
.expect("timeout")
.expect("stream closed")
.expect("stream lagged")
else {
panic!("Returned unexpected discovery event!");
};
assert_eq!(item.node_id(), passive_node_id);
assert_eq!(item.provenance(), "test-disco-passive");

Expand Down
11 changes: 8 additions & 3 deletions iroh/src/discovery/dns.rs
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ use iroh_relay::dns::DnsResolver;
pub use iroh_relay::dns::{N0_DNS_NODE_ORIGIN_PROD, N0_DNS_NODE_ORIGIN_STAGING};
use n0_future::boxed::BoxStream;

use super::{DiscoveryContext, DiscoveryError, IntoDiscovery, IntoDiscoveryError};
use super::{DiscoveryContext, DiscoveryError, DiscoveryEvent, IntoDiscovery, IntoDiscoveryError};
use crate::{
discovery::{Discovery, DiscoveryItem},
endpoint::force_staging_infra,
Expand Down Expand Up @@ -105,15 +105,20 @@ impl IntoDiscovery for DnsDiscoveryBuilder {
}

impl Discovery for DnsDiscovery {
fn resolve(&self, node_id: NodeId) -> Option<BoxStream<Result<DiscoveryItem, DiscoveryError>>> {
fn resolve(
&self,
node_id: NodeId,
) -> Option<BoxStream<Result<DiscoveryEvent, DiscoveryError>>> {
let resolver = self.dns_resolver.clone();
let origin_domain = self.origin_domain.clone();
let fut = async move {
let node_info = resolver
.lookup_node_by_id_staggered(&node_id, &origin_domain, DNS_STAGGERING_MS)
.await
.map_err(|e| DiscoveryError::from_err("dns", e))?;
Ok(DiscoveryItem::new(node_info, "dns", None))
Ok(DiscoveryEvent::Discovered(DiscoveryItem::new(
node_info, "dns", None,
)))
};
let stream = n0_future::stream::once_future(fut);
Some(Box::pin(stream))
Expand Down
Loading
Loading