From 09a7973bf17d72496094a1397a3a431bf0a5034f Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 4 Mar 2024 16:45:45 +0100 Subject: [PATCH 01/66] feat: combine discovery services --- iroh-net/src/discovery.rs | 93 ++++++++++++++++++++ iroh-net/src/lib.rs | 1 + iroh-net/src/magic_endpoint.rs | 150 +++++++++++++++++++++++---------- iroh-net/src/magicsock.rs | 59 +++++-------- 4 files changed, 221 insertions(+), 82 deletions(-) create mode 100644 iroh-net/src/discovery.rs diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs new file mode 100644 index 00000000000..211ca5385c0 --- /dev/null +++ b/iroh-net/src/discovery.rs @@ -0,0 +1,93 @@ +//! Trait and utils for the node discovery mechanism. + +use anyhow::Result; +use futures::stream::BoxStream; + +use crate::{AddrInfo, MagicEndpoint, NodeId}; + +/// Node discovery for [`super::MagicEndpoint`]. +/// +/// The purpose of this trait is to hoop up a node discovery mechanism that +/// allows finding information such as the derp url and current addresses +/// of a node given the id. +/// +/// To allow for discovery, the [`super::MagicEndpoint`] will call `publish` whenever +/// discovery information changes. If a discovery mechanism requires a periodic +/// refresh, it should start it's own task. +pub trait Discovery: std::fmt::Debug + Send + Sync { + /// Publish the given [`AddrInfo`] to the discovery mechanisms. + /// + /// This is fire and forget, since the magicsock can not wait for successful + /// publishing. If publishing is async, the implementation should start it's + /// own task. + /// + /// This will be called from a tokio task, so it is safe to spawn new tasks. + /// These tasks will be run on the runtime of the [`super::MagicEndpoint`]. + fn publish(&self, _info: &AddrInfo) {} + + /// Resolve the [`AddrInfo`] for the given [`PublicKey`]. + /// + /// Once the returned [`BoxStream`] is dropped, the service should stop any pending + /// work. + fn resolve( + &self, + _endpoint: MagicEndpoint, + _node_id: NodeId, + ) -> Option>> { + None + } +} + +/// The results returned from [`Discovery::resolve`]. +#[derive(Debug, Clone)] +pub struct DiscoveryItem { + /// A static string to identify the discovery source. + /// + /// Should be uniform per discovery service. + pub provenance: &'static str, + /// Optional timestamp when this node address info was last updated. + /// + /// Must be microseconds since the unix epoch. + pub last_updated: Option, + /// The adress info for the node being resolved. + pub addr_info: AddrInfo, +} + +/// A discovery service that combines multiple discovery sources. +#[derive(Debug, Default)] +pub struct CombinedDiscovery { + services: Vec>, +} + +impl CombinedDiscovery { + /// Create a new [`CombinedDiscovery`]. + pub fn new() -> Self { + Self::default() + } + + /// Add a [`Discovery`] service. + pub fn add(&mut self, service: impl Discovery + 'static) { + self.services.push(Box::new(service)); + } +} + +impl Discovery for CombinedDiscovery { + fn publish(&self, info: &AddrInfo) { + for service in &self.services { + service.publish(info); + } + } + + fn resolve( + &self, + endpoint: MagicEndpoint, + node_id: NodeId, + ) -> Option>> { + let streams = self + .services + .iter() + .filter_map(|service| service.resolve(endpoint.clone(), node_id)); + let streams = futures::stream::select_all(streams); + Some(Box::pin(streams)) + } +} diff --git a/iroh-net/src/lib.rs b/iroh-net/src/lib.rs index 18e4ca6233a..d7fc8c8c3fd 100644 --- a/iroh-net/src/lib.rs +++ b/iroh-net/src/lib.rs @@ -14,6 +14,7 @@ pub mod config; pub mod defaults; pub mod derp; pub mod dialer; +pub mod discovery; mod disco; mod dns; pub mod magic_endpoint; diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index faa7021d8a1..c71e051fabd 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -2,18 +2,21 @@ use std::{net::SocketAddr, path::PathBuf, sync::Arc, time::Duration}; -use anyhow::{anyhow, ensure, Context, Result}; +use anyhow::{anyhow, bail, ensure, Context, Result}; use derive_more::Debug; use futures::StreamExt; use quinn_proto::VarInt; -use tracing::{debug, trace}; +use tokio::sync::oneshot; +use tokio_util::sync::CancellationToken; +use tracing::{debug, trace, warn}; use crate::{ config, defaults::default_derp_map, derp::{DerpMap, DerpMode, DerpUrl}, + discovery::Discovery, key::{PublicKey, SecretKey}, - magicsock::{self, Discovery, MagicSock}, + magicsock::{self, MagicSock}, tls, NodeId, }; @@ -182,6 +185,7 @@ pub struct MagicEndpoint { msock: MagicSock, endpoint: quinn::Endpoint, keylog: bool, + cancel_token: CancellationToken, } impl MagicEndpoint { @@ -224,6 +228,7 @@ impl MagicEndpoint { msock, endpoint, keylog, + cancel_token: CancellationToken::new(), }) } @@ -335,13 +340,71 @@ impl MagicEndpoint { self.msock.tracked_endpoint(node_id) } - async fn resolve(&self, node_id: &PublicKey) -> Result { - if let Some(discovery) = self.msock.discovery() { - debug!("no mapping address for {node_id}, resolving via {discovery:?}"); - discovery.resolve(node_id).await - } else { - anyhow::bail!("no discovery mechanism configured"); - } + async fn resolve_until_first( + &self, + node_id: PublicKey, + cancel: CancellationToken, + ) -> anyhow::Result<()> { + let (on_first_result_tx, on_first_result_rx) = oneshot::channel(); + self.resolve_inner(node_id, on_first_result_tx, cancel)?; + on_first_result_rx.await??; + Ok(()) + } + + fn resolve_inner( + &self, + node_id: PublicKey, + on_first_result: oneshot::Sender>, + cancel: CancellationToken, + ) -> Result<()> { + let Some(discovery) = self.msock.discovery() else { + bail!("no discovery mechanism configured"); + }; + debug!("no mapping address for {node_id}, resolving via {discovery:?}"); + let Some(mut stream) = discovery.resolve(self.clone(), node_id) else { + bail!("no discovery service is able to resolve node {}", node_id.fmt_short()); + }; + let ep = self.clone(); + tokio::task::spawn(async move { + let mut on_first_result = Some(on_first_result); + loop { + let next = tokio::select! { + // cancel on endpoint close + _ = ep.cancel_token.cancelled() => break, + // cancel on explicit cancel request + _ = cancel.cancelled() => break, + res = stream.next() => match res { + None => break, + Some(res) => res + } + }; + match next { + Ok(r) => { + debug!(node = %node_id.fmt_short(), provenance = r.provenance, addr = ?r.addr_info, "discovery: new address found"); + let addr = NodeAddr { + info: r.addr_info, + node_id, + }; + ep.add_node_addr(addr).ok(); + if let Some(tx) = on_first_result.take() { + tx.send(Ok(())).ok(); + } + } + Err(err) => { + warn!(?err, "discovery service failed"); + if let Some(tx) = on_first_result.take() { + tx.send(Err(err)).ok(); + } + break; + } + } + } + if let Some(tx) = on_first_result.take() { + tx.send(Err(anyhow!("No results for {}", node_id.fmt_short()))) + .ok(); + } + }); + Ok(()) } /// Connect to a remote endpoint, using just the nodes's [`PublicKey`]. @@ -350,23 +413,8 @@ impl MagicEndpoint { node_id: &PublicKey, alpn: &[u8], ) -> Result { - let addr = match self.msock.get_mapping_addr(node_id) { - Some(addr) => addr, - None => { - let info = self.resolve(node_id).await?; - let peer_addr = NodeAddr { - node_id: *node_id, - info, - }; - self.add_node_addr(peer_addr)?; - self.msock.get_mapping_addr(node_id).ok_or_else(|| { - anyhow!("Failed to retrieve the mapped address from the magic socket. Unable to dial node {node_id:?}") - })? - } - }; - - debug!("connecting to {}: (via {})", node_id, addr); - self.connect_inner(node_id, alpn, addr).await + let addr = NodeAddr::new(*node_id); + self.connect(addr, alpn).await } /// Connect to a remote endpoint. @@ -380,17 +428,24 @@ impl MagicEndpoint { /// /// If no UDP addresses and no DERP Url is provided, it will error. pub async fn connect(&self, node_addr: NodeAddr, alpn: &[u8]) -> Result { - self.add_node_addr(node_addr.clone())?; + if !node_addr.info.is_empty() { + self.add_node_addr(node_addr.clone())?; + } let NodeAddr { node_id, info } = node_addr; - let addr = self.msock.get_mapping_addr(&node_id); - let Some(addr) = addr else { - return Err(match (info.direct_addresses.is_empty(), info.derp_url) { - (true, None) => { - anyhow!("No UDP addresses or DERP Url provided. Unable to dial node {node_id:?}") - } - _ => anyhow!("Failed to retrieve the mapped address from the magic socket. Unable to dial node {node_id:?}") - }); + + // Store a cancel token if we start a discovery for the node id. + let mut cancel_discovery = None; + let addr = match self.msock.get_mapping_addr(&node_id) { + Some(addr) => addr, + None => { + let cancel_token = CancellationToken::new(); + cancel_discovery = Some(cancel_token.clone()); + self.resolve_until_first(node_id, cancel_token).await?; + self.msock.get_mapping_addr(&node_id).await.ok_or_else(|| { + anyhow!("Failed to retrieve the mapped address from the magic socket. Unable to dial node {node_id:?}") + })? + } }; debug!( @@ -398,10 +453,17 @@ impl MagicEndpoint { node_id, addr, info.direct_addresses ); - self.connect_inner(&node_id, alpn, addr).await + let conn = self.connect_quinn(&node_id, alpn, addr).await?; + + // Connection was successfull, cancel the node discovery task (if still running). + if let Some(cancel_token) = cancel_discovery { + cancel_token.cancel(); + } + + Ok(conn) } - async fn connect_inner( + async fn connect_quinn( &self, node_id: &PublicKey, alpn: &[u8], @@ -440,6 +502,7 @@ impl MagicEndpoint { /// /// If no UDP addresses are added, and `derp_url` is `None`, it will error. /// If no UDP addresses are added, and the given `derp_url` cannot be dialed, it will error. + // TODO: This is infallible, stop returning a result. pub fn add_node_addr(&self, node_addr: NodeAddr) -> Result<()> { self.msock.add_node_addr(node_addr); Ok(()) @@ -456,6 +519,7 @@ impl MagicEndpoint { /// Returns an error if closing the magic socket failed. /// TODO: Document error cases. pub async fn close(&self, error_code: VarInt, reason: &[u8]) -> Result<()> { + self.cancel_token.cancel(); self.endpoint.close(error_code, reason); self.msock.close().await?; Ok(()) @@ -501,9 +565,9 @@ pub async fn get_alpn(connecting: &mut quinn::Connecting) -> Result { match data.downcast::() { Ok(data) => match data.protocol { Some(protocol) => std::string::String::from_utf8(protocol).map_err(Into::into), - None => anyhow::bail!("no ALPN protocol available"), + None => bail!("no ALPN protocol available"), }, - Err(_) => anyhow::bail!("unknown handshake type"), + Err(_) => bail!("unknown handshake type"), } } @@ -511,11 +575,11 @@ pub async fn get_alpn(connecting: &mut quinn::Connecting) -> Result { pub fn get_remote_node_id(connection: &quinn::Connection) -> Result { let data = connection.peer_identity(); match data { - None => anyhow::bail!("no peer certificate found"), + None => bail!("no peer certificate found"), Some(data) => match data.downcast::>() { Ok(certs) => { if certs.len() != 1 { - anyhow::bail!( + bail!( "expected a single peer certificate, but {} found", certs.len() ); @@ -523,7 +587,7 @@ pub fn get_remote_node_id(connection: &quinn::Connection) -> Result { let cert = tls::certificate::parse(&certs[0])?; Ok(cert.peer_id()) } - Err(_) => anyhow::bail!("invalid peer certificate"), + Err(_) => bail!("invalid peer certificate"), }, } } diff --git a/iroh-net/src/magicsock.rs b/iroh-net/src/magicsock.rs index 6b73cf11a03..374e433d262 100644 --- a/iroh-net/src/magicsock.rs +++ b/iroh-net/src/magicsock.rs @@ -35,7 +35,7 @@ use std::{ use anyhow::{anyhow, Context as _, Result}; use bytes::Bytes; -use futures::{future::BoxFuture, FutureExt, Stream}; +use futures::{FutureExt, Stream}; use iroh_metrics::{inc, inc_by}; use quinn::AsyncUdpSocket; use rand::{seq::SliceRandom, Rng, SeedableRng}; @@ -55,6 +55,7 @@ use crate::{ config, derp::{DerpMap, DerpUrl}, disco::{self, SendAddr}, + discovery::Discovery, dns::DNS_RESOLVER, key::{PublicKey, SecretKey, SharedSecret}, magic_endpoint::NodeAddr, @@ -119,35 +120,6 @@ pub struct Options { pub discovery: Option>, } -/// Node discovery for [`super::MagicEndpoint`]. -/// -/// The purpose of this trait is to hoop up a node discovery mechanism that -/// allows finding information such as the derp url and current addresses -/// of a node given the id. -/// -/// To allow for discovery, the [`super::MagicEndpoint`] will call `publish` whenever -/// discovery information changes. If a discovery mechanism requires a periodic -/// refresh, it should start it's own task. -pub trait Discovery: std::fmt::Debug + Send + Sync { - /// Publish the given [`AddrInfo`] to the discovery mechanisms. - /// - /// This is fire and forget, since the magicsock can not wait for successful - /// publishing. If publishing is async, the implementation should start it's - /// own task. - /// - /// This will be called from a tokio task, so it is safe to spawn new tasks. - /// These tasks will be run on the runtime of the [`super::MagicEndpoint`]. - fn publish(&self, info: &AddrInfo); - - /// Resolve the [`AddrInfo`] for the given [`PublicKey`]. - /// - /// This is only called from [`super::MagicEndpoint::connect_by_node_id`], and only if - /// the [`AddrInfo`] is not already known. - /// - /// This is async since the connect can not proceed without the [`AddrInfo`]. - fn resolve<'a>(&'a self, node_id: &'a PublicKey) -> BoxFuture<'a, Result>; -} - impl Default for Options { fn default() -> Self { Options { @@ -996,6 +968,22 @@ impl Inner { inc!(MagicsockMetrics, re_stun_calls); self.endpoints_update_state.schedule_run(why); } + + /// Publishes our address to a discovery service, if configured. + /// + /// Called whenever our addresses or home derper changes. + fn publish_my_addr(&self) { + if let Some(ref discovery) = self.discovery { + let eps = self.endpoints.read(); + let derp_url = self.my_derp(); + let direct_addresses = eps.iter().map(|ep| ep.addr).collect(); + let info = AddrInfo { + derp_url, + direct_addresses, + }; + discovery.publish(&info); + } + } } #[derive(Clone, Debug)] @@ -2039,15 +2027,7 @@ impl Actor { if updated { let eps = self.inner.endpoints.read(); eps.log_endpoint_change(); - - if let Some(ref discovery) = self.inner.discovery { - let direct_addresses = eps.iter().map(|ep| ep.addr).collect(); - let info = AddrInfo { - derp_url: self.inner.my_derp(), - direct_addresses, - }; - discovery.publish(&info); - } + self.inner.publish_my_addr(); } // Regardless of whether our local endpoints changed, we now want to send any queued @@ -2203,6 +2183,7 @@ impl Actor { // On change, notify all currently connected DERP servers and // start connecting to our home DERP if we are not already. info!("home is now derp {}", derp_url); + self.inner.publish_my_addr(); self.send_derp_actor(DerpActorMessage::NotePreferred(derp_url.clone())); self.send_derp_actor(DerpActorMessage::Connect { From a088781ac24b5e7ee9fc300ec3de2756428d2b14 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 6 Mar 2024 11:02:21 +0100 Subject: [PATCH 02/66] refactor: impl from iterator for combined discovery --- iroh-net/src/discovery.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 211ca5385c0..84dd9fd0e38 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -71,6 +71,16 @@ impl CombinedDiscovery { } } +impl From for CombinedDiscovery +where + T: Iterator>, +{ + fn from(iter: T) -> Self { + let services = iter.collect::>(); + Self { services } + } +} + impl Discovery for CombinedDiscovery { fn publish(&self, info: &AddrInfo) { for service in &self.services { From 42f1f3b0b0752b495b184de0bf676081f9664df1 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 6 Mar 2024 12:03:41 +0100 Subject: [PATCH 03/66] tests: add tests for combined discovery --- iroh-net/src/discovery.rs | 248 +++++++++++++++++++++++++++++++++++++- 1 file changed, 246 insertions(+), 2 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 84dd9fd0e38..11bcb33a670 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -73,10 +73,10 @@ impl CombinedDiscovery { impl From for CombinedDiscovery where - T: Iterator>, + T: IntoIterator>, { fn from(iter: T) -> Self { - let services = iter.collect::>(); + let services = iter.into_iter().collect::>(); Self { services } } } @@ -101,3 +101,247 @@ impl Discovery for CombinedDiscovery { Some(Box::pin(streams)) } } + +#[cfg(test)] +mod tests { + use std::{ + collections::{BTreeSet, HashMap}, + net::SocketAddr, + sync::Arc, + time::{Duration, SystemTime}, + }; + + use futures::{stream, StreamExt}; + use parking_lot::Mutex; + + use crate::{derp::DerpMode, key::SecretKey, NodeAddr}; + + use super::*; + + #[derive(Debug, Clone, Default)] + struct TestDiscoveryShared { + nodes: Arc>>, + } + impl TestDiscoveryShared { + pub fn create_discovery(&self, node_id: NodeId) -> TestDiscovery { + TestDiscovery { + node_id, + shared: self.clone(), + publish: true, + resolve_wrong: false, + delay: Duration::from_millis(200), + } + } + + pub fn create_lying_discovery(&self, node_id: NodeId) -> TestDiscovery { + TestDiscovery { + node_id, + shared: self.clone(), + publish: false, + resolve_wrong: true, + delay: Duration::from_millis(100), + } + } + } + #[derive(Debug)] + struct TestDiscovery { + node_id: NodeId, + shared: TestDiscoveryShared, + publish: bool, + resolve_wrong: bool, + delay: Duration, + } + + impl Discovery for TestDiscovery { + fn publish(&self, info: &AddrInfo) { + if !self.publish { + return; + } + let now = system_time_now(); + self.shared + .nodes + .lock() + .insert(self.node_id, (info.clone(), now)); + } + + fn resolve( + &self, + endpoint: MagicEndpoint, + node_id: NodeId, + ) -> Option>> { + let addr_info = match self.resolve_wrong { + false => self.shared.nodes.lock().get(&node_id).cloned(), + true => { + let ts = system_time_now() - 100_000; + let port: u16 = rand::random(); + // "240.0.0.0/4" is reserved and unreachable + let addr: SocketAddr = format!("240.0.0.1:{port}").parse().unwrap(); + let addr_info = AddrInfo { + derp_url: None, + direct_addresses: BTreeSet::from([addr]), + }; + Some((addr_info, ts)) + } + }; + let stream = match addr_info { + Some((addr_info, ts)) => { + let item = DiscoveryItem { + provenance: "test-disco", + last_updated: Some(ts), + addr_info, + }; + let delay = self.delay; + let fut = async move { + tokio::time::sleep(delay).await; + tracing::debug!( + "resolve on {}: {} = {item:?}", + endpoint.node_id().fmt_short(), + node_id.fmt_short() + ); + Ok(item) + }; + stream::once(fut).boxed() + } + None => stream::empty().boxed(), + }; + Some(stream) + } + } + + #[derive(Debug)] + struct EmptyDiscovery; + impl Discovery for EmptyDiscovery { + fn publish(&self, _info: &AddrInfo) {} + + fn resolve( + &self, + _endpoint: MagicEndpoint, + _node_id: NodeId, + ) -> Option>> { + Some(stream::empty().boxed()) + } + } + + const TEST_ALPN: &[u8] = b"n0/iroh/test"; + + /// This is a smoke test for our discovery mechanism. + #[tokio::test] + async fn magic_endpoint_discovery_simple_shared() -> anyhow::Result<()> { + let _guard = iroh_test::logging::setup(); + let disco_shared = TestDiscoveryShared::default(); + let ep1 = { + let secret = SecretKey::generate(); + let disco = disco_shared.create_discovery(secret.public()); + new_endpoint(secret, disco).await + }; + let ep2 = { + let secret = SecretKey::generate(); + let disco = disco_shared.create_discovery(secret.public()); + new_endpoint(secret, disco).await + }; + let ep1_addr = NodeAddr::new(ep1.node_id()); + // wait for out address to be updated and thus published at least once + ep1.my_addr().await?; + let _conn = ep2.connect(ep1_addr, TEST_ALPN).await?; + Ok(()) + } + + /// This test adds an empty discovery which provides no addresses. + #[tokio::test] + async fn magic_endpoint_discovery_combined_with_empty() -> anyhow::Result<()> { + let _guard = iroh_test::logging::setup(); + let disco_shared = TestDiscoveryShared::default(); + let ep1 = { + let secret = SecretKey::generate(); + let disco = disco_shared.create_discovery(secret.public()); + new_endpoint(secret, disco).await + }; + let ep2 = { + let secret = SecretKey::generate(); + let disco1 = EmptyDiscovery; + let disco2 = disco_shared.create_discovery(secret.public()); + let mut disco = CombinedDiscovery::new(); + disco.add(disco1); + disco.add(disco2); + new_endpoint(secret, disco).await + }; + let ep1_addr = NodeAddr::new(ep1.node_id()); + // wait for out address to be updated and thus published at least once + ep1.my_addr().await?; + let _conn = ep2.connect(ep1_addr, TEST_ALPN).await?; + Ok(()) + } + + /// This test adds a "lying" discovery which provides a wrong address. + /// This is to make sure that as long as one of the discoveries returns a working address, we + /// will connect successfully. + #[tokio::test] + async fn magic_endpoint_discovery_combined_with_empty_and_wrong() -> anyhow::Result<()> { + let _guard = iroh_test::logging::setup(); + let disco_shared = TestDiscoveryShared::default(); + let ep1 = { + let secret = SecretKey::generate(); + let disco = disco_shared.create_discovery(secret.public()); + new_endpoint(secret, disco).await + }; + let ep2 = { + let secret = SecretKey::generate(); + let disco1 = EmptyDiscovery; + let disco2 = disco_shared.create_lying_discovery(secret.public()); + let disco3 = disco_shared.create_discovery(secret.public()); + let mut disco = CombinedDiscovery::new(); + disco.add(disco1); + disco.add(disco2); + disco.add(disco3); + new_endpoint(secret, disco).await + }; + let ep1_addr = NodeAddr::new(ep1.node_id()); + // wait for out address to be updated and thus published at least once + ep1.my_addr().await?; + let _conn = ep2.connect(ep1_addr, TEST_ALPN).await?; + Ok(()) + } + + /// This test only has the "lying" discovery. It is here to make sure that this actually fails. + #[tokio::test] + async fn magic_endpoint_discovery_combined_wrong_only() -> anyhow::Result<()> { + let _guard = iroh_test::logging::setup(); + let disco_shared = TestDiscoveryShared::default(); + let ep1 = { + let secret = SecretKey::generate(); + let disco = disco_shared.create_discovery(secret.public()); + new_endpoint(secret, disco).await + }; + let ep2 = { + let secret = SecretKey::generate(); + let disco1 = disco_shared.create_lying_discovery(secret.public()); + let mut disco = CombinedDiscovery::new(); + disco.add(disco1); + new_endpoint(secret, disco).await + }; + let ep1_addr = NodeAddr::new(ep1.node_id()); + // wait for out address to be updated and thus published at least once + ep1.my_addr().await?; + let res = ep2.connect(ep1_addr, TEST_ALPN).await; + assert!(matches!(res, Err(_))); + Ok(()) + } + + async fn new_endpoint(secret: SecretKey, disco: impl Discovery + 'static) -> MagicEndpoint { + MagicEndpoint::builder() + .secret_key(secret) + .discovery(Box::new(disco)) + .derp_mode(DerpMode::Disabled) + .alpns(vec![TEST_ALPN.to_vec()]) + .bind(0) + .await + .unwrap() + } + + fn system_time_now() -> u64 { + SystemTime::now() + .duration_since(SystemTime::UNIX_EPOCH) + .expect("time drift") + .as_micros() as u64 + } +} From 8bd2ea3c1cb35c44c68083b6e68f5de549a14c9e Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 6 Mar 2024 12:04:53 +0100 Subject: [PATCH 04/66] chore: fmt --- iroh-net/src/lib.rs | 2 +- iroh-net/src/magic_endpoint.rs | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/iroh-net/src/lib.rs b/iroh-net/src/lib.rs index d7fc8c8c3fd..95362a2ca18 100644 --- a/iroh-net/src/lib.rs +++ b/iroh-net/src/lib.rs @@ -14,8 +14,8 @@ pub mod config; pub mod defaults; pub mod derp; pub mod dialer; -pub mod discovery; mod disco; +pub mod discovery; mod dns; pub mod magic_endpoint; pub mod magicsock; diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index c71e051fabd..56e39de5121 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -362,7 +362,10 @@ impl MagicEndpoint { }; debug!("no mapping address for {node_id}, resolving via {discovery:?}"); let Some(mut stream) = discovery.resolve(self.clone(), node_id) else { - bail!("no discovery service is able to resolve node {}", node_id.fmt_short()); + bail!( + "no discovery service is able to resolve node {}", + node_id.fmt_short() + ); }; let ep = self.clone(); tokio::task::spawn(async move { From 6008c66547cd04169a7c0456c696668258b9d3f0 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 6 Mar 2024 12:09:49 +0100 Subject: [PATCH 05/66] chore: fix doc link --- iroh-net/src/discovery.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 11bcb33a670..f6bae1f5045 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -25,7 +25,7 @@ pub trait Discovery: std::fmt::Debug + Send + Sync { /// These tasks will be run on the runtime of the [`super::MagicEndpoint`]. fn publish(&self, _info: &AddrInfo) {} - /// Resolve the [`AddrInfo`] for the given [`PublicKey`]. + /// Resolve the [`AddrInfo`] for the given [`NodeId`]. /// /// Once the returned [`BoxStream`] is dropped, the service should stop any pending /// work. From 5a0fd66b61ebcda38c68ce9ca6a3d501f9e55bc9 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 6 Mar 2024 12:17:29 +0100 Subject: [PATCH 06/66] chore: clippy --- iroh-net/src/discovery.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index f6bae1f5045..7bed6be0c8e 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -323,7 +323,7 @@ mod tests { // wait for out address to be updated and thus published at least once ep1.my_addr().await?; let res = ep2.connect(ep1_addr, TEST_ALPN).await; - assert!(matches!(res, Err(_))); + assert!(res.is_err()); Ok(()) } From 4c81604a11ffe56fba02e7effe784813f36deb99 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 01:52:40 +0100 Subject: [PATCH 07/66] refactor: better controls for discovery tasks --- iroh-net/src/discovery.rs | 166 +++++++++++++++++++- iroh-net/src/magic_endpoint.rs | 114 ++++---------- iroh-net/src/magicsock/node_map/endpoint.rs | 11 ++ 3 files changed, 203 insertions(+), 88 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 7bed6be0c8e..ec4c68c85e2 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -1,7 +1,12 @@ //! Trait and utils for the node discovery mechanism. -use anyhow::Result; -use futures::stream::BoxStream; +use std::time::Duration; + +use anyhow::{anyhow, Result}; +use futures::{stream::BoxStream, StreamExt}; +use iroh_base::node_addr::NodeAddr; +use tokio::sync::oneshot; +use tracing::debug; use crate::{AddrInfo, MagicEndpoint, NodeId}; @@ -102,6 +107,133 @@ impl Discovery for CombinedDiscovery { } } +/// Threshold for the duration since we last received control or data from an endpoint to make us +/// start a discovery. +const MAX_AGE: Duration = Duration::from_secs(10); + +/// A wrapper around a tokio task which runs a node discovery. +pub(super) struct DiscoveryTask { + on_first_result: oneshot::Receiver>, + task: tokio::task::JoinHandle<()>, +} + +impl DiscoveryTask { + /// Start a discovery task. + pub fn start(ep: &MagicEndpoint, node_id: NodeId) -> Result { + let stream = Self::create_stream(&ep, node_id)?; + let (tx, rx) = oneshot::channel(); + let ep = ep.clone(); + let task = tokio::task::spawn(async move { Self::run(&ep, node_id, stream, tx).await }); + Ok(Self { + task, + on_first_result: rx, + }) + } + + /// Start a discovery task after a delay and only if no path to the node was recently active. + pub async fn maybe_start( + ep: &MagicEndpoint, + node_id: NodeId, + delay: Option, + ) -> Result> { + if !Self::needs_discovery(ep, node_id).await { + return Ok(None); + } + let stream = Self::create_stream(ep, node_id)?; + let (tx, rx) = oneshot::channel(); + let ep = ep.clone(); + let task = tokio::task::spawn(async move { + if let Some(delay) = delay { + tokio::time::sleep(delay).await; + if !Self::needs_discovery(&ep, node_id).await { + return; + } + } + Self::run(&ep, node_id, stream, tx).await + }); + Ok(Some(Self { + task, + on_first_result: rx, + })) + } + + pub async fn first_arrived(&mut self) -> Result<()> { + let fut = &mut self.on_first_result; + fut.await??; + Ok(()) + } + + pub fn cancel(&self) { + self.task.abort(); + } + + fn create_stream( + ep: &MagicEndpoint, + node_id: NodeId, + ) -> Result>> { + let discovery = ep + .discovery() + .ok_or_else(|| anyhow!("No discovery service configured"))?; + let stream = discovery + .resolve(ep.clone(), node_id) + .ok_or_else(|| anyhow!("no discovery service can resolve node {node_id}",))?; + Ok(stream) + } + + async fn needs_discovery(ep: &MagicEndpoint, node_id: NodeId) -> bool { + match ep.connection_info(node_id).await { + Err(_) | Ok(None) => true, + Ok(Some(info)) => match info.last_received() { + None => true, + Some(elapsed) => elapsed > MAX_AGE, + }, + } + } + + async fn run( + ep: &MagicEndpoint, + node_id: NodeId, + mut stream: BoxStream<'static, Result>, + on_first_result: oneshot::Sender>, + ) { + let mut on_first_result = Some(on_first_result); + loop { + let next = tokio::select! { + _ = ep.cancelled() => break, + next = stream.next() => next + }; + match next { + Some(Ok(r)) => { + debug!(node = %node_id.fmt_short(), provenance = r.provenance, addr = ?r.addr_info, "discovery: new address found"); + let addr = NodeAddr { + info: r.addr_info, + node_id, + }; + ep.add_node_addr(addr).ok(); + if let Some(tx) = on_first_result.take() { + tx.send(Ok(())).ok(); + } + } + Some(Err(err)) => { + tracing::warn!(?err, "discovery service failed"); + break; + } + None => break, + } + } + if let Some(tx) = on_first_result.take() { + tx.send(Err(anyhow!("No results for {}", node_id.fmt_short()))) + .ok(); + } + } +} + +impl Drop for DiscoveryTask { + fn drop(&mut self) { + self.task.abort(); + } +} + #[cfg(test)] mod tests { use std::{ @@ -327,6 +459,36 @@ mod tests { Ok(()) } + /// This test first adds a wrong address manually (e.g. from an outdated ticket). + #[tokio::test] + async fn magic_endpoint_discovery_with_wrong_existing_addr() -> anyhow::Result<()> { + let _guard = iroh_test::logging::setup(); + let disco_shared = TestDiscoveryShared::default(); + let ep1 = { + let secret = SecretKey::generate(); + let disco = disco_shared.create_discovery(secret.public()); + new_endpoint(secret, disco).await + }; + let ep2 = { + let secret = SecretKey::generate(); + let disco = disco_shared.create_discovery(secret.public()); + new_endpoint(secret, disco).await + }; + let ep1_addr = NodeAddr::new(ep1.node_id()); + // wait for out address to be updated and thus published at least once + ep1.my_addr().await?; + let wrong_addr = NodeAddr { + node_id: ep1.node_id(), + info: AddrInfo { + derp_url: None, + direct_addresses: BTreeSet::from(["240.0.0.1:1000".parse().unwrap()]), + }, + }; + ep2.add_node_addr(wrong_addr)?; + let _conn = ep2.connect(ep1_addr, TEST_ALPN).await?; + Ok(()) + } + async fn new_endpoint(secret: SecretKey, disco: impl Discovery + 'static) -> MagicEndpoint { MagicEndpoint::builder() .secret_key(secret) diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index 56e39de5121..98be6ec2a6c 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -6,15 +6,14 @@ use anyhow::{anyhow, bail, ensure, Context, Result}; use derive_more::Debug; use futures::StreamExt; use quinn_proto::VarInt; -use tokio::sync::oneshot; -use tokio_util::sync::CancellationToken; -use tracing::{debug, trace, warn}; +use tokio_util::sync::{CancellationToken, WaitForCancellationFuture}; +use tracing::{debug, trace}; use crate::{ config, defaults::default_derp_map, derp::{DerpMap, DerpMode, DerpUrl}, - discovery::Discovery, + discovery::{Discovery, DiscoveryTask}, key::{PublicKey, SecretKey}, magicsock::{self, MagicSock}, tls, NodeId, @@ -24,6 +23,8 @@ pub use super::magicsock::{EndpointInfo as ConnectionInfo, LocalEndpointsStream} pub use iroh_base::node_addr::{AddrInfo, NodeAddr}; +const DISCOVERY_WAIT_PERIOD: Duration = Duration::from_millis(500); + /// Builder for [MagicEndpoint] #[derive(Debug)] pub struct MagicEndpointBuilder { @@ -340,74 +341,8 @@ impl MagicEndpoint { self.msock.tracked_endpoint(node_id) } - async fn resolve_until_first( - &self, - node_id: PublicKey, - cancel: CancellationToken, - ) -> anyhow::Result<()> { - let (on_first_result_tx, on_first_result_rx) = oneshot::channel(); - self.resolve_inner(node_id, on_first_result_tx, cancel)?; - on_first_result_rx.await??; - Ok(()) - } - - fn resolve_inner( - &self, - node_id: PublicKey, - on_first_result: oneshot::Sender>, - cancel: CancellationToken, - ) -> Result<()> { - let Some(discovery) = self.msock.discovery() else { - bail!("no discovery mechanism configured"); - }; - debug!("no mapping address for {node_id}, resolving via {discovery:?}"); - let Some(mut stream) = discovery.resolve(self.clone(), node_id) else { - bail!( - "no discovery service is able to resolve node {}", - node_id.fmt_short() - ); - }; - let ep = self.clone(); - tokio::task::spawn(async move { - let mut on_first_result = Some(on_first_result); - loop { - let next = tokio::select! { - // cancel on endpoint close - _ = ep.cancel_token.cancelled() => break, - // cancel on explicit cancel request - _ = cancel.cancelled() => break, - res = stream.next() => match res { - None => break, - Some(res) => res - } - }; - match next { - Ok(r) => { - debug!(node = %node_id.fmt_short(), provenance = r.provenance, addr = ?r.addr_info, "discovery: new address found"); - let addr = NodeAddr { - info: r.addr_info, - node_id, - }; - ep.add_node_addr(addr).ok(); - if let Some(tx) = on_first_result.take() { - tx.send(Ok(())).ok(); - } - } - Err(err) => { - warn!(?err, "discovery service failed"); - if let Some(tx) = on_first_result.take() { - tx.send(Err(err)).ok(); - } - break; - } - } - } - if let Some(tx) = on_first_result.take() { - tx.send(Err(anyhow!("No results for {}", node_id.fmt_short()))) - .ok(); - } - }); - Ok(()) + pub(crate) fn cancelled(&self) -> WaitForCancellationFuture<'_> { + self.cancel_token.cancelled() } /// Connect to a remote endpoint, using just the nodes's [`PublicKey`]. @@ -437,17 +372,22 @@ impl MagicEndpoint { let NodeAddr { node_id, info } = node_addr; - // Store a cancel token if we start a discovery for the node id. - let mut cancel_discovery = None; - let addr = match self.msock.get_mapping_addr(&node_id) { - Some(addr) => addr, + let (addr, discovery) = match self.msock.get_mapping_addr(&node_id) { + Some(addr) => { + // If the passed node_addr contains derp or direct addresses, apply a delay (to + // test these addresses) before starting a discovery. + let delay = (!info.is_empty()).then_some(DISCOVERY_WAIT_PERIOD); + let discovery = DiscoveryTask::maybe_start(self, node_id, delay).await?; + (addr, discovery) + } + None => { - let cancel_token = CancellationToken::new(); - cancel_discovery = Some(cancel_token.clone()); - self.resolve_until_first(node_id, cancel_token).await?; - self.msock.get_mapping_addr(&node_id).await.ok_or_else(|| { + let mut discovery = DiscoveryTask::start(self, node_id)?; + discovery.first_arrived().await?; + let addr = self.msock.get_mapping_addr(&node_id).ok_or_else(|| { anyhow!("Failed to retrieve the mapped address from the magic socket. Unable to dial node {node_id:?}") - })? + })?; + (addr, Some(discovery)) } }; @@ -456,14 +396,16 @@ impl MagicEndpoint { node_id, addr, info.direct_addresses ); - let conn = self.connect_quinn(&node_id, alpn, addr).await?; + // Start connecting via quinn. This will time out after 10 seconds if no reachable address + // is available. + let conn = self.connect_quinn(&node_id, alpn, addr).await; - // Connection was successfull, cancel the node discovery task (if still running). - if let Some(cancel_token) = cancel_discovery { - cancel_token.cancel(); + // Cancel the node discovery task (if still running). + if let Some(discovery) = discovery { + discovery.cancel(); } - Ok(conn) + conn } async fn connect_quinn( diff --git a/iroh-net/src/magicsock/node_map/endpoint.rs b/iroh-net/src/magicsock/node_map/endpoint.rs index b05d98d3bba..510a994da6e 100644 --- a/iroh-net/src/magicsock/node_map/endpoint.rs +++ b/iroh-net/src/magicsock/node_map/endpoint.rs @@ -1350,6 +1350,17 @@ pub struct EndpointInfo { pub last_used: Option, } +impl EndpointInfo { + /// Get the duration since the last activity we received from this endpoint + /// on any of its direct addresses. + pub fn last_received(&self) -> Option { + self.addrs + .iter() + .filter_map(|addr| addr.last_control.map(|x| x.0).min(addr.last_payload)) + .min() + } +} + /// The type of connection we have to the endpoint. #[derive(derive_more::Display, Debug, Clone, Eq, PartialEq, Serialize, Deserialize)] pub enum ConnectionType { From 19e9ae7d53bf3c12c4bb9928481ea1cbac682baf Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 11:08:47 +0100 Subject: [PATCH 08/66] cleanup --- iroh-net/src/discovery.rs | 14 +++++++------- iroh-net/src/magic_endpoint.rs | 4 +++- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index ec4c68c85e2..343cdf91b42 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -131,12 +131,12 @@ impl DiscoveryTask { } /// Start a discovery task after a delay and only if no path to the node was recently active. - pub async fn maybe_start( + pub fn maybe_start( ep: &MagicEndpoint, node_id: NodeId, delay: Option, ) -> Result> { - if !Self::needs_discovery(ep, node_id).await { + if !Self::needs_discovery(ep, node_id) { return Ok(None); } let stream = Self::create_stream(ep, node_id)?; @@ -145,7 +145,7 @@ impl DiscoveryTask { let task = tokio::task::spawn(async move { if let Some(delay) = delay { tokio::time::sleep(delay).await; - if !Self::needs_discovery(&ep, node_id).await { + if !Self::needs_discovery(&ep, node_id) { return; } } @@ -180,10 +180,10 @@ impl DiscoveryTask { Ok(stream) } - async fn needs_discovery(ep: &MagicEndpoint, node_id: NodeId) -> bool { - match ep.connection_info(node_id).await { - Err(_) | Ok(None) => true, - Ok(Some(info)) => match info.last_received() { + fn needs_discovery(ep: &MagicEndpoint, node_id: NodeId) -> bool { + match ep.connection_info(node_id) { + None => true, + Some(info) => match info.last_received() { None => true, Some(elapsed) => elapsed > MAX_AGE, }, diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index 98be6ec2a6c..dcb92da9595 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -377,7 +377,9 @@ impl MagicEndpoint { // If the passed node_addr contains derp or direct addresses, apply a delay (to // test these addresses) before starting a discovery. let delay = (!info.is_empty()).then_some(DISCOVERY_WAIT_PERIOD); - let discovery = DiscoveryTask::maybe_start(self, node_id, delay).await?; + let discovery = DiscoveryTask::maybe_start(self, node_id, delay) + .ok() + .flatten(); (addr, discovery) } From 2db91c8005bc4d7d3913a291d69d7fad5db8dd31 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 11:45:09 +0100 Subject: [PATCH 09/66] docs and cleanup --- iroh-net/src/discovery.rs | 80 +++++++++++++++++++--------------- iroh-net/src/magic_endpoint.rs | 44 +++++++++++++++---- 2 files changed, 80 insertions(+), 44 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 343cdf91b42..7fe0ca509ca 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -6,19 +6,19 @@ use anyhow::{anyhow, Result}; use futures::{stream::BoxStream, StreamExt}; use iroh_base::node_addr::NodeAddr; use tokio::sync::oneshot; -use tracing::debug; +use tracing::{debug, warn}; use crate::{AddrInfo, MagicEndpoint, NodeId}; /// Node discovery for [`super::MagicEndpoint`]. /// -/// The purpose of this trait is to hoop up a node discovery mechanism that -/// allows finding information such as the derp url and current addresses -/// of a node given the id. +/// The purpose of this trait is to hook up a node discovery mechanism that +/// allows finding information such as the Derp URL and direct addresses +/// of a node given its [`NodeId`]. /// /// To allow for discovery, the [`super::MagicEndpoint`] will call `publish` whenever /// discovery information changes. If a discovery mechanism requires a periodic -/// refresh, it should start it's own task. +/// refresh, it should start its own task. pub trait Discovery: std::fmt::Debug + Send + Sync { /// Publish the given [`AddrInfo`] to the discovery mechanisms. /// @@ -107,13 +107,13 @@ impl Discovery for CombinedDiscovery { } } -/// Threshold for the duration since we last received control or data from an endpoint to make us -/// start a discovery. +/// Maximum duration since the last control or data message received from an endpoint to make us +/// start a discovery task. const MAX_AGE: Duration = Duration::from_secs(10); /// A wrapper around a tokio task which runs a node discovery. pub(super) struct DiscoveryTask { - on_first_result: oneshot::Receiver>, + on_first_rx: oneshot::Receiver>, task: tokio::task::JoinHandle<()>, } @@ -121,48 +121,55 @@ impl DiscoveryTask { /// Start a discovery task. pub fn start(ep: &MagicEndpoint, node_id: NodeId) -> Result { let stream = Self::create_stream(&ep, node_id)?; - let (tx, rx) = oneshot::channel(); + let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); - let task = tokio::task::spawn(async move { Self::run(&ep, node_id, stream, tx).await }); - Ok(Self { - task, - on_first_result: rx, - }) + let task = + tokio::task::spawn(async move { Self::run(&ep, node_id, stream, on_first_tx).await }); + Ok(Self { task, on_first_rx }) } /// Start a discovery task after a delay and only if no path to the node was recently active. - pub fn maybe_start( + /// + /// This returns `None` if we received data or control messages from the remote endpoint + /// recently enough. If not it returns a [`DiscoveryTask`]. + /// + /// If `delay` is set, the [`DiscoveryTask`] will first wait for `delay` and then check again + /// if we recently received messages from remote endpoint. If true, the task will abort. + /// Otherwise, or if no `delay` is set, the discovery will be started. + pub fn maybe_start_after_delay( ep: &MagicEndpoint, node_id: NodeId, delay: Option, ) -> Result> { + // If discovery is not needed, don't even spawn a task. if !Self::needs_discovery(ep, node_id) { return Ok(None); } let stream = Self::create_stream(ep, node_id)?; - let (tx, rx) = oneshot::channel(); + let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); let task = tokio::task::spawn(async move { + // If delay is set, wait and recheck if discovery is needed. If not, early-exit. if let Some(delay) = delay { tokio::time::sleep(delay).await; if !Self::needs_discovery(&ep, node_id) { + on_first_tx.send(Ok(())).ok(); return; } } - Self::run(&ep, node_id, stream, tx).await + Self::run(&ep, node_id, stream, on_first_tx).await }); - Ok(Some(Self { - task, - on_first_result: rx, - })) + Ok(Some(Self { task, on_first_rx })) } + /// Wait until the discovery task produced at least one result. pub async fn first_arrived(&mut self) -> Result<()> { - let fut = &mut self.on_first_result; + let fut = &mut self.on_first_rx; fut.await??; Ok(()) } + /// Cancel the discovery task. pub fn cancel(&self) { self.task.abort(); } @@ -182,9 +189,12 @@ impl DiscoveryTask { fn needs_discovery(ep: &MagicEndpoint, node_id: NodeId) -> bool { match ep.connection_info(node_id) { + // No connection info means no path to node -> start discovery. None => true, Some(info) => match info.last_received() { + // No path to node -> start discovery. None => true, + // If we haven't received for MAX_AGE, start discovery. Some(elapsed) => elapsed > MAX_AGE, }, } @@ -194,9 +204,9 @@ impl DiscoveryTask { ep: &MagicEndpoint, node_id: NodeId, mut stream: BoxStream<'static, Result>, - on_first_result: oneshot::Sender>, + on_first_tx: oneshot::Sender>, ) { - let mut on_first_result = Some(on_first_result); + let mut on_first_tx = Some(on_first_tx); loop { let next = tokio::select! { _ = ep.cancelled() => break, @@ -210,20 +220,20 @@ impl DiscoveryTask { node_id, }; ep.add_node_addr(addr).ok(); - if let Some(tx) = on_first_result.take() { + if let Some(tx) = on_first_tx.take() { tx.send(Ok(())).ok(); } } Some(Err(err)) => { - tracing::warn!(?err, "discovery service failed"); + warn!(?err, "discovery service produced error"); break; } None => break, } } - if let Some(tx) = on_first_result.take() { - tx.send(Err(anyhow!("No results for {}", node_id.fmt_short()))) - .ok(); + if let Some(tx) = on_first_tx.take() { + let err = anyhow!("Discovery produced no results for {}", node_id.fmt_short()); + tx.send(Err(err)).ok(); } } } @@ -245,6 +255,7 @@ mod tests { use futures::{stream, StreamExt}; use parking_lot::Mutex; + use rand::Rng; use crate::{derp::DerpMode, key::SecretKey, NodeAddr}; @@ -305,7 +316,7 @@ mod tests { false => self.shared.nodes.lock().get(&node_id).cloned(), true => { let ts = system_time_now() - 100_000; - let port: u16 = rand::random(); + let port: u16 = rand::thread_rng().gen_range(10_000..20_000); // "240.0.0.0/4" is reserved and unreachable let addr: SocketAddr = format!("240.0.0.1:{port}").parse().unwrap(); let addr_info = AddrInfo { @@ -459,7 +470,8 @@ mod tests { Ok(()) } - /// This test first adds a wrong address manually (e.g. from an outdated ticket). + /// This test first adds a wrong address manually (e.g. from an outdated&node_id ticket). + /// Connect should still succeed because the discovery service will be invoked (after a delay). #[tokio::test] async fn magic_endpoint_discovery_with_wrong_existing_addr() -> anyhow::Result<()> { let _guard = iroh_test::logging::setup(); @@ -474,18 +486,16 @@ mod tests { let disco = disco_shared.create_discovery(secret.public()); new_endpoint(secret, disco).await }; - let ep1_addr = NodeAddr::new(ep1.node_id()); // wait for out address to be updated and thus published at least once ep1.my_addr().await?; - let wrong_addr = NodeAddr { + let ep1_wrong_addr = NodeAddr { node_id: ep1.node_id(), info: AddrInfo { derp_url: None, direct_addresses: BTreeSet::from(["240.0.0.1:1000".parse().unwrap()]), }, }; - ep2.add_node_addr(wrong_addr)?; - let _conn = ep2.connect(ep1_addr, TEST_ALPN).await?; + let _conn = ep2.connect(ep1_wrong_addr, TEST_ALPN).await?; Ok(()) } diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index dcb92da9595..94355b16a6b 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -126,6 +126,14 @@ impl MagicEndpointBuilder { } /// Optionally set a discovery mechanism for this endpoint. + /// + /// If you want to combine multiple discovery services, you can pass a + /// [`discovery::CombinedDiscovery`]. + /// + /// If no discovery service is set, connecting to a node without providing its + /// direct addresses or Derp URLs will fail. + /// + /// See the documentation of the [`Discovery`] trait for details. pub fn discovery(mut self, discovery: Box) -> Self { self.discovery = Some(discovery); self @@ -357,14 +365,21 @@ impl MagicEndpoint { /// Connect to a remote endpoint. /// - /// The PublicKey and the ALPN protocol are required. If you happen to know dialable addresses of - /// the remote endpoint, they can be specified and will be used to try and establish a direct - /// connection without involving a DERP server. If no addresses are specified, the endpoint - /// will try to dial the peer through the configured DERP servers. + /// A [`NodeAddr`] is required. It must contain the [`NodeId`] to dial and may also contain a + /// Derp URL and direct addresses. If direct addresses are provided, they will be used to + /// try and establish a direct connection without involving a Derp server. + /// + /// The `alpn`, or application-level protocol identifier, is also required. The remote endpoint + /// must support this `alpn`, otherwise the connection attempt will fail with an error. /// - /// If the `derp_url` is not `None` and the configured DERP servers do not include a DERP node from the given `derp_url`, it will error. + /// If the [`NodeAddr`] contains only [`NodeId`] and no direct addresses and no Derp servers, + /// a discovery service will be invoked, if configured, to try and discover the node's + /// addressing information. The discovery services must be configured globally per [`MagicEndpoint`] + /// with [`MagicEndpointBuilder::discovery`]. The discovery service will also be invoked if + /// none of the existing or provided direct addresses are reachable. /// - /// If no UDP addresses and no DERP Url is provided, it will error. + /// If addresses or Derp servers are neither provided nor can be discovered, the connection + /// attempt will fail with an error. pub async fn connect(&self, node_addr: NodeAddr, alpn: &[u8]) -> Result { if !node_addr.info.is_empty() { self.add_node_addr(node_addr.clone())?; @@ -372,18 +387,29 @@ impl MagicEndpoint { let NodeAddr { node_id, info } = node_addr; + // Get the mapped IPv6 addres from the magic socket. Quinn will connect to this address. let (addr, discovery) = match self.msock.get_mapping_addr(&node_id) { Some(addr) => { - // If the passed node_addr contains derp or direct addresses, apply a delay (to - // test these addresses) before starting a discovery. + // We got a mapped address, which means we either spoke to this endpoint before, or + // the user provided addressing info with the [`NodeAddr`]. + // This does not mean that we can actually connect to any of these addresses. + // Therefore, we will invoke the discovery service if we haven't received from the + // endpoint on any of the existing paths recently. + // If the user provided addresses in this connect call, we will add a delay + // followed by a recheck before starting the discovery, to give the magicsocket a + // chance to test the newly provided addresses. let delay = (!info.is_empty()).then_some(DISCOVERY_WAIT_PERIOD); - let discovery = DiscoveryTask::maybe_start(self, node_id, delay) + let discovery = DiscoveryTask::maybe_start_after_delay(self, node_id, delay) .ok() .flatten(); (addr, discovery) } None => { + // We have not spoken to this endpoint before, and the user provided no direct + // addresses or Derp URLs. Thus, we start a discovery task and wait for the first + // result to arrive, and only then continue, because otherwise we wouldn't have any + // path to the remote endpoint. let mut discovery = DiscoveryTask::start(self, node_id)?; discovery.first_arrived().await?; let addr = self.msock.get_mapping_addr(&node_id).ok_or_else(|| { From ed52cedf4993e2beacb96a882011c7adc0d4b56b Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 11:48:05 +0100 Subject: [PATCH 10/66] more docs --- iroh-net/src/magic_endpoint.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index 94355b16a6b..0c5303fc09e 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -23,6 +23,8 @@ pub use super::magicsock::{EndpointInfo as ConnectionInfo, LocalEndpointsStream} pub use iroh_base::node_addr::{AddrInfo, NodeAddr}; +/// The delay we add before starting a discovery in [`MagicEndpoint::connect`] if the user provided +/// new direct addresses (to try these addresses before starting the discovery). const DISCOVERY_WAIT_PERIOD: Duration = Duration::from_millis(500); /// Builder for [MagicEndpoint] From 25c847b2fec476c2530a26d44e9f55e722b274fa Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 11:56:10 +0100 Subject: [PATCH 11/66] chore: clippy & doclinks --- iroh-net/src/discovery.rs | 2 +- iroh-net/src/magic_endpoint.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 7fe0ca509ca..573203464c0 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -120,7 +120,7 @@ pub(super) struct DiscoveryTask { impl DiscoveryTask { /// Start a discovery task. pub fn start(ep: &MagicEndpoint, node_id: NodeId) -> Result { - let stream = Self::create_stream(&ep, node_id)?; + let stream = Self::create_stream(ep, node_id)?; let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); let task = diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index 0c5303fc09e..dde375c035c 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -130,7 +130,7 @@ impl MagicEndpointBuilder { /// Optionally set a discovery mechanism for this endpoint. /// /// If you want to combine multiple discovery services, you can pass a - /// [`discovery::CombinedDiscovery`]. + /// [`crate::discovery::CombinedDiscovery`]. /// /// If no discovery service is set, connecting to a node without providing its /// direct addresses or Derp URLs will fail. From b5f97281fc025610e65955cf4bd9f2cf45d7da77 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 12:23:10 +0100 Subject: [PATCH 12/66] better tracing logs --- iroh-net/src/discovery.rs | 30 ++++++++++++++++++++---------- iroh-net/src/magic_endpoint.rs | 2 +- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 573203464c0..ca0678677ed 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -5,8 +5,8 @@ use std::time::Duration; use anyhow::{anyhow, Result}; use futures::{stream::BoxStream, StreamExt}; use iroh_base::node_addr::NodeAddr; -use tokio::sync::oneshot; -use tracing::{debug, warn}; +use tokio::{sync::oneshot, task::JoinHandle}; +use tracing::{debug, error_span, instrument::Instrumented, warn, Instrument}; use crate::{AddrInfo, MagicEndpoint, NodeId}; @@ -114,7 +114,7 @@ const MAX_AGE: Duration = Duration::from_secs(10); /// A wrapper around a tokio task which runs a node discovery. pub(super) struct DiscoveryTask { on_first_rx: oneshot::Receiver>, - task: tokio::task::JoinHandle<()>, + task: Instrumented>, } impl DiscoveryTask { @@ -124,7 +124,8 @@ impl DiscoveryTask { let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); let task = - tokio::task::spawn(async move { Self::run(&ep, node_id, stream, on_first_tx).await }); + tokio::task::spawn(async move { Self::run(&ep, node_id, stream, on_first_tx).await }) + .instrument(error_span!("discovery", node = %node_id.fmt_short())); Ok(Self { task, on_first_rx }) } @@ -145,7 +146,6 @@ impl DiscoveryTask { if !Self::needs_discovery(ep, node_id) { return Ok(None); } - let stream = Self::create_stream(ep, node_id)?; let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); let task = tokio::task::spawn(async move { @@ -153,12 +153,21 @@ impl DiscoveryTask { if let Some(delay) = delay { tokio::time::sleep(delay).await; if !Self::needs_discovery(&ep, node_id) { + debug!("no discovery needed, abort"); on_first_tx.send(Ok(())).ok(); return; } } + let stream = match Self::create_stream(&ep, node_id) { + Ok(stream) => stream, + Err(err) => { + on_first_tx.send(Err(err)).ok(); + return; + } + }; Self::run(&ep, node_id, stream, on_first_tx).await - }); + }) + .instrument(error_span!("discovery", node = %node_id.fmt_short())); Ok(Some(Self { task, on_first_rx })) } @@ -171,7 +180,7 @@ impl DiscoveryTask { /// Cancel the discovery task. pub fn cancel(&self) { - self.task.abort(); + self.task.inner().abort(); } fn create_stream( @@ -183,7 +192,7 @@ impl DiscoveryTask { .ok_or_else(|| anyhow!("No discovery service configured"))?; let stream = discovery .resolve(ep.clone(), node_id) - .ok_or_else(|| anyhow!("no discovery service can resolve node {node_id}",))?; + .ok_or_else(|| anyhow!("No discovery service can resolve node {node_id}",))?; Ok(stream) } @@ -207,6 +216,7 @@ impl DiscoveryTask { on_first_tx: oneshot::Sender>, ) { let mut on_first_tx = Some(on_first_tx); + debug!("discovery: start"); loop { let next = tokio::select! { _ = ep.cancelled() => break, @@ -214,7 +224,7 @@ impl DiscoveryTask { }; match next { Some(Ok(r)) => { - debug!(node = %node_id.fmt_short(), provenance = r.provenance, addr = ?r.addr_info, "discovery: new address found"); + debug!(provenance = %r.provenance, addr = ?r.addr_info, "discovery: new address found"); let addr = NodeAddr { info: r.addr_info, node_id, @@ -240,7 +250,7 @@ impl DiscoveryTask { impl Drop for DiscoveryTask { fn drop(&mut self) { - self.task.abort(); + self.task.inner().abort(); } } diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index dde375c035c..330d69f7324 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -389,7 +389,7 @@ impl MagicEndpoint { let NodeAddr { node_id, info } = node_addr; - // Get the mapped IPv6 addres from the magic socket. Quinn will connect to this address. + // Get the mapped IPv6 address from the magic socket. Quinn will connect to this address. let (addr, discovery) = match self.msock.get_mapping_addr(&node_id) { Some(addr) => { // We got a mapped address, which means we either spoke to this endpoint before, or From adbc41ff0b6004cef2737bbde53a10d29475669d Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 12:25:44 +0100 Subject: [PATCH 13/66] fix instrumentation --- iroh-net/src/discovery.rs | 58 ++++++++++++++++++++++----------------- 1 file changed, 33 insertions(+), 25 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index ca0678677ed..5931b4e9017 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -6,7 +6,7 @@ use anyhow::{anyhow, Result}; use futures::{stream::BoxStream, StreamExt}; use iroh_base::node_addr::NodeAddr; use tokio::{sync::oneshot, task::JoinHandle}; -use tracing::{debug, error_span, instrument::Instrumented, warn, Instrument}; +use tracing::{debug, error_span, warn, Instrument}; use crate::{AddrInfo, MagicEndpoint, NodeId}; @@ -114,7 +114,7 @@ const MAX_AGE: Duration = Duration::from_secs(10); /// A wrapper around a tokio task which runs a node discovery. pub(super) struct DiscoveryTask { on_first_rx: oneshot::Receiver>, - task: Instrumented>, + task: JoinHandle<()>, } impl DiscoveryTask { @@ -123,9 +123,12 @@ impl DiscoveryTask { let stream = Self::create_stream(ep, node_id)?; let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); - let task = - tokio::task::spawn(async move { Self::run(&ep, node_id, stream, on_first_tx).await }) - .instrument(error_span!("discovery", node = %node_id.fmt_short())); + let me = ep.node_id(); + let task = tokio::task::spawn( + async move { Self::run(&ep, node_id, stream, on_first_tx).await }.instrument( + error_span!("discovery", me = %me.fmt_short(), node = %node_id.fmt_short()), + ), + ); Ok(Self { task, on_first_rx }) } @@ -148,26 +151,31 @@ impl DiscoveryTask { } let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); - let task = tokio::task::spawn(async move { - // If delay is set, wait and recheck if discovery is needed. If not, early-exit. - if let Some(delay) = delay { - tokio::time::sleep(delay).await; - if !Self::needs_discovery(&ep, node_id) { - debug!("no discovery needed, abort"); - on_first_tx.send(Ok(())).ok(); - return; + let me = ep.node_id(); + let task = tokio::task::spawn( + async move { + // If delay is set, wait and recheck if discovery is needed. If not, early-exit. + if let Some(delay) = delay { + tokio::time::sleep(delay).await; + if !Self::needs_discovery(&ep, node_id) { + debug!("no discovery needed, abort"); + on_first_tx.send(Ok(())).ok(); + return; + } } + let stream = match Self::create_stream(&ep, node_id) { + Ok(stream) => stream, + Err(err) => { + on_first_tx.send(Err(err)).ok(); + return; + } + }; + Self::run(&ep, node_id, stream, on_first_tx).await } - let stream = match Self::create_stream(&ep, node_id) { - Ok(stream) => stream, - Err(err) => { - on_first_tx.send(Err(err)).ok(); - return; - } - }; - Self::run(&ep, node_id, stream, on_first_tx).await - }) - .instrument(error_span!("discovery", node = %node_id.fmt_short())); + .instrument( + error_span!("discovery", me = %me.fmt_short(), node = %node_id.fmt_short()), + ), + ); Ok(Some(Self { task, on_first_rx })) } @@ -180,7 +188,7 @@ impl DiscoveryTask { /// Cancel the discovery task. pub fn cancel(&self) { - self.task.inner().abort(); + self.task.abort(); } fn create_stream( @@ -250,7 +258,7 @@ impl DiscoveryTask { impl Drop for DiscoveryTask { fn drop(&mut self) { - self.task.inner().abort(); + self.task.abort(); } } From 9d62fea35e9e610850bf2d29e5030eb9a7a5b735 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 12:56:43 +0100 Subject: [PATCH 14/66] refactor: do not require static lifetime on discovery streams --- iroh-net/src/discovery.rs | 67 +++++++++++++++++----------------- iroh-net/src/magic_endpoint.rs | 2 +- 2 files changed, 34 insertions(+), 35 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 5931b4e9017..a780c1977a2 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -2,7 +2,7 @@ use std::time::Duration; -use anyhow::{anyhow, Result}; +use anyhow::{anyhow, bail, Result}; use futures::{stream::BoxStream, StreamExt}; use iroh_base::node_addr::NodeAddr; use tokio::{sync::oneshot, task::JoinHandle}; @@ -34,11 +34,11 @@ pub trait Discovery: std::fmt::Debug + Send + Sync { /// /// Once the returned [`BoxStream`] is dropped, the service should stop any pending /// work. - fn resolve( - &self, + fn resolve<'a>( + &'a self, _endpoint: MagicEndpoint, _node_id: NodeId, - ) -> Option>> { + ) -> Option>> { None } } @@ -93,11 +93,11 @@ impl Discovery for CombinedDiscovery { } } - fn resolve( - &self, + fn resolve<'a>( + &'a self, endpoint: MagicEndpoint, node_id: NodeId, - ) -> Option>> { + ) -> Option>> { let streams = self .services .iter() @@ -119,13 +119,14 @@ pub(super) struct DiscoveryTask { impl DiscoveryTask { /// Start a discovery task. - pub fn start(ep: &MagicEndpoint, node_id: NodeId) -> Result { - let stream = Self::create_stream(ep, node_id)?; + pub fn start(ep: MagicEndpoint, node_id: NodeId) -> Result { + if ep.discovery().is_none() { + bail!("No discovery services configured"); + } let (on_first_tx, on_first_rx) = oneshot::channel(); - let ep = ep.clone(); let me = ep.node_id(); let task = tokio::task::spawn( - async move { Self::run(&ep, node_id, stream, on_first_tx).await }.instrument( + async move { Self::run(ep, node_id, on_first_tx).await }.instrument( error_span!("discovery", me = %me.fmt_short(), node = %node_id.fmt_short()), ), ); @@ -149,6 +150,9 @@ impl DiscoveryTask { if !Self::needs_discovery(ep, node_id) { return Ok(None); } + if ep.discovery().is_none() { + bail!("No discovery services configured"); + } let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); let me = ep.node_id(); @@ -163,14 +167,7 @@ impl DiscoveryTask { return; } } - let stream = match Self::create_stream(&ep, node_id) { - Ok(stream) => stream, - Err(err) => { - on_first_tx.send(Err(err)).ok(); - return; - } - }; - Self::run(&ep, node_id, stream, on_first_tx).await + Self::run(ep, node_id, on_first_tx).await } .instrument( error_span!("discovery", me = %me.fmt_short(), node = %node_id.fmt_short()), @@ -191,10 +188,10 @@ impl DiscoveryTask { self.task.abort(); } - fn create_stream( - ep: &MagicEndpoint, + fn create_stream<'a>( + ep: &'a MagicEndpoint, node_id: NodeId, - ) -> Result>> { + ) -> Result>> { let discovery = ep .discovery() .ok_or_else(|| anyhow!("No discovery service configured"))?; @@ -217,12 +214,14 @@ impl DiscoveryTask { } } - async fn run( - ep: &MagicEndpoint, - node_id: NodeId, - mut stream: BoxStream<'static, Result>, - on_first_tx: oneshot::Sender>, - ) { + async fn run(ep: MagicEndpoint, node_id: NodeId, on_first_tx: oneshot::Sender>) { + let mut stream = match Self::create_stream(&ep, node_id) { + Ok(stream) => stream, + Err(err) => { + on_first_tx.send(Err(err)).ok(); + return; + } + }; let mut on_first_tx = Some(on_first_tx); debug!("discovery: start"); loop { @@ -325,11 +324,11 @@ mod tests { .insert(self.node_id, (info.clone(), now)); } - fn resolve( - &self, + fn resolve<'a>( + &'a self, endpoint: MagicEndpoint, node_id: NodeId, - ) -> Option>> { + ) -> Option>> { let addr_info = match self.resolve_wrong { false => self.shared.nodes.lock().get(&node_id).cloned(), true => { @@ -374,11 +373,11 @@ mod tests { impl Discovery for EmptyDiscovery { fn publish(&self, _info: &AddrInfo) {} - fn resolve( - &self, + fn resolve<'a>( + &'a self, _endpoint: MagicEndpoint, _node_id: NodeId, - ) -> Option>> { + ) -> Option>> { Some(stream::empty().boxed()) } } diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index 330d69f7324..ba4d18d3315 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -412,7 +412,7 @@ impl MagicEndpoint { // addresses or Derp URLs. Thus, we start a discovery task and wait for the first // result to arrive, and only then continue, because otherwise we wouldn't have any // path to the remote endpoint. - let mut discovery = DiscoveryTask::start(self, node_id)?; + let mut discovery = DiscoveryTask::start(self.clone(), node_id)?; discovery.first_arrived().await?; let addr = self.msock.get_mapping_addr(&node_id).ok_or_else(|| { anyhow!("Failed to retrieve the mapped address from the magic socket. Unable to dial node {node_id:?}") From b5d5d55d0ca94eeadfaf224566f9a4f9636f2b3d Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 13:15:06 +0100 Subject: [PATCH 15/66] chore: clippy --- iroh-net/src/discovery.rs | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index a780c1977a2..b6991eb06a0 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -34,11 +34,11 @@ pub trait Discovery: std::fmt::Debug + Send + Sync { /// /// Once the returned [`BoxStream`] is dropped, the service should stop any pending /// work. - fn resolve<'a>( - &'a self, + fn resolve( + &self, _endpoint: MagicEndpoint, _node_id: NodeId, - ) -> Option>> { + ) -> Option>> { None } } @@ -93,11 +93,11 @@ impl Discovery for CombinedDiscovery { } } - fn resolve<'a>( - &'a self, + fn resolve( + &self, endpoint: MagicEndpoint, node_id: NodeId, - ) -> Option>> { + ) -> Option>> { let streams = self .services .iter() @@ -188,10 +188,10 @@ impl DiscoveryTask { self.task.abort(); } - fn create_stream<'a>( - ep: &'a MagicEndpoint, + fn create_stream( + ep: &MagicEndpoint, node_id: NodeId, - ) -> Result>> { + ) -> Result>> { let discovery = ep .discovery() .ok_or_else(|| anyhow!("No discovery service configured"))?; @@ -324,11 +324,11 @@ mod tests { .insert(self.node_id, (info.clone(), now)); } - fn resolve<'a>( - &'a self, + fn resolve( + &self, endpoint: MagicEndpoint, node_id: NodeId, - ) -> Option>> { + ) -> Option>> { let addr_info = match self.resolve_wrong { false => self.shared.nodes.lock().get(&node_id).cloned(), true => { @@ -373,11 +373,11 @@ mod tests { impl Discovery for EmptyDiscovery { fn publish(&self, _info: &AddrInfo) {} - fn resolve<'a>( - &'a self, + fn resolve( + &self, _endpoint: MagicEndpoint, _node_id: NodeId, - ) -> Option>> { + ) -> Option>> { Some(stream::empty().boxed()) } } From b380b191831bf2412c4d1e793b801f2579da3765 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Fri, 8 Mar 2024 00:14:06 +0100 Subject: [PATCH 16/66] rename to ConcurrentDiscovery --- iroh-net/src/discovery.rs | 28 +++++++++++++--------------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index b6991eb06a0..3a5b87152c0 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -2,7 +2,7 @@ use std::time::Duration; -use anyhow::{anyhow, bail, Result}; +use anyhow::{anyhow, ensure, Result}; use futures::{stream::BoxStream, StreamExt}; use iroh_base::node_addr::NodeAddr; use tokio::{sync::oneshot, task::JoinHandle}; @@ -59,13 +59,15 @@ pub struct DiscoveryItem { } /// A discovery service that combines multiple discovery sources. +/// +/// The discovery services will resolve concurrently. #[derive(Debug, Default)] -pub struct CombinedDiscovery { +pub struct ConcurrentDiscovery { services: Vec>, } -impl CombinedDiscovery { - /// Create a new [`CombinedDiscovery`]. +impl ConcurrentDiscovery { + /// Create a new [`ConcurrentDiscovery`]. pub fn new() -> Self { Self::default() } @@ -76,7 +78,7 @@ impl CombinedDiscovery { } } -impl From for CombinedDiscovery +impl From for ConcurrentDiscovery where T: IntoIterator>, { @@ -86,7 +88,7 @@ where } } -impl Discovery for CombinedDiscovery { +impl Discovery for ConcurrentDiscovery { fn publish(&self, info: &AddrInfo) { for service in &self.services { service.publish(info); @@ -120,9 +122,7 @@ pub(super) struct DiscoveryTask { impl DiscoveryTask { /// Start a discovery task. pub fn start(ep: MagicEndpoint, node_id: NodeId) -> Result { - if ep.discovery().is_none() { - bail!("No discovery services configured"); - } + ensure!(ep.discovery().is_some(), "No discovery services configured"); let (on_first_tx, on_first_rx) = oneshot::channel(); let me = ep.node_id(); let task = tokio::task::spawn( @@ -150,9 +150,7 @@ impl DiscoveryTask { if !Self::needs_discovery(ep, node_id) { return Ok(None); } - if ep.discovery().is_none() { - bail!("No discovery services configured"); - } + ensure!(ep.discovery().is_some(), "No discovery services configured"); let (on_first_tx, on_first_rx) = oneshot::channel(); let ep = ep.clone(); let me = ep.node_id(); @@ -420,7 +418,7 @@ mod tests { let secret = SecretKey::generate(); let disco1 = EmptyDiscovery; let disco2 = disco_shared.create_discovery(secret.public()); - let mut disco = CombinedDiscovery::new(); + let mut disco = ConcurrentDiscovery::new(); disco.add(disco1); disco.add(disco2); new_endpoint(secret, disco).await @@ -449,7 +447,7 @@ mod tests { let disco1 = EmptyDiscovery; let disco2 = disco_shared.create_lying_discovery(secret.public()); let disco3 = disco_shared.create_discovery(secret.public()); - let mut disco = CombinedDiscovery::new(); + let mut disco = ConcurrentDiscovery::new(); disco.add(disco1); disco.add(disco2); disco.add(disco3); @@ -475,7 +473,7 @@ mod tests { let ep2 = { let secret = SecretKey::generate(); let disco1 = disco_shared.create_lying_discovery(secret.public()); - let mut disco = CombinedDiscovery::new(); + let mut disco = ConcurrentDiscovery::new(); disco.add(disco1); new_endpoint(secret, disco).await }; From 938342796dbe48db0eb186daf92369addb5ae909 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Fri, 8 Mar 2024 00:38:58 +0100 Subject: [PATCH 17/66] chore: doc link --- iroh-net/src/magic_endpoint.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index ba4d18d3315..01bedc5946b 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -130,7 +130,7 @@ impl MagicEndpointBuilder { /// Optionally set a discovery mechanism for this endpoint. /// /// If you want to combine multiple discovery services, you can pass a - /// [`crate::discovery::CombinedDiscovery`]. + /// [`crate::discovery::ConcurrentDiscovery`]. /// /// If no discovery service is set, connecting to a node without providing its /// direct addresses or Derp URLs will fail. From 646de44ea599bfd0a37017bf9b384f59476bfe74 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 10:49:49 +0100 Subject: [PATCH 18/66] feat: add iroh-dns crate --- Cargo.lock | 176 +++++++++++++++++ Cargo.toml | 1 + iroh-base/src/node_addr.rs | 6 + iroh-dns/Cargo.toml | 36 ++++ iroh-dns/examples/publish.rs | 90 +++++++++ iroh-dns/examples/resolve.rs | 59 ++++++ iroh-dns/examples/sig0.rs | 201 +++++++++++++++++++ iroh-dns/src/client.rs | 40 ++++ iroh-dns/src/discovery.rs | 55 ++++++ iroh-dns/src/lib.rs | 4 + iroh-dns/src/packet.rs | 367 +++++++++++++++++++++++++++++++++++ iroh-dns/src/publish.rs | 75 +++++++ iroh-dns/src/resolve.rs | 112 +++++++++++ 13 files changed, 1222 insertions(+) create mode 100644 iroh-dns/Cargo.toml create mode 100644 iroh-dns/examples/publish.rs create mode 100644 iroh-dns/examples/resolve.rs create mode 100644 iroh-dns/examples/sig0.rs create mode 100644 iroh-dns/src/client.rs create mode 100644 iroh-dns/src/discovery.rs create mode 100644 iroh-dns/src/lib.rs create mode 100644 iroh-dns/src/packet.rs create mode 100644 iroh-dns/src/publish.rs create mode 100644 iroh-dns/src/resolve.rs diff --git a/Cargo.lock b/Cargo.lock index 8895b449583..ab51e02b62d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1487,6 +1487,21 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" +[[package]] +name = "foreign-types" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6f339eb8adc052cd2ca78910fda869aefa38d22d5cb648e6485e4d3fc06f3b1" +dependencies = [ + "foreign-types-shared", +] + +[[package]] +name = "foreign-types-shared" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" + [[package]] name = "form_urlencoded" version = "1.2.1" @@ -1781,21 +1796,29 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "091a6fbccf4860009355e3efc52ff4acf37a63489aad7435372d44ceeb6fbbcf" dependencies = [ "async-trait", + "bytes", "cfg-if", "data-encoding", "enum-as-inner", "futures-channel", "futures-io", "futures-util", + "h2", + "http 0.2.11", "idna 0.4.0", "ipnet", "once_cell", "rand", + "ring 0.16.20", + "rustls", + "rustls-pemfile", "thiserror", "tinyvec", "tokio", + "tokio-rustls", "tracing", "url", + "webpki-roots", ] [[package]] @@ -1813,10 +1836,13 @@ dependencies = [ "parking_lot", "rand", "resolv-conf", + "rustls", "smallvec", "thiserror", "tokio", + "tokio-rustls", "tracing", + "webpki-roots", ] [[package]] @@ -2032,6 +2058,19 @@ dependencies = [ "tokio-io-timeout", ] +[[package]] +name = "hyper-tls" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6183ddfa99b85da61a140bea0efc93fdf56ceaa041b37d553518030827f9905" +dependencies = [ + "bytes", + "hyper 0.14.28", + "native-tls", + "tokio", + "tokio-native-tls", +] + [[package]] name = "hyper-util" version = "0.1.3" @@ -2353,6 +2392,30 @@ dependencies = [ "tracing-subscriber", ] +[[package]] +name = "iroh-dns" +version = "0.12.0" +dependencies = [ + "anyhow", + "clap", + "derive_more", + "ed25519-dalek", + "futures", + "hex", + "hickory-proto", + "hickory-resolver", + "iroh-base", + "iroh-net", + "pkarr", + "reqwest", + "ring 0.16.20", + "rustls", + "tokio", + "tracing", + "tracing-subscriber", + "url", +] + [[package]] name = "iroh-gossip" version = "0.12.0" @@ -2791,6 +2854,24 @@ dependencies = [ "getrandom", ] +[[package]] +name = "native-tls" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07226173c32f2926027b63cce4bcd8076c3552846cbe7925f3aaffeac0a3b92e" +dependencies = [ + "lazy_static", + "libc", + "log", + "openssl", + "openssl-probe", + "openssl-sys", + "schannel", + "security-framework", + "security-framework-sys", + "tempfile", +] + [[package]] name = "netlink-packet-core" version = "0.7.0" @@ -3113,12 +3194,50 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c08d65885ee38876c4f86fa503fb49d7b507c2b62552df7c70b2fce627e06381" +[[package]] +name = "openssl" +version = "0.10.64" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95a0481286a310808298130d22dd1fef0fa571e05a8f44ec801801e84b216b1f" +dependencies = [ + "bitflags 2.4.2", + "cfg-if", + "foreign-types", + "libc", + "once_cell", + "openssl-macros", + "openssl-sys", +] + +[[package]] +name = "openssl-macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.49", +] + [[package]] name = "openssl-probe" version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" +[[package]] +name = "openssl-sys" +version = "0.9.101" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dda2b0f344e78efc2facf7d195d098df0dd72151b26ab98da807afc26c198dff" +dependencies = [ + "cc", + "libc", + "pkg-config", + "vcpkg", +] + [[package]] name = "option-ext" version = "0.2.0" @@ -3357,6 +3476,23 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pkarr" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bff6873180b2a43f312c5111da7a0ecaa519243aa8fe154bb61f8ebaafd4171" +dependencies = [ + "bytes", + "ed25519-dalek", + "rand", + "reqwest", + "self_cell", + "simple-dns", + "thiserror", + "url", + "z32", +] + [[package]] name = "pkcs1" version = "0.7.5" @@ -3378,6 +3514,12 @@ dependencies = [ "spki", ] +[[package]] +name = "pkg-config" +version = "0.3.30" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" + [[package]] name = "platforms" version = "3.3.0" @@ -4065,10 +4207,12 @@ dependencies = [ "http-body 0.4.6", "hyper 0.14.28", "hyper-rustls", + "hyper-tls", "ipnet", "js-sys", "log", "mime", + "native-tls", "once_cell", "percent-encoding", "pin-project-lite", @@ -4080,6 +4224,7 @@ dependencies = [ "sync_wrapper", "system-configuration", "tokio", + "tokio-native-tls", "tokio-rustls", "tower-service", "url", @@ -4609,6 +4754,15 @@ dependencies = [ "rand_core", ] +[[package]] +name = "simple-dns" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fadeff7784cac170aa987627327470c682eff6daa8ba9cc31d10313a5fb4b0ea" +dependencies = [ + "bitflags 2.4.2", +] + [[package]] name = "slab" version = "0.4.9" @@ -5108,6 +5262,16 @@ dependencies = [ "syn 2.0.52", ] +[[package]] +name = "tokio-native-tls" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbae76ab933c85776efabc971569dd6119c580d8f5d448769dec1764bf796ef2" +dependencies = [ + "native-tls", + "tokio", +] + [[package]] name = "tokio-rustls" version = "0.24.1" @@ -5513,6 +5677,12 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" +[[package]] +name = "vcpkg" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" + [[package]] name = "version_check" version = "0.9.4" @@ -6034,6 +6204,12 @@ dependencies = [ "time", ] +[[package]] +name = "z32" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5c2c4e1757d25c3bfa9e137b15e458bc028bf569c2c22e59ecf9f62545d45fd" + [[package]] name = "zerocopy" version = "0.7.32" diff --git a/Cargo.toml b/Cargo.toml index 751f092c909..d1cdd26c07b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -3,6 +3,7 @@ members = [ "iroh", "iroh-bytes", "iroh-base", + "iroh-dns", "iroh-gossip", "iroh-metrics", "iroh-net", diff --git a/iroh-base/src/node_addr.rs b/iroh-base/src/node_addr.rs index c696bd86c6d..7daf2ee2698 100644 --- a/iroh-base/src/node_addr.rs +++ b/iroh-base/src/node_addr.rs @@ -142,6 +142,12 @@ impl FromStr for DerpUrl { } } +impl From for Url { + fn from(value: DerpUrl) -> Self { + value.0 + } +} + /// Dereference to the wrapped [`Url`]. /// /// Note that [`DerefMut`] is not implemented on purpose, so this type has more flexibility diff --git a/iroh-dns/Cargo.toml b/iroh-dns/Cargo.toml new file mode 100644 index 00000000000..9acbe13a6d5 --- /dev/null +++ b/iroh-dns/Cargo.toml @@ -0,0 +1,36 @@ +[package] +name = "iroh-dns" +version = "0.12.0" +edition = "2021" +readme = "README.md" +description = "DNS resolver and pkarr publisher for iroh" +license = "MIT OR Apache-2.0" +authors = ["n0 team"] +repository = "https://github.com/n0-computer/iroh" +keywords = ["networking", "p2p", "holepunching", "ipfs"] + +# Sadly this also needs to be updated in .github/workflows/ci.yml +rust-version = "1.75" + +[dependencies] +anyhow = "1.0.80" +derive_more = { version = "1.0.0-beta.1", features = ["debug", "display"] } +ed25519-dalek = { version = "2.1.1", features = ["pkcs8"] } +futures = "0.3.30" +hex = "0.4.3" +hickory-proto = { version = "0.24.0", features = ["dnssec", "ring"] } +hickory-resolver = { version = "0.24.0", features = ["dns-over-https", "dns-over-tls", "tokio-rustls", "webpki-roots", "dns-over-rustls", "dns-over-https-rustls"] } +iroh-base = { version = "0.12.0", path = "../iroh-base", default_features = false, features = ["base32"] } +iroh-net = { version = "0.12.0", path = "../iroh-net", default_features = false } +pkarr = { version = "1.1.1", features = ["async", "relay"], default_features = false } +reqwest = { version = "0.11.24", features = ["rustls"] } +ring = "0.16" +rustls = "0.21" +tokio = { version = "1", features = ["rt", "sync"] } +tracing = "0.1" +url = { version = "2", features = ["serde"] } + +[dev-dependencies] +clap = { version = "4.5.1", features = ["derive"] } +tokio = { version = "1", features = ["full"] } +tracing-subscriber = "0.3" diff --git a/iroh-dns/examples/publish.rs b/iroh-dns/examples/publish.rs new file mode 100644 index 00000000000..01d6892f16b --- /dev/null +++ b/iroh-dns/examples/publish.rs @@ -0,0 +1,90 @@ +use std::str::FromStr; + +use anyhow::{bail, Result}; +use clap::{Parser, ValueEnum}; +use iroh_net::{key::SecretKey, AddrInfo, NodeId}; +use url::Url; + +use iroh_dns::{ + packet::IROH_NODE_TXT_LABEL, + publish::{Config, Publisher}, + resolve::{EXAMPLE_DOMAIN, IROH_TEST_DOMAIN}, +}; + +#[derive(ValueEnum, Clone, Debug, Default, Copy)] +pub enum Env { + /// Use the irohdns test server at testdns.iroh.link + #[default] + IrohTest, + /// Use a relay listening at localhost:8080 + LocalDev, +} + +/// Publish a record to an irohdns server. +/// +/// You have to set the IROH_SECRET environment variable to the node secret for which to publish. +#[derive(Parser, Debug)] +struct Cli { + /// Environment to publish to. + #[clap(value_enum, short, long, default_value_t = Env::IrohTest)] + env: Env, + /// Relay URL. If set, the --env option will be ignored. + #[clap(short, long, conflicts_with = "env")] + relay: Option, + /// Home Derp server to publish for this node + #[clap(short, long)] + derp_url: Url, + /// Create a new node secret if IROH_SECRET is unset. Only for development / debugging. + #[clap(short, long)] + create: bool, +} + +#[tokio::main] +async fn main() -> Result<()> { + tracing_subscriber::fmt::init(); + let args = Cli::parse(); + let secret_key = match std::env::var("IROH_SECRET") { + Ok(s) => SecretKey::from_str(&s)?, + Err(_) if args.create => { + let s = SecretKey::generate(); + println!("Generated a new node secret. To reuse, set"); + println!("IROH_SECRET={s}"); + s + } + Err(_) => { + bail!("Environtment variable IROH_SECRET is not set. To create a new secret, use the --create option.") + } + }; + let node_id = secret_key.public(); + println!("node: {node_id}"); + println!("derp: {}", args.derp_url); + let config = match (args.relay, args.env) { + (Some(pkarr_relay), _) => Config::new(secret_key, pkarr_relay), + (None, Env::IrohTest) => Config::with_iroh_test(secret_key), + (None, Env::LocalDev) => Config::localhost_dev(secret_key), + }; + let publisher = Publisher::new(config); + + let info = AddrInfo { + derp_url: Some(args.derp_url), + direct_addresses: Default::default(), + }; + // let an = NodeAnnounce::new(node_id, Some(args.home_derp), vec![]); + publisher.publish_addr_info(&info).await?; + println!("published signed record to {}!", publisher.pkarr_relay()); + match args.env { + Env::IrohTest => println!( + "TXT record resolvable at {}", + node_domain(node_id, IROH_TEST_DOMAIN) + ), + Env::LocalDev => println!( + "TXT record resolvable at {}", + node_domain(node_id, EXAMPLE_DOMAIN) + ), + } + Ok(()) +} + +fn node_domain(node_id: NodeId, origin: &str) -> String { + format!("{}.{}.{}", IROH_NODE_TXT_LABEL, node_id, origin) +} diff --git a/iroh-dns/examples/resolve.rs b/iroh-dns/examples/resolve.rs new file mode 100644 index 00000000000..91a7fdcd3e2 --- /dev/null +++ b/iroh-dns/examples/resolve.rs @@ -0,0 +1,59 @@ +use clap::Parser; +use clap::ValueEnum; +use iroh_dns::resolve::{Config, Resolver}; +use iroh_net::NodeId; + +#[derive(ValueEnum, Clone, Debug, Default)] +pub enum Env { + /// Use cloudflare and the irohdns test server at testdns.iroh.link + #[default] + IrohTest, + /// Use a localhost domain server listening on port 5353 + LocalDev, +} + +#[derive(Debug, Parser)] +struct Cli { + #[clap(value_enum, short, long, default_value_t = Env::IrohTest)] + env: Env, + #[clap(subcommand)] + command: Command, +} + +#[derive(Debug, Parser)] +enum Command { + /// Resolve node info by node id. + Node { node_id: NodeId }, + /// Resolve node info by domain. + Domain { domain: String }, +} + +#[tokio::main] +async fn main() -> anyhow::Result<()> { + let args = Cli::parse(); + let config = match args.env { + Env::IrohTest => Config::with_cloudflare_and_iroh_test(), + Env::LocalDev => Config::localhost_dev(), + }; + let resolver = Resolver::new(config)?; + match args.command { + Command::Node { node_id } => { + let addr = resolver.resolve_node_by_id(node_id).await?; + let derp_url = addr.derp_url.map(|u| u.to_string()).unwrap_or_default(); + println!("node_id: {node_id}"); + println!("derp_url: {derp_url}"); + } + Command::Domain { domain } => { + let addr = resolver.resolve_node_by_domain(&domain).await?; + let node_id = addr.node_id; + let derp_url = addr + .info + .derp_url + .map(|u| u.to_string()) + .unwrap_or_default(); + println!("node_id: {node_id}"); + println!("derp_url: {derp_url}"); + } + } + Ok(()) +} diff --git a/iroh-dns/examples/sig0.rs b/iroh-dns/examples/sig0.rs new file mode 100644 index 00000000000..1509727fb20 --- /dev/null +++ b/iroh-dns/examples/sig0.rs @@ -0,0 +1,201 @@ +fn main() { + unimplemented!() +} +// use anyhow::Result; +// use ed25519_dalek::SigningKey; +// use iroh_dns::{packet::NodeAnnounce}; +// use iroh_net::key::SecretKey; +// use url::Url; +// +// #[tokio::main] +// async fn main() -> Result<()> { +// tracing_subscriber::fmt::init(); +// let node_secret = SecretKey::generate(); +// let signing_key = SigningKey::from_bytes(&node_secret.to_bytes()); +// let node_id = node_secret.public(); +// +// println!("node_id {}", node_id); +// +// let home_derp: Url = "https://derp.example".parse()?; +// let msg = NodeAnnounce { +// node_id, +// home_derp: Some(home_derp.clone()), +// home_dns: Default::default(), +// }; +// +// // let name_server: SocketAddr = "127.0.0.1:5353".parse()?; +// // let res = publish_dns_sig0(name_server, msg, signing_key).await; +// let url: Url = "http://localhost:8080".parse()?; +// let res = sig0::publish_https(url, msg, signing_key).await; +// println!("res {res:?}"); +// res +// } +// mod sig0 { +// +// use std::{net::SocketAddr, str::FromStr, time::UNIX_EPOCH}; +// +// use anyhow::{anyhow, bail, Result}; +// use hickory_client::{ +// client::{Client, ClientHandle, SyncClient}, +// op::ResponseCode, +// proto::rr::dnssec::{Algorithm, KeyPair, SigSigner}, +// rr::{rdata::key::KEY, Name}, +// udp::UdpClientConnection, +// }; +// use hickory_proto::serialize::binary::{BinDecodable, BinEncodable}; +// use reqwest::header::CONTENT_TYPE; +// use ring::signature::Ed25519KeyPair; +// use tracing::debug; +// use url::Url; +// +// use iroh_dns::packet::NodeAnnounce; +// +// pub async fn publish_https( +// mut url: Url, +// announce: NodeAnnounce, +// signing_key: ed25519_dalek::SigningKey, +// ) -> Result<()> { +// let public_key = signing_key.verifying_key(); +// let node_zone = create_node_zone_name(public_key)?; +// let mut message = announce.into_hickory_update_message()?; +// println!("message {message:?}"); +// let signer = create_sig0_signer(signing_key, node_zone.clone())?; +// let ts = get_now_timestamp()?; +// // if signer.should_finalize_message(&message) { +// match message.finalize(&signer, ts) { +// Ok(_answer_verifier) => {} +// Err(e) => { +// debug!("could not sign message: {}", e); +// bail!(e) +// } +// } +// // } +// let body = message.to_bytes()?; +// +// url.set_path("/dns-query"); +// let client = reqwest::Client::new(); +// let res = client +// .post(url) +// .body(body) +// .header(CONTENT_TYPE, "application/dns-message") +// .send() +// .await?; +// let headers = res.headers().clone(); +// let body = res.bytes().await?; +// let parsed = hickory_proto::op::Message::from_bytes(&body)?; +// println!("headers {headers:#?}"); +// println!("body {parsed:#?}"); +// +// Ok(()) +// } +// +// fn get_now_timestamp() -> Result { +// let now = match std::time::SystemTime::now().duration_since(UNIX_EPOCH) { +// Ok(now) => now.as_secs(), +// Err(_) => bail!("Current time is before the Unix epoch."), +// }; +// +// // TODO: truncates u64 to u32, error on overflow? +// let now = now as u32; +// Ok(now) +// } +// +// pub fn create_node_zone_name(public_key: ed25519_dalek::VerifyingKey) -> Result { +// let node_id_str = iroh_base::base32::fmt(public_key.as_bytes()); +// let node_zone = Name::from_str(&format!("{}.", node_id_str))?; +// Ok(node_zone) +// } +// +// pub async fn publish_dns_sig0( +// nameserver: SocketAddr, +// announce: NodeAnnounce, +// signing_key: ed25519_dalek::SigningKey, +// ) -> Result<()> { +// let public_key = signing_key.verifying_key(); +// let record = announce.into_hickory_dns_record()?; +// let node_zone = create_node_zone_name(public_key)?; +// +// let signer = create_sig0_signer(signing_key, node_zone.clone())?; +// +// // TODO: HttpsClientConnection silently ignores the signer, there's a TODO in the +// // hickory-client code... +// // let conn = { +// // let client_config = insecure_client_config(); +// // let name_server: SocketAddr = "127.0.0.1:8443".parse()?; +// // let dns_name = "localhost".to_string(); +// // let conn: HttpsClientConnection> = +// // HttpsClientConnection::new(name_server, dns_name, client_config); +// // conn +// // }; +// +// let conn = UdpClientConnection::new(nameserver)?; +// let mut client = { +// let client = SyncClient::with_signer(conn, signer); +// let (client, bg) = client.new_future().await?; +// tokio::task::spawn(bg); +// client +// }; +// +// // Create the record. +// let result = client.create(record, node_zone).await?; +// match result.response_code() { +// ResponseCode::NoError => Ok(()), +// code @ _ => Err(anyhow!("request failed: {code}")), +// } +// } +// +// fn create_sig0_signer(signing_key: ed25519_dalek::SigningKey, zone: Name) -> Result { +// // Create the Hickory DNS SIG(0) signing facility. Generally the signer_name is the label +// // associated with KEY record in the server. +// let public_key = signing_key.verifying_key(); +// let key = Ed25519KeyPair::from_seed_and_public_key( +// &signing_key.to_bytes(), +// public_key.as_bytes(), +// )?; +// let key = KeyPair::from_ed25519(key); +// // Create the RData KEY associated with the key. This example uses defaults for all the +// // KeyTrust, KeyUsage, UpdateScope, Protocol. Many of these have been deprecated in current +// // DNS RFCs, but are still supported by many servers for auth. See auth docs of the remote +// // server for help in understanding it's requirements and support of these options. +// let sig0key = KEY::new( +// Default::default(), +// Default::default(), +// Default::default(), +// Default::default(), +// Algorithm::ED25519, +// key.to_public_bytes()?, +// ); +// Ok(SigSigner::sig0(sig0key, key, zone)) +// } +// +// // fn insecure_client_config() -> Arc { +// // let crypto = rustls::ClientConfig::builder() +// // .with_safe_defaults() +// // .with_custom_certificate_verifier(SkipServerVerification::new()) +// // .with_no_client_auth(); +// // Arc::new(crypto) +// // } +// // /// Dummy certificate verifier that treats any certificate as valid. +// // /// NOTE, such verification is vulnerable to MITM attacks, but convenient for testing. +// // struct SkipServerVerification; +// // +// // impl SkipServerVerification { +// // fn new() -> Arc { +// // Arc::new(Self) +// // } +// // } +// // +// // impl rustls::client::ServerCertVerifier for SkipServerVerification { +// // fn verify_server_cert( +// // &self, +// // _end_entity: &rustls::Certificate, +// // _intermediates: &[rustls::Certificate], +// // _server_name: &rustls::ServerName, +// // _scts: &mut dyn Iterator, +// // _ocsp_response: &[u8], +// // _now: std::time::SystemTime, +// // ) -> Result { +// // Ok(rustls::client::ServerCertVerified::assertion()) +// // } +// // } +// } diff --git a/iroh-dns/src/client.rs b/iroh-dns/src/client.rs new file mode 100644 index 00000000000..9857c3767d7 --- /dev/null +++ b/iroh-dns/src/client.rs @@ -0,0 +1,40 @@ +// // use iroh_net::NodeId +// use anyhow::Result; +// use tokio::task::JoinSet; +// use tokio::sync::mpsc; +// +// use crate::{packet::Announcement, NodeId}; +// +// // pub enum ResolveError { +// // NotFound, +// // F +// // } +// +// pub trait Resolver: Send { +// fn resolve(&self, node_id: NodeId) -> impl Stream> + Send + 'static; +// } +// +// pub trait Publisher: Send { +// fn publish(&self, announcement: Announcement) -> impl Future> + Send + 'static; +// } +// +// pub struct Client { +// tasks: JoinSet, +// // queries: +// } +// +// impl Client { +// fn resolve(&self, node_id: NodeId) -> impl Stream> { +// let (tx, rx) = mpsc::channel(1); +// +// } +// async fn resolve_one(&self, node_id: NodeId) -> Result { +// +// } +// fn add_resolver(&mut self, resolver: impl Resolver) { +// self.tasks.spawn(async move { +// +// +// }) +// } +// } diff --git a/iroh-dns/src/discovery.rs b/iroh-dns/src/discovery.rs new file mode 100644 index 00000000000..a99ab9acb1c --- /dev/null +++ b/iroh-dns/src/discovery.rs @@ -0,0 +1,55 @@ +use std::sync::Arc; + +use anyhow::Result; +use futures::future::{BoxFuture, FutureExt}; +use iroh_net::key::SecretKey; +use iroh_net::magicsock::Discovery; +use iroh_net::{AddrInfo, NodeId}; +use tracing::warn; + +use crate::publish::{self, Publisher}; +use crate::resolve::{self, Resolver}; + +#[derive(Debug)] +pub struct DnsDiscovery { + publisher: Option>, + resolver: Resolver, +} + +impl DnsDiscovery { + pub fn new(resolver: Resolver, publisher: Option>) -> Self { + Self { + resolver, + publisher, + } + } + pub fn with_iroh_test(secret_key: Option) -> Result { + let publisher = + secret_key.map(|k| Arc::new(Publisher::new(publish::Config::with_iroh_test(k)))); + let resolver = Resolver::new(resolve::Config::with_cloudflare_and_iroh_test())?; + Ok(Self::new(resolver, publisher)) + } + pub fn localhost_dev(secret_key: Option) -> Result { + let publisher = + secret_key.map(|k| Arc::new(Publisher::new(publish::Config::localhost_dev(k)))); + let resolver = Resolver::new(resolve::Config::localhost_dev())?; + Ok(Self::new(resolver, publisher)) + } +} + +impl Discovery for DnsDiscovery { + fn publish(&self, info: &AddrInfo) { + if let Some(publisher) = self.publisher.clone() { + let info = info.clone(); + tokio::task::spawn(async move { + if let Err(err) = publisher.publish_addr_info(&info).await { + warn!("failed to publish address update: {err:?}"); + } + }); + } + } + + fn resolve<'a>(&'a self, node_id: &'a NodeId) -> BoxFuture<'a, Result> { + self.resolver.resolve_node_by_id(*node_id).boxed() + } +} diff --git a/iroh-dns/src/lib.rs b/iroh-dns/src/lib.rs new file mode 100644 index 00000000000..20c55182476 --- /dev/null +++ b/iroh-dns/src/lib.rs @@ -0,0 +1,4 @@ +pub mod discovery; +pub mod packet; +pub mod publish; +pub mod resolve; diff --git a/iroh-dns/src/packet.rs b/iroh-dns/src/packet.rs new file mode 100644 index 00000000000..60d03edf653 --- /dev/null +++ b/iroh-dns/src/packet.rs @@ -0,0 +1,367 @@ +use std::{collections::HashMap, fmt::Display, str::FromStr}; + +// use hickory_proto::rr::Name; +use anyhow::{anyhow, bail, Result}; +use hickory_proto::error::ProtoError; +use iroh_net::{AddrInfo, NodeAddr, NodeId}; +use url::Url; + +pub const IROH_ROOT_ZONE: &'static str = "iroh"; +pub const IROH_NODE_TXT_LABEL: &'static str = "_iroh_node"; +pub const DEFAULT_TTL: u32 = 30; + +pub const ATTR_DERP: &'static str = "derp"; +pub const ATTR_NODE_ID: &'static str = "node"; +pub const ATTR_DNS: &'static str = "dns"; + +#[derive(derive_more::Debug, Clone, Eq, PartialEq)] +pub struct NodeAnnounce { + pub node_id: NodeId, + #[debug("{:?}", self.home_derp.as_ref().map(|s| s.to_string()))] + pub home_derp: Option, + pub home_dns: Vec, +} + +impl From for NodeAddr { + fn from(value: NodeAnnounce) -> Self { + NodeAddr { + node_id: value.node_id, + info: value.into(), + } + } +} + +impl From for AddrInfo { + fn from(value: NodeAnnounce) -> Self { + AddrInfo { + derp_url: value.home_derp.map(|u| u.into()), + direct_addresses: Default::default(), + } + } +} + +impl NodeAnnounce { + pub fn new(node_id: NodeId, derp: Option, dns: Vec) -> Self { + Self { + node_id, + home_derp: derp, + home_dns: dns, + } + } + + pub fn to_attr_string(&self) -> String { + let mut attrs = vec![]; + attrs.push(fmt_attr(ATTR_NODE_ID, &self.node_id)); + if let Some(derp) = &self.home_derp { + attrs.push(fmt_attr(ATTR_DERP, &derp)); + } + for dns in &self.home_dns { + attrs.push(fmt_attr(ATTR_DNS, &dns)); + } + attrs.join(" ") + } + + pub fn zone(&self, absolute: bool) -> String { + match absolute { + true => format!("{}.{}.", self.node_id, IROH_ROOT_ZONE), + false => format!("{}.{}", self.node_id, IROH_ROOT_ZONE), + } + } + + pub fn hickory_zone(&self, absolute: bool) -> Result { + hickory_proto::rr::Name::from_str(&self.zone(absolute)) + } + + pub fn into_hickory_answers_message(&self) -> Result { + use hickory_proto::op; + let record = self.into_hickory_dns_record()?; + let mut packet = op::Message::new(); + packet.answers_mut().push(record); + Ok(packet) + } + + pub fn into_hickory_update_message(&self) -> Result { + use hickory_proto::{op, rr}; + let record = self.into_hickory_dns_record()?; + let zone = rr::Name::from_str(&self.zone(true))?; + let message = op::update_message::create(record.into(), zone, false); + Ok(message) + } + + pub fn into_hickory_dns_record(&self) -> Result { + use hickory_proto::rr; + let origin = rr::Name::from_str(IROH_ROOT_ZONE)?; + self.into_hickory_dns_record_with_origin(&origin) + } + + pub fn into_hickory_dns_record_with_origin( + &self, + origin: &hickory_proto::rr::Name, + ) -> Result { + use hickory_proto::rr; + let zone = rr::Name::from_str(&self.node_id.to_string())?; + let zone = zone.append_domain(&origin)?; + let name = rr::Name::parse(IROH_NODE_TXT_LABEL, Some(&zone))?; + let txt_value = self.to_attr_string(); + let txt_data = rr::rdata::TXT::new(vec![txt_value]); + let rdata = rr::RData::TXT(txt_data); + let record = rr::Record::from_rdata(name, DEFAULT_TTL, rdata); + Ok(record) + } + + pub fn into_pkarr_dns_packet(&self) -> Result> { + use pkarr::dns::{self, rdata}; + let mut packet = dns::Packet::new_reply(0); + // let name = format!("{}.{}", IROH_NODE_TXT_NAME, self.zone()); + let name = IROH_NODE_TXT_LABEL; + let name = dns::Name::new(&name)?.into_owned(); + let txt_value = self.to_attr_string(); + let txt_data = rdata::TXT::new().with_string(&txt_value)?.into_owned(); + let rdata = rdata::RData::TXT(txt_data); + packet.answers.push(dns::ResourceRecord::new( + name, + dns::CLASS::IN, + DEFAULT_TTL, + rdata, + )); + Ok(packet) + } + + pub fn into_pkarr_signed_packet( + &self, + signing_key: &ed25519_dalek::SigningKey, + ) -> Result { + // TODO: PR to pkarr for impl From for pkarr::Keypair + let keypair = pkarr::Keypair::from_secret_key(&signing_key.to_bytes()); + let packet = self.into_pkarr_dns_packet()?; + let signed_packet = pkarr::SignedPacket::from_packet(&keypair, &packet)?; + Ok(signed_packet) + } + + pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { + use pkarr::dns::{self, rdata::RData}; + let pubkey = packet.public_key(); + let pubkey_z32 = pubkey.to_z32(); + let node_id = NodeId::from(*pubkey.verifying_key()); + let zone = dns::Name::new(&pubkey_z32)?; + let inner = packet.packet(); + let txt_record = inner + .answers + .iter() + .find_map(|rr| match &rr.rdata { + RData::TXT(txt) => match rr.name.without(&zone) { + Some(name) if &name.to_string() == IROH_NODE_TXT_LABEL => Some(txt), + Some(_) | None => None, + }, + _ => None, + }) + .ok_or_else(|| anyhow!("missing _iroh_node txt record"))?; + + let txt_record = txt_record.to_owned(); + let txt = String::try_from(txt_record)?; + let an = Self::parse_from_attributes(&txt)?; + if an.node_id != node_id { + bail!("node id mismatch between record name and TXT value"); + } + Ok(an) + } + + pub fn from_hickory_answers_message(message: &hickory_proto::op::Message) -> Result { + Self::from_hickory_records(message.answers()) + } + + pub fn from_hickory_lookup(lookup: &hickory_resolver::lookup::Lookup) -> Result { + Self::from_hickory_records(lookup.records()) + } + + pub fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { + use hickory_proto::rr; + let (node_id, txt) = records + .iter() + .find_map(|rr| match rr.data() { + Some(rr::RData::TXT(txt)) => { + if let Some(node_id) = is_hickory_node_info_name(rr.name()) { + Some((node_id, txt)) + } else { + None + } + } + _ => None, + }) + .ok_or_else(|| anyhow!("no TXT record with name _iroh_node.b32encodedpubkey found"))?; + let attr_str = txt.to_string(); + let an = Self::parse_from_attributes(&attr_str)?; + if an.node_id != node_id { + bail!("node id mismatch between record name and TXT value"); + } + Ok(an) + } + + pub fn parse_from_attributes(attrs: &str) -> Result { + let attrs = parse_attrs(attrs); + let Some(node) = attrs.get(ATTR_NODE_ID) else { + bail!("missing required node attr"); + }; + if node.len() != 1 { + bail!("more than one node attr is not allowed"); + } + let node_id = NodeId::from_str(&node[0])?; + let home_derp: Option = attrs + .get(ATTR_DERP) + .into_iter() + .flatten() + .find_map(|x| Url::parse(x).ok()); + let home_dns: Vec = attrs + .get(ATTR_DNS) + .into_iter() + .map(|x| x.into_iter()) + .flatten() + .map(|s| s.to_string()) + .collect(); + Ok(Self { + node_id, + home_derp, + home_dns, + }) + } +} + +fn is_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { + if name.num_labels() < 2 { + return None; + } + let mut labels = name.iter(); + let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; + if label != IROH_NODE_TXT_LABEL { + return None; + } + let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; + let node_id = NodeId::from_str(label).ok()?; + Some(node_id) +} + +fn parse_attrs<'a>(s: &'a str) -> HashMap<&'a str, Vec<&'a str>> { + let mut map: HashMap<&'a str, Vec<&'a str>> = HashMap::new(); + let parts = s.split(" "); + for part in parts { + if let Some((name, value)) = part.split_once("=") { + map.entry(name).or_default().push(value); + } + } + map +} + +fn fmt_attr(label: &str, value: impl Display) -> String { + format!("{label}={value}") +} + +// fn simple_dns_to_hickory( +// signed_packet: &pkarr::SignedPacket, +// ) -> anyhow::Result { +// let encoded = signed_packet.encoded_packet(); +// let parsed1 = pkarr::dns::Packet::parse(&encoded)?; +// println!("simple_dns {parsed1:#?}"); +// let parsed2 = hickory_proto::op::Message::from_bytes(&encoded)?; +// println!("hickory {parsed2:#?}"); +// Ok(parsed2) +// } + +#[cfg(test)] +mod tests { + // TODO: The tests are not comprehensive in any way, more like examples while getting things to + // work + + use std::str::FromStr; + + use hickory_proto::serialize::binary::{BinDecodable, BinEncodable}; + use url::Url; + + use super::*; + + #[test] + fn create_signed_packet() -> Result<()> { + let signing_key = iroh_net::key::SecretKey::generate(); + let node_id = signing_key.public(); + let home_derp: Url = "https://derp.example/".parse()?; + let an = NodeAnnounce { + node_id, + home_derp: Some(home_derp), + home_dns: vec![], + }; + let signing_key = ed25519_dalek::SigningKey::from_bytes(&signing_key.to_bytes()); + let sp = an.into_pkarr_signed_packet(&signing_key)?; + println!("sp {sp:#?}"); + println!("packet {:#?}", sp.packet()); + let an2 = NodeAnnounce::from_pkarr_signed_packet(&sp)?; + assert_eq!(an, an2); + let _p = an.into_hickory_answers_message()?; + Ok(()) + } + + #[test] + fn convert2() -> anyhow::Result<()> { + let key = iroh_net::key::SecretKey::generate(); + let node_id = key.public(); + let home_derp: Url = "https://derp.example".parse()?; + let a = NodeAnnounce { + node_id, + home_derp: Some(home_derp), + home_dns: Default::default(), + }; + let packet_simpdns = a.into_hickory_answers_message()?; + let packet_hickory = a.into_hickory_answers_message()?; + let buf_simpdns = packet_simpdns.to_bytes()?; + let buf_hickory = packet_hickory.to_bytes()?; + println!( + "simple_dns {} {}", + buf_simpdns.len(), + hex::encode(&buf_simpdns) + ); + println!( + "hickory {} {}", + buf_hickory.len(), + hex::encode(&buf_hickory) + ); + let _simpdns_from_hickory = pkarr::dns::Packet::parse(&buf_hickory)?; + let _hickory_form_simpdns = hickory_proto::op::Message::from_bytes(&buf_simpdns)?; + + Ok(()) + } + + #[test] + fn convert3() -> anyhow::Result<()> { + use hickory_proto as proto; + use pkarr::dns; + let ttl = 300; + let (packet1, bytes1) = { + use dns::rdata; + let mut packet = dns::Packet::new_reply(0); + let name = dns::Name::new("foo")?; + let rdata = rdata::RData::TXT(rdata::TXT::new().with_string("bar")?); + let record = dns::ResourceRecord::new(name, dns::CLASS::IN, ttl, rdata); + packet.answers.push(record); + let bytes = packet.build_bytes_vec()?; + (packet, bytes) + }; + let (packet2, bytes2) = { + use proto::rr; + use proto::serialize::binary::BinEncodable; + let mut packet = proto::op::Message::new(); + let name = rr::Name::from_str("foo")?; + let rdata = rr::RData::TXT(rr::rdata::TXT::new(vec!["bar".to_string()])); + let mut record = rr::Record::with(name, rr::RecordType::TXT, ttl); + record.set_data(Some(rdata)); + packet.answers_mut().push(record); + let bytes = packet.to_bytes()?; + (packet, bytes) + }; + println!("simple_dns deb {:#?}", packet1); + println!("hickory deb {:#?}", packet2); + println!("simple_dns len {}", bytes1.len()); + println!("hickory len {}", bytes2.len()); + println!("simple_dns hex {}", hex::encode(&bytes1)); + println!("hickory hex {}", hex::encode(&bytes2)); + + Ok(()) + } +} diff --git a/iroh-dns/src/publish.rs b/iroh-dns/src/publish.rs new file mode 100644 index 00000000000..a045f9ec997 --- /dev/null +++ b/iroh-dns/src/publish.rs @@ -0,0 +1,75 @@ +use anyhow::Result; +use ed25519_dalek::SigningKey; +use iroh_net::{key::SecretKey, AddrInfo, NodeId}; +use pkarr::PkarrClient; +use url::Url; + +use crate::packet::NodeAnnounce; + +pub const IROH_TEST_PKARR_RELAY: &'static str = "https://testdns.iroh.link/pkarr"; +pub const LOCALHOST_PKARR_RELAY: &'static str = "http://localhost:8080/pkarr"; + +/// Publisher config +pub struct Config { + pub secret_key: SecretKey, + pub pkarr_relay: Url, +} + +impl Config { + pub fn new(secret_key: SecretKey, pkarr_relay: Url) -> Self { + Self { + secret_key, + pkarr_relay, + } + } + + pub fn with_iroh_test(secret_key: SecretKey) -> Self { + let pkarr_relay: Url = IROH_TEST_PKARR_RELAY.parse().expect("url is valid"); + Self::new(secret_key, pkarr_relay) + } + + pub fn localhost_dev(secret_key: SecretKey) -> Self { + let pkarr_relay: Url = LOCALHOST_PKARR_RELAY.parse().expect("url is valid"); + Self::new(secret_key, pkarr_relay) + } +} + +/// Publish node announces to a pkarr relay. +#[derive(Debug)] +pub struct Publisher { + node_id: NodeId, + signing_key: SigningKey, + pkarr_relay: Url, + pkarr_client: PkarrClient, +} + +impl Publisher { + pub fn new(config: Config) -> Self { + let pkarr_client = PkarrClient::builder().build(); + let node_id = config.secret_key.public(); + let signing_key = ed25519_dalek::SigningKey::from_bytes(&config.secret_key.to_bytes()); + Self { + node_id, + signing_key, + pkarr_relay: config.pkarr_relay, + pkarr_client, + } + } + + pub async fn publish_addr_info(&self, info: &AddrInfo) -> Result<()> { + let an = NodeAnnounce::new( + self.node_id, + info.derp_url.as_ref().map(|u| u.clone().into()), + Default::default(), + ); + let signed_packet = an.into_pkarr_signed_packet(&self.signing_key)?; + self.pkarr_client + .relay_put(&self.pkarr_relay, &signed_packet) + .await?; + Ok(()) + } + + pub fn pkarr_relay(&self) -> &Url { + &self.pkarr_relay + } +} diff --git a/iroh-dns/src/resolve.rs b/iroh-dns/src/resolve.rs new file mode 100644 index 00000000000..72a96428e5b --- /dev/null +++ b/iroh-dns/src/resolve.rs @@ -0,0 +1,112 @@ +use std::{net::Ipv4Addr, str::FromStr}; + +use anyhow::Result; +use hickory_proto::error::ProtoError; +use hickory_resolver::{ + config::{NameServerConfigGroup, ResolverConfig, ResolverOpts}, + name_server::{GenericConnector, TokioRuntimeProvider}, + AsyncResolver, Name, +}; +use iroh_net::{AddrInfo, NodeAddr, NodeId}; + +use crate::packet::{NodeAnnounce, IROH_NODE_TXT_LABEL}; + +pub const IROH_TEST_DNS_IPV4: Ipv4Addr = Ipv4Addr::new(5, 75, 181, 3); +pub const IROH_TEST_DOMAIN: &'static str = "testdns.iroh.link."; +pub const EXAMPLE_DOMAIN: &'static str = "irohdns.example."; + +pub type HickoryResolver = AsyncResolver>; + +/// Resolver config +pub struct Config { + name_servers: NameServerConfigGroup, + default_node_origin: String, +} + +impl Config { + // TODO: Add with_system_and_iroh_test() + + pub fn with_cloudflare_and_iroh_test() -> Self { + let cloudflare_dns = NameServerConfigGroup::cloudflare(); + let cloudflare_https = NameServerConfigGroup::cloudflare_https(); + let iroh_test_https = NameServerConfigGroup::from_ips_https( + &[IROH_TEST_DNS_IPV4.into()], + 443, + IROH_TEST_DOMAIN.to_string(), + true, + ); + let iroh_test_dns = + NameServerConfigGroup::from_ips_clear(&[IROH_TEST_DNS_IPV4.into()], 53, false); + + let mut name_servers = NameServerConfigGroup::new(); + name_servers.merge(cloudflare_https); + name_servers.merge(cloudflare_dns); + name_servers.merge(iroh_test_https); + name_servers.merge(iroh_test_dns); + Self { + name_servers, + default_node_origin: IROH_TEST_DOMAIN.to_string(), + } + } + + pub fn localhost_dev() -> Self { + let name_servers = + NameServerConfigGroup::from_ips_clear(&[Ipv4Addr::LOCALHOST.into()], 5353, true); + Self { + name_servers, + default_node_origin: EXAMPLE_DOMAIN.to_string(), + } + } +} + +/// Resolve iroh nodes through DNS +#[derive(Debug, Clone)] +pub struct Resolver { + default_node_origin: Name, + dns_resolver: HickoryResolver, +} + +impl Resolver { + pub fn new(config: Config) -> Result { + let default_node_origin = Name::from_str(&config.default_node_origin)?; + // TODO: If we add our default node origin as search domain, we can resolve just node IDs! + // let domain = Some(config.default_node_origin); + let domain = None; + let resolv_conf = ResolverConfig::from_parts(domain, vec![], config.name_servers); + let dns_resolver = AsyncResolver::tokio(resolv_conf, ResolverOpts::default()); + Ok(Self { + dns_resolver, + default_node_origin, + }) + } + + pub fn resolver(&self) -> &HickoryResolver { + &self.dns_resolver + } + + pub async fn resolve_node_by_domain(&self, domain: &str) -> Result { + let name = Name::from_str(&domain)?; + self.resolve_node(name).await + } + + pub async fn resolve_node_by_id(&self, node_id: NodeId) -> Result { + let name = Name::parse(&node_id.to_string(), Some(&self.default_node_origin))?; + let addr = self.resolve_node(name).await?; + Ok(addr.info) + } + + async fn resolve_node(&self, name: Name) -> Result { + let name = with_iroh_node_txt_label(name)?; + let lookup = self.dns_resolver.txt_lookup(name).await?; + let an = NodeAnnounce::from_hickory_lookup(lookup.as_lookup())?; + Ok(an.into()) + } +} + +fn with_iroh_node_txt_label(name: Name) -> Result { + if name.iter().next() == Some(IROH_NODE_TXT_LABEL.as_bytes()) { + Ok(name) + } else { + Name::parse(IROH_NODE_TXT_LABEL, Some(&name)) + } +} From ae3e0e7e4490eff02479d75611fa5054cae3f543 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 10:50:25 +0100 Subject: [PATCH 19/66] feat: discover nodes over DNS by default --- Cargo.lock | 1 + iroh/Cargo.toml | 1 + iroh/src/commands/start.rs | 4 ++++ iroh/src/node.rs | 17 +++++++++++++++++ 4 files changed, 23 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index ab51e02b62d..74557587c23 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2263,6 +2263,7 @@ dependencies = [ "indicatif", "iroh-base", "iroh-bytes", + "iroh-dns", "iroh-gossip", "iroh-io", "iroh-metrics", diff --git a/iroh/Cargo.toml b/iroh/Cargo.toml index e7f92eddda2..b359351fb3a 100644 --- a/iroh/Cargo.toml +++ b/iroh/Cargo.toml @@ -29,6 +29,7 @@ hashlink = "0.8.4" hex = { version = "0.4.3" } iroh-bytes = { version = "0.12.0", path = "../iroh-bytes", features = ["downloader"] } iroh-base = { version = "0.12.0", path = "../iroh-base", features = ["key"] } +iroh-dns = { version = "0.12.0", path = "../iroh-dns" } iroh-io = { version = "0.4.0", features = ["stats"] } iroh-metrics = { version = "0.12.0", path = "../iroh-metrics", optional = true } iroh-net = { version = "0.12.0", path = "../iroh-net" } diff --git a/iroh/src/commands/start.rs b/iroh/src/commands/start.rs index 65c80976eb6..9c999b2aa29 100644 --- a/iroh/src/commands/start.rs +++ b/iroh/src/commands/start.rs @@ -14,6 +14,7 @@ use iroh::{ rpc_protocol::{ProviderRequest, ProviderResponse, ProviderService}, util::{fs::load_secret_key, path::IrohPaths}, }; +use iroh_dns::discovery::{DnsDiscovery}; use iroh_net::{ derp::{DerpMap, DerpMode}, key::SecretKey, @@ -219,12 +220,15 @@ pub(crate) async fn start_node( Some(derp_map) => DerpMode::Custom(derp_map), }; + let discovery = DnsDiscovery::with_iroh_test(Some(secret_key.clone()))?; + Node::builder(bao_store, doc_store) .derp_mode(derp_mode) .peers_data_path(peers_data_path) .local_pool(rt) .rpc_endpoint(rpc_endpoint) .secret_key(secret_key) + .node_discovery(Box::new(discovery)) .spawn() .await } diff --git a/iroh/src/node.rs b/iroh/src/node.rs index 4095dff0e03..723641ea778 100644 --- a/iroh/src/node.rs +++ b/iroh/src/node.rs @@ -37,6 +37,7 @@ use iroh_net::derp::DerpUrl; use iroh_net::magic_endpoint::get_alpn; use iroh_net::magicsock::LocalEndpointsStream; use iroh_net::util::AbortingJoinHandle; +use iroh_net::magicsock::Discovery; use iroh_net::{ derp::DerpMode, key::{PublicKey, SecretKey}, @@ -135,6 +136,8 @@ where docs: S, /// Path to store peer data. If `None`, peer data will not be persisted. peers_data_path: Option, + /// Discovery service to retreive node dialing info. + node_discovery: Option>, } const PROTOCOLS: [&[u8]; 3] = [&iroh_bytes::protocol::ALPN, GOSSIP_ALPN, SYNC_ALPN]; @@ -153,6 +156,7 @@ impl Builder { rt: None, docs, peers_data_path: None, + node_discovery: None, } } } @@ -180,6 +184,7 @@ where rt: self.rt, docs: self.docs, peers_data_path: self.peers_data_path, + node_discovery: self.node_discovery, } } @@ -243,6 +248,14 @@ where self } + /// Sets the node discovery service. + /// + /// If not set, nodes without direct or derp addresses will not be dialable. + pub fn node_discovery(mut self, discovery: Box) -> Self { + self.node_discovery = Some(discovery); + self + } + /// Spawns the [`Node`] in a tokio task. /// /// This will create the underlying network server and spawn a tokio task accepting @@ -274,6 +287,10 @@ where .transport_config(transport_config) .concurrent_connections(MAX_CONNECTIONS) .derp_mode(self.derp_mode); + let endpoint = match self.node_discovery { + Some(discovery) => endpoint.discovery(discovery), + None => endpoint + }; let endpoint = match self.peers_data_path { Some(path) => endpoint.peers_data_path(path), None => endpoint, From 89fffc93f0702ae124da2b11107cc6e8f1f878e0 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 10:58:20 +0100 Subject: [PATCH 20/66] fix: only republish if derp url changes --- Cargo.lock | 1 + iroh-dns/Cargo.toml | 1 + iroh-dns/src/publish.rs | 7 +++++++ 3 files changed, 9 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index 74557587c23..8cafc3bdd38 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2407,6 +2407,7 @@ dependencies = [ "hickory-resolver", "iroh-base", "iroh-net", + "parking_lot", "pkarr", "reqwest", "ring 0.16.20", diff --git a/iroh-dns/Cargo.toml b/iroh-dns/Cargo.toml index 9acbe13a6d5..f255a25daeb 100644 --- a/iroh-dns/Cargo.toml +++ b/iroh-dns/Cargo.toml @@ -22,6 +22,7 @@ hickory-proto = { version = "0.24.0", features = ["dnssec", "ring"] } hickory-resolver = { version = "0.24.0", features = ["dns-over-https", "dns-over-tls", "tokio-rustls", "webpki-roots", "dns-over-rustls", "dns-over-https-rustls"] } iroh-base = { version = "0.12.0", path = "../iroh-base", default_features = false, features = ["base32"] } iroh-net = { version = "0.12.0", path = "../iroh-net", default_features = false } +parking_lot = "0.12.1" pkarr = { version = "1.1.1", features = ["async", "relay"], default_features = false } reqwest = { version = "0.11.24", features = ["rustls"] } ring = "0.16" diff --git a/iroh-dns/src/publish.rs b/iroh-dns/src/publish.rs index a045f9ec997..d4ad7b345a6 100644 --- a/iroh-dns/src/publish.rs +++ b/iroh-dns/src/publish.rs @@ -1,6 +1,7 @@ use anyhow::Result; use ed25519_dalek::SigningKey; use iroh_net::{key::SecretKey, AddrInfo, NodeId}; +use parking_lot::RwLock; use pkarr::PkarrClient; use url::Url; @@ -41,6 +42,7 @@ pub struct Publisher { signing_key: SigningKey, pkarr_relay: Url, pkarr_client: PkarrClient, + last_announce: RwLock> } impl Publisher { @@ -53,6 +55,7 @@ impl Publisher { signing_key, pkarr_relay: config.pkarr_relay, pkarr_client, + last_announce: Default::default() } } @@ -62,6 +65,10 @@ impl Publisher { info.derp_url.as_ref().map(|u| u.clone().into()), Default::default(), ); + if self.last_announce.read().as_ref() == Some(&an) { + return Ok(()); + } + let _ = self.last_announce.write().insert(an.clone()); let signed_packet = an.into_pkarr_signed_packet(&self.signing_key)?; self.pkarr_client .relay_put(&self.pkarr_relay, &signed_packet) From 8f26558cbe121cb23aa28e304d1f899b434897fa Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 11:57:20 +0100 Subject: [PATCH 21/66] fix(iroh-dns): do not log secret key --- iroh-dns/src/discovery.rs | 2 +- iroh-dns/src/publish.rs | 6 +++++- iroh-dns/src/resolve.rs | 9 +++++++-- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/iroh-dns/src/discovery.rs b/iroh-dns/src/discovery.rs index a99ab9acb1c..117627bf89b 100644 --- a/iroh-dns/src/discovery.rs +++ b/iroh-dns/src/discovery.rs @@ -1,4 +1,4 @@ -use std::sync::Arc; +use std::{sync::Arc}; use anyhow::Result; use futures::future::{BoxFuture, FutureExt}; diff --git a/iroh-dns/src/publish.rs b/iroh-dns/src/publish.rs index d4ad7b345a6..8942a4235fd 100644 --- a/iroh-dns/src/publish.rs +++ b/iroh-dns/src/publish.rs @@ -36,12 +36,16 @@ impl Config { } /// Publish node announces to a pkarr relay. -#[derive(Debug)] +#[derive(derive_more::Debug)] pub struct Publisher { node_id: NodeId, + #[debug("SigningKey")] signing_key: SigningKey, + #[debug("{}", self.pkarr_relay)] pkarr_relay: Url, + #[debug("PkarrClient")] pkarr_client: PkarrClient, + #[debug(skip)] last_announce: RwLock> } diff --git a/iroh-dns/src/resolve.rs b/iroh-dns/src/resolve.rs index 72a96428e5b..973dadd805c 100644 --- a/iroh-dns/src/resolve.rs +++ b/iroh-dns/src/resolve.rs @@ -8,6 +8,7 @@ use hickory_resolver::{ AsyncResolver, Name, }; use iroh_net::{AddrInfo, NodeAddr, NodeId}; +use tracing::debug; use crate::packet::{NodeAnnounce, IROH_NODE_TXT_LABEL}; @@ -60,9 +61,10 @@ impl Config { } /// Resolve iroh nodes through DNS -#[derive(Debug, Clone)] +#[derive(derive_more::Debug, Clone)] pub struct Resolver { default_node_origin: Name, + #[debug("HickoryResolver")] dns_resolver: HickoryResolver, } @@ -90,8 +92,11 @@ impl Resolver { } pub async fn resolve_node_by_id(&self, node_id: NodeId) -> Result { + debug!(?node_id, "resolve node by id"); let name = Name::parse(&node_id.to_string(), Some(&self.default_node_origin))?; - let addr = self.resolve_node(name).await?; + let addr = self.resolve_node(name).await; + debug!(?node_id, ?addr, "resolved"); + let addr = addr?; Ok(addr.info) } From 58ad6f53801ee73dc28ca63e062073a9777733da Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 11:59:56 +0100 Subject: [PATCH 22/66] feat: allow to connect via node id only --- Cargo.lock | 4 ++-- iroh/src/commands/blob.rs | 5 ----- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8cafc3bdd38..5b994016cfe 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1804,7 +1804,7 @@ dependencies = [ "futures-io", "futures-util", "h2", - "http 0.2.11", + "http 0.2.12", "idna 0.4.0", "ipnet", "once_cell", @@ -3219,7 +3219,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] diff --git a/iroh/src/commands/blob.rs b/iroh/src/commands/blob.rs index 8a4a34326aa..806ffe49dd7 100644 --- a/iroh/src/commands/blob.rs +++ b/iroh/src/commands/blob.rs @@ -199,11 +199,6 @@ impl BlobCommands { return Err(anyhow::anyhow!("The input arguments refer to a collection of blobs and output is set to STDOUT. Only single blobs may be passed in this case.")); } - if node_addr.info.is_empty() { - return Err(anyhow::anyhow!( - "no Derp url provided and no direct addresses provided" - )); - } let tag = match tag { Some(tag) => SetTagOption::Named(Tag::from(tag)), None => SetTagOption::Auto, From 04611f2a1beca937bccf10922e141825d6e41587 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 12:27:24 +0100 Subject: [PATCH 23/66] fix: feature flags --- Cargo.lock | 567 +++++++++++++++++--------------------------- iroh-dns/Cargo.toml | 2 +- 2 files changed, 213 insertions(+), 356 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5b994016cfe..2040d77a28e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -30,9 +30,9 @@ dependencies = [ [[package]] name = "ahash" -version = "0.8.11" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +checksum = "42cd52102d3df161c77a887b608d7a4897d7cc112886a9537b738a887a03aaff" dependencies = [ "cfg-if", "once_cell", @@ -84,9 +84,9 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstream" -version = "0.6.13" +version = "0.6.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d96bd03f33fe50a863e394ee9718a706f988b9079b20c3784fb726e7678b62fb" +checksum = "96b09b5178381e0874812a9b157f7fe84982617e48f71f4e3235482775e5b540" dependencies = [ "anstyle", "anstyle-parse", @@ -206,7 +206,7 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -217,7 +217,7 @@ checksum = "c980ee35e870bd1a4d2c8294d4c04d0499e67bca1e4b5cefcc693c2fa00caea9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -226,7 +226,7 @@ version = "0.16.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdb8867f378f33f78a811a8eb9bf108ad99430d7aad43315dd9319c827ef6247" dependencies = [ - "http 0.2.12", + "http 0.2.11", "log", "url", "wildmatch", @@ -249,7 +249,7 @@ dependencies = [ "bitflags 1.3.2", "bytes", "futures-util", - "http 0.2.12", + "http 0.2.11", "http-body 0.4.6", "hyper 0.14.28", "itoa", @@ -275,7 +275,7 @@ dependencies = [ "async-trait", "bytes", "futures-util", - "http 0.2.12", + "http 0.2.11", "http-body 0.4.6", "mime", "rustversion", @@ -415,9 +415,9 @@ checksum = "78a6932c88f1d2c29533a3b8a5f5a2f84cc19c3339b431677c3160c5c2e6ca85" [[package]] name = "bumpalo" -version = "3.15.3" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ea184aa71bb362a1157c896979544cc23974e08fd265f29ea96b59f0b4a555b" +checksum = "c764d619ca78fccbf3069b37bd7af92577f044bb15236036662d79b6559f25b7" [[package]] name = "byteorder" @@ -473,9 +473,12 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.89" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0ba8f7aaa012f30d5b2861462f6708eccd49c3c39863fe083a308035f63d723" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +dependencies = [ + "libc", +] [[package]] name = "cfg-if" @@ -496,9 +499,9 @@ dependencies = [ [[package]] name = "chrono" -version = "0.4.35" +version = "0.4.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8eaf5903dcbc0a39312feb77df2ff4c76387d591b9fc7b04a238dcf8bb62639a" +checksum = "5bc015644b92d5890fab7489e49d21f879d5c990186827d42ec511919404f38b" dependencies = [ "android-tzdata", "iana-time-zone", @@ -506,7 +509,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-targets 0.52.4", + "windows-targets 0.52.0", ] [[package]] @@ -578,7 +581,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -827,9 +830,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.12" +version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab3db02a9c5b5121e1e42fbdb1aeb65f5e02624cc58c43f2884c6ccac0b82f95" +checksum = "176dc175b78f56c0f321911d9c8eb2b77a78a4860b9c19db83835fea1a46649b" dependencies = [ "crossbeam-utils", ] @@ -967,14 +970,14 @@ checksum = "f46882e17999c6cc590af592290432be3bce0428cb0d5f8b6715e4dc7b383eb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] name = "darling" -version = "0.20.8" +version = "0.20.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54e36fcd13ed84ffdfda6f5be89b31287cbb80c439841fe69e04841435464391" +checksum = "c376d08ea6aa96aafe61237c7200d1241cb177b7d3a542d791f2d118e9cbb955" dependencies = [ "darling_core", "darling_macro", @@ -982,27 +985,27 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.20.8" +version = "0.20.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c2cf1c23a687a1feeb728783b993c4e1ad83d99f351801977dd809b48d0a70f" +checksum = "33043dcd19068b8192064c704b3f83eb464f91f1ff527b44a4e2b08d9cdb8855" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim 0.10.0", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] name = "darling_macro" -version = "0.20.8" +version = "0.20.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a668eda54683121533a393014d8692171709ff57a7d61f187b6e782719f8933f" +checksum = "c5a91391accf613803c2a9bf9abccdbaa07c54b4244a5b64883f9c3c137c86be" dependencies = [ "darling_core", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -1094,7 +1097,7 @@ checksum = "5fe87ce4529967e0ba1dcf8450bab64d97dfd5010a6256187ffe2e43e6f0e049" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -1124,7 +1127,7 @@ checksum = "2bba3e9872d7c58ce7ef0fcf1844fcc3e23ef2a58377b50df35dd98e42a5726e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", "unicode-xid", ] @@ -1207,7 +1210,7 @@ checksum = "487585f4d0c6655fe74905e2504d8ad6908e4db67f744eb140876906c2f3175d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -1352,7 +1355,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -1365,7 +1368,7 @@ dependencies = [ "num-traits", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -1487,21 +1490,6 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" -[[package]] -name = "foreign-types" -version = "0.3.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6f339eb8adc052cd2ca78910fda869aefa38d22d5cb648e6485e4d3fc06f3b1" -dependencies = [ - "foreign-types-shared", -] - -[[package]] -name = "foreign-types-shared" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" - [[package]] name = "form_urlencoded" version = "1.2.1" @@ -1567,7 +1555,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -1584,9 +1572,9 @@ checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" [[package]] name = "futures-timer" -version = "3.0.3" +version = "3.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f288b0a4f20f9a56b5d1da57e2227c661b7b16168e2f72365f57b63326e29b24" +checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" [[package]] name = "futures-util" @@ -1715,8 +1703,8 @@ dependencies = [ "futures-core", "futures-sink", "futures-util", - "http 0.2.12", - "indexmap 2.2.5", + "http 0.2.11", + "indexmap 2.2.3", "slab", "tokio", "tokio-util", @@ -1725,9 +1713,9 @@ dependencies = [ [[package]] name = "half" -version = "2.4.0" +version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5eceaaeec696539ddaf7b333340f1af35a5aa87ae3e4f3ead0532f72affab2e" +checksum = "bc52e53916c08643f1b56ec082790d1e86a32e58dc5268f897f313fbae7b4872" dependencies = [ "cfg-if", "crunchy", @@ -1779,9 +1767,9 @@ checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" [[package]] name = "hermit-abi" -version = "0.3.9" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" +checksum = "bd5256b483761cd23699d0da46cc6fd2ee3be420bbe6d020ae4a091e70b7e9fd" [[package]] name = "hex" @@ -1804,7 +1792,7 @@ dependencies = [ "futures-io", "futures-util", "h2", - "http 0.2.12", + "http 0.2.11", "idna 0.4.0", "ipnet", "once_cell", @@ -1897,9 +1885,9 @@ checksum = "f558a64ac9af88b5ba400d99b579451af0d39c6d360980045b91aac966d705e2" [[package]] name = "http" -version = "0.2.12" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "601cbb57e577e2f5ef5be8e7b83f0f63994f25aa94d673e54a92d5c516d101f1" +checksum = "8947b1a6fad4393052c7ba1f4cd97bed3e953a95c79c92ad9b051a04611d9fbb" dependencies = [ "bytes", "fnv", @@ -1908,9 +1896,9 @@ dependencies = [ [[package]] name = "http" -version = "1.1.0" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21b9ddb458710bc376481b842f5da65cdf31522de232c1ca8146abce2a358258" +checksum = "b32afd38673a8016f7c9ae69e5af41a58f81b1d31689040f2f1959594ce194ea" dependencies = [ "bytes", "fnv", @@ -1924,7 +1912,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7ceab25649e9960c0311ea418d17bee82c0dcec1bd053b5f9a66e265a693bed2" dependencies = [ "bytes", - "http 0.2.12", + "http 0.2.11", "pin-project-lite", ] @@ -1935,7 +1923,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1cac85db508abc24a2e48553ba12a996e87244a0395ce011e62b37158745d643" dependencies = [ "bytes", - "http 1.1.0", + "http 1.0.0", ] [[package]] @@ -1946,7 +1934,7 @@ checksum = "41cb79eb393015dadd30fc252023adb0b2400a0caee0fa2a077e6e21a551e840" dependencies = [ "bytes", "futures-util", - "http 1.1.0", + "http 1.0.0", "http-body 1.0.0", "pin-project-lite", ] @@ -1999,7 +1987,7 @@ dependencies = [ "futures-core", "futures-util", "h2", - "http 0.2.12", + "http 0.2.11", "http-body 0.4.6", "httparse", "httpdate", @@ -2014,20 +2002,19 @@ dependencies = [ [[package]] name = "hyper" -version = "1.2.0" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "186548d73ac615b32a73aafe38fb4f56c0d340e110e5a200bcadbaf2e199263a" +checksum = "fb5aa53871fc917b1a9ed87b683a5d86db645e23acb32c2e0785a353e522fb75" dependencies = [ "bytes", "futures-channel", "futures-util", - "http 1.1.0", + "http 1.0.0", "http-body 1.0.0", "httparse", "httpdate", "itoa", "pin-project-lite", - "smallvec", "tokio", "want", ] @@ -2039,7 +2026,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", - "http 0.2.12", + "http 0.2.11", "hyper 0.14.28", "rustls", "tokio", @@ -2058,19 +2045,6 @@ dependencies = [ "tokio-io-timeout", ] -[[package]] -name = "hyper-tls" -version = "0.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6183ddfa99b85da61a140bea0efc93fdf56ceaa041b37d553518030827f9905" -dependencies = [ - "bytes", - "hyper 0.14.28", - "native-tls", - "tokio", - "tokio-native-tls", -] - [[package]] name = "hyper-util" version = "0.1.3" @@ -2079,9 +2053,9 @@ checksum = "ca38ef113da30126bbff9cd1705f9273e15d45498615d138b0c20279ac7a76aa" dependencies = [ "bytes", "futures-util", - "http 1.1.0", + "http 1.0.0", "http-body 1.0.0", - "hyper 1.2.0", + "hyper 1.1.0", "pin-project-lite", "socket2", "tokio", @@ -2145,7 +2119,7 @@ dependencies = [ "attohttpc", "bytes", "futures", - "http 0.2.12", + "http 0.2.11", "hyper 0.14.28", "log", "rand", @@ -2167,9 +2141,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.2.5" +version = "2.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b0b929d511467233429c45a44ac1dcaa21ba0f5ba11e4879e6ed28ddb4f9df4" +checksum = "233cf39063f058ea2caae4091bf4a3ef70a653afbc026f5c4a4135d114e3c177" dependencies = [ "equivalent", "hashbrown 0.14.3", @@ -2430,7 +2404,7 @@ dependencies = [ "ed25519-dalek", "futures", "genawaiter", - "indexmap 2.2.5", + "indexmap 2.2.3", "iroh-base", "iroh-blake3", "iroh-metrics", @@ -2470,7 +2444,7 @@ dependencies = [ "anyhow", "erased_set", "http-body-util", - "hyper 1.2.0", + "hyper 1.1.0", "hyper-util", "once_cell", "prometheus-client", @@ -2505,9 +2479,9 @@ dependencies = [ "hex", "hickory-resolver", "hostname", - "http 1.1.0", + "http 1.0.0", "http-body-util", - "hyper 1.2.0", + "hyper 1.1.0", "hyper-util", "igd", "iroh-base", @@ -2680,9 +2654,9 @@ checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c" [[package]] name = "js-sys" -version = "0.3.69" +version = "0.3.68" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" +checksum = "406cda4b368d531c842222cf9d2600a9a4acce8d29423695379c6868a143a9ee" dependencies = [ "wasm-bindgen", ] @@ -2743,15 +2717,15 @@ dependencies = [ [[package]] name = "log" -version = "0.4.21" +version = "0.4.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" [[package]] name = "lru" -version = "0.12.3" +version = "0.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3262e75e648fce39813cb56ac41f3c3e3f65217ebf3844d818d1f9398cfb0dc" +checksum = "db2c024b41519440580066ba82aab04092b333e09066a5eb86c7c4890df31f22" dependencies = [ "hashbrown 0.14.3", ] @@ -2827,9 +2801,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.11" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c" +checksum = "8f3d0b296e374a4e6f3c7b0a1f5a51d748a0d34c85e7dc48fc3fa9a87657fe09" dependencies = [ "libc", "wasi", @@ -2856,24 +2830,6 @@ dependencies = [ "getrandom", ] -[[package]] -name = "native-tls" -version = "0.2.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07226173c32f2926027b63cce4bcd8076c3552846cbe7925f3aaffeac0a3b92e" -dependencies = [ - "lazy_static", - "libc", - "log", - "openssl", - "openssl-probe", - "openssl-sys", - "schannel", - "security-framework", - "security-framework-sys", - "tempfile", -] - [[package]] name = "netlink-packet-core" version = "0.7.0" @@ -3142,7 +3098,7 @@ dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -3192,35 +3148,9 @@ checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" [[package]] name = "opaque-debug" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c08d65885ee38876c4f86fa503fb49d7b507c2b62552df7c70b2fce627e06381" - -[[package]] -name = "openssl" -version = "0.10.64" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95a0481286a310808298130d22dd1fef0fa571e05a8f44ec801801e84b216b1f" -dependencies = [ - "bitflags 2.4.2", - "cfg-if", - "foreign-types", - "libc", - "once_cell", - "openssl-macros", - "openssl-sys", -] - -[[package]] -name = "openssl-macros" -version = "0.1.1" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.52", -] +checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" [[package]] name = "openssl-probe" @@ -3228,18 +3158,6 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" -[[package]] -name = "openssl-sys" -version = "0.9.101" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dda2b0f344e78efc2facf7d195d098df0dd72151b26ab98da807afc26c198dff" -dependencies = [ - "cc", - "libc", - "pkg-config", - "vcpkg", -] - [[package]] name = "option-ext" version = "0.2.0" @@ -3278,7 +3196,7 @@ dependencies = [ "proc-macro2", "proc-macro2-diagnostics", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -3403,9 +3321,9 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "pest" -version = "2.7.8" +version = "2.7.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f8023d0fb78c8e03784ea1c7f3fa36e68a723138990b8d5a47d916b651e7a8" +checksum = "219c0dcc30b6a27553f9cc242972b67f75b60eb0db71f0b5462f38b058c41546" dependencies = [ "memchr", "thiserror", @@ -3414,9 +3332,9 @@ dependencies = [ [[package]] name = "pest_derive" -version = "2.7.8" +version = "2.7.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0d24f72393fd16ab6ac5738bc33cdb6a9aa73f8b902e8fe29cf4e67d7dd1026" +checksum = "22e1288dbd7786462961e69bfd4df7848c1e37e8b74303dbdab82c3a9cdd2809" dependencies = [ "pest", "pest_generator", @@ -3424,22 +3342,22 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.7.8" +version = "2.7.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdc17e2a6c7d0a492f0158d7a4bd66cc17280308bbaff78d5bef566dca35ab80" +checksum = "1381c29a877c6d34b8c176e734f35d7f7f5b3adaefe940cb4d1bb7af94678e2e" dependencies = [ "pest", "pest_meta", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] name = "pest_meta" -version = "2.7.8" +version = "2.7.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "934cd7631c050f4674352a6e835d5f6711ffbfb9345c2fc0107155ac495ae293" +checksum = "d0934d6907f148c22a3acbda520c7eed243ad7487a30f51f6ce52b58b7077a8a" dependencies = [ "once_cell", "pest", @@ -3448,22 +3366,22 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.5" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bf43b791c5b9e34c3d182969b4abb522f9343702850a2e57f460d00d09b4b3" +checksum = "0302c4a0442c456bd56f841aee5c3bfd17967563f6fadc9ceb9f9c23cf3807e0" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.5" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" +checksum = "266c042b60c9c76b8d53061e52b2e0d1116abc57cefc8c5cd671619a56ac3690" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -3480,9 +3398,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkarr" -version = "1.1.1" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bff6873180b2a43f312c5111da7a0ecaa519243aa8fe154bb61f8ebaafd4171" +checksum = "2d4025a211a70a716314d4ea6464aed150f696deb81651bebf62f874cee5aac7" dependencies = [ "bytes", "ed25519-dalek", @@ -3516,12 +3434,6 @@ dependencies = [ "spki", ] -[[package]] -name = "pkg-config" -version = "0.3.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" - [[package]] name = "platforms" version = "3.3.0" @@ -3558,39 +3470,39 @@ dependencies = [ [[package]] name = "pnet_base" -version = "0.34.0" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe4cf6fb3ab38b68d01ab2aea03ed3d1132b4868fa4e06285f29f16da01c5f4c" +checksum = "872e46346144ebf35219ccaa64b1dffacd9c6f188cd7d012bd6977a2a838f42e" dependencies = [ "no-std-net", ] [[package]] name = "pnet_macros" -version = "0.34.0" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "688b17499eee04a0408aca0aa5cba5fc86401d7216de8a63fdf7a4c227871804" +checksum = "2a780e80005c2e463ec25a6e9f928630049a10b43945fea83207207d4a7606f4" dependencies = [ "proc-macro2", "quote", "regex", - "syn 2.0.52", + "syn 1.0.109", ] [[package]] name = "pnet_macros_support" -version = "0.34.0" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eea925b72f4bd37f8eab0f221bbe4c78b63498350c983ffa9dd4bcde7e030f56" +checksum = "e6d932134f32efd7834eb8b16d42418dac87086347d1bc7d142370ef078582bc" dependencies = [ "pnet_base", ] [[package]] name = "pnet_packet" -version = "0.34.0" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9a005825396b7fe7a38a8e288dbc342d5034dac80c15212436424fef8ea90ba" +checksum = "8bde678bbd85cb1c2d99dc9fc596e57f03aa725f84f3168b0eaf33eeccb41706" dependencies = [ "glob", "pnet_base", @@ -3782,9 +3694,9 @@ checksum = "af066a9c399a26e020ada66a034357a868728e72cd426f3adcd35f80d88d88c8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", "version_check", - "yansi 1.0.0", + "yansi 1.0.0-rc.1", ] [[package]] @@ -3807,7 +3719,7 @@ checksum = "440f724eba9f6996b75d63681b0a92b06947f1457076d503a4d2e2c8f56442b8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -3850,7 +3762,7 @@ dependencies = [ "itertools 0.11.0", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -4041,9 +3953,9 @@ dependencies = [ [[package]] name = "rayon" -version = "1.9.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4963ed1bc86e4f3ee217022bd855b297cef07fb9eac5dfa1f788b220b49b3bd" +checksum = "fa7237101a77a10773db45d62004a272517633fbcc3df19d96455ede1122e051" dependencies = [ "either", "rayon-core", @@ -4129,18 +4041,18 @@ checksum = "5fddb4f8d99b0a2ebafc65a87a69a7b9875e4b1ae1f00db265d300ef7f28bccc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] name = "reflink-copy" -version = "0.1.15" +version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52b1349400e2ffd64a9fb5ed9008e33c0b8ef86bd5bae8f73080839c7082f1d5" +checksum = "767be24c0da52e7448d495b8d162506a9aa125426651d547d545d6c2b4b65b62" dependencies = [ "cfg-if", "rustix", - "windows 0.54.0", + "windows 0.52.0", ] [[package]] @@ -4151,7 +4063,7 @@ checksum = "b62dbe01f0b06f9d8dc7d49e05a0785f153b00b2c227856282f671e0318c9b15" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.6", + "regex-automata 0.4.5", "regex-syntax 0.8.2", ] @@ -4166,9 +4078,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.6" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea" +checksum = "5bb987efffd3c6d0d8f5f89510bb458559eab11e4f869acb20bf845e016259cd" dependencies = [ "aho-corasick", "memchr", @@ -4205,16 +4117,14 @@ dependencies = [ "futures-core", "futures-util", "h2", - "http 0.2.12", + "http 0.2.11", "http-body 0.4.6", "hyper 0.14.28", "hyper-rustls", - "hyper-tls", "ipnet", "js-sys", "log", "mime", - "native-tls", "once_cell", "percent-encoding", "pin-project-lite", @@ -4226,7 +4136,6 @@ dependencies = [ "sync_wrapper", "system-configuration", "tokio", - "tokio-native-tls", "tokio-rustls", "tower-service", "url", @@ -4568,9 +4477,9 @@ dependencies = [ [[package]] name = "serde" -version = "1.0.197" +version = "1.0.196" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fb1c873e1b9b056a4dc4c0c198b24c3ffa059243875552b2bd0933b1aee4ce2" +checksum = "870026e60fa08c69f064aa766c10f10b1d62db9ccd4d0abb206472bee0ce3b32" dependencies = [ "serde_derive", ] @@ -4595,22 +4504,22 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.197" +version = "1.0.196" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eb0b34b42edc17f6b7cac84a52a1c5f0e1bb2227e997ca9011ea3dd34e8610b" +checksum = "33c85360c95e7d137454dc81d9a4ed2b8efd8fbe19cee57357b32b9771fccb67" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] name = "serde_json" -version = "1.0.114" +version = "1.0.113" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5f09b1bd632ef549eaa9f60a1f8de742bdbc698e6cee2095fc84dde5f549ae0" +checksum = "69801b70b1c3dac963ecb03a364ba0ceda9cf60c71cfe475e99864759c8b8a79" dependencies = [ - "indexmap 2.2.5", + "indexmap 2.2.3", "itoa", "ryu", "serde", @@ -4656,7 +4565,7 @@ dependencies = [ "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.2.5", + "indexmap 2.2.3", "serde", "serde_derive", "serde_json", @@ -4673,7 +4582,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -4758,9 +4667,9 @@ dependencies = [ [[package]] name = "simple-dns" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fadeff7784cac170aa987627327470c682eff6daa8ba9cc31d10313a5fb4b0ea" +checksum = "01607fe2e61894468c6dc0b26103abb073fb08b79a3d9e4b6d76a1a341549958" dependencies = [ "bitflags 2.4.2", ] @@ -4785,12 +4694,12 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.6" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05ffd9c0a93b7543e062e759284fcf5f5e3b098501104bfbdde4d404db792871" +checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" dependencies = [ "libc", - "windows-sys 0.52.0", + "windows-sys 0.48.0", ] [[package]] @@ -4913,7 +4822,7 @@ dependencies = [ "proc-macro2", "quote", "struct_iterable_internal", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -4931,7 +4840,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta-derive", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -4942,7 +4851,7 @@ checksum = "a60bcaff7397072dca0017d1db428e30d5002e00b6847703e2e42005c95fbe00" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -4964,7 +4873,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -4997,9 +4906,9 @@ checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" [[package]] name = "surge-ping" -version = "0.8.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efbf95ce4c7c5b311d2ce3f088af2b93edef0f09727fa50fbe03c7a979afce77" +checksum = "af341b2be485d647b5dc4cfb2da99efac35b5c95748a08fb7233480fedc5ead3" dependencies = [ "hex", "parking_lot", @@ -5024,9 +4933,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.52" +version = "2.0.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b699d15b36d1f02c3e7c69f8ffef53de37aefae075d8488d4ba1a7788d574a07" +checksum = "915aea9e586f80826ee59f8453c1101f9d1c4b3964cd2460185ee8e299ada496" dependencies = [ "proc-macro2", "quote", @@ -5099,9 +5008,9 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.10.1" +version = "3.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" +checksum = "a365e8cd18e44762ef95d87f284f4b5cd04107fec2ff3052bd6a3e6069669e67" dependencies = [ "cfg-if", "fastrand", @@ -5118,7 +5027,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -5152,14 +5061,14 @@ checksum = "a953cb265bef375dae3de6663da4d3804eee9682ea80d8e2542529b73c531c81" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] name = "thread_local" -version = "1.1.8" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b9ef9bad013ada3808854ceac7b46812a6465ba368859a37e2100283d2d719c" +checksum = "3fdd6f064ccff2d6567adcb3873ca630700f00b5ad3f060c25b5dcfd9a4ce152" dependencies = [ "cfg-if", "once_cell", @@ -5261,17 +5170,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", -] - -[[package]] -name = "tokio-native-tls" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbae76ab933c85776efabc971569dd6119c580d8f5d448769dec1764bf796ef2" -dependencies = [ - "native-tls", - "tokio", + "syn 2.0.49", ] [[package]] @@ -5391,7 +5290,7 @@ version = "0.19.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" dependencies = [ - "indexmap 2.2.5", + "indexmap 2.2.3", "toml_datetime", "winnow 0.5.40", ] @@ -5402,7 +5301,7 @@ version = "0.21.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8534fd7f78b5405e860340ad6575217ce99f38d4d5c8f2442cb5ecb50090e1" dependencies = [ - "indexmap 2.2.5", + "indexmap 2.2.3", "toml_datetime", "winnow 0.5.40", ] @@ -5413,11 +5312,11 @@ version = "0.22.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2c1b5fd4128cc8d3e0cb74d4ed9a9cc7c7284becd4df68f5f940e1ad123606f6" dependencies = [ - "indexmap 2.2.5", + "indexmap 2.2.3", "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.5", + "winnow 0.6.1", ] [[package]] @@ -5432,7 +5331,7 @@ dependencies = [ "base64 0.21.7", "bytes", "h2", - "http 0.2.12", + "http 0.2.11", "http-body 0.4.6", "hyper 0.14.28", "hyper-timeout", @@ -5499,7 +5398,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -5608,9 +5507,9 @@ checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" [[package]] name = "unicode-normalization" -version = "0.1.23" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" +checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" dependencies = [ "tinyvec", ] @@ -5679,12 +5578,6 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" -[[package]] -name = "vcpkg" -version = "0.2.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" - [[package]] name = "version_check" version = "0.9.4" @@ -5702,9 +5595,9 @@ dependencies = [ [[package]] name = "walkdir" -version = "2.5.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +checksum = "d71d857dc86794ca4c280d616f7da00d2dbfd8cd788846559a6813e6aa4b54ee" dependencies = [ "same-file", "winapi-util", @@ -5725,17 +5618,11 @@ version = "0.11.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" -[[package]] -name = "wasite" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" - [[package]] name = "wasm-bindgen" -version = "0.2.92" +version = "0.2.91" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" +checksum = "c1e124130aee3fb58c5bdd6b639a0509486b0338acaaae0c84a5124b0f588b7f" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -5743,24 +5630,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.92" +version = "0.2.91" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" +checksum = "c9e7e1900c352b609c8488ad12639a311045f40a35491fb69ba8c12f758af70b" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.42" +version = "0.4.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76bc14366121efc8dbb487ab05bcc9d346b3b5ec0eaa76e46594cabbe51762c0" +checksum = "877b9c3f61ceea0e56331985743b13f3d25c406a7098d45180fb5f09bc19ed97" dependencies = [ "cfg-if", "js-sys", @@ -5770,9 +5657,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.92" +version = "0.2.91" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" +checksum = "b30af9e2d358182b5c7449424f017eba305ed32a7010509ede96cdc4696c46ed" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -5780,22 +5667,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.92" +version = "0.2.91" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" +checksum = "642f325be6301eb8107a83d12a8ac6c1e1c54345a7ef1a9261962dfefda09e66" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.92" +version = "0.2.91" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" +checksum = "4f186bd2dcf04330886ce82d6f33dd75a7bfcf69ecf5763b89fcde53b6ac9838" [[package]] name = "watchable" @@ -5811,9 +5698,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.69" +version = "0.3.68" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" +checksum = "96565907687f7aceb35bc5fc03770a8a0471d82e479f25832f54a0e3f4b28446" dependencies = [ "js-sys", "wasm-bindgen", @@ -5827,12 +5714,11 @@ checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1" [[package]] name = "whoami" -version = "1.5.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fec781d48b41f8163426ed18e8fc2864c12937df9ce54c88ede7bd47270893e" +checksum = "22fc3756b8a9133049b26c7f61ab35416c130e8c09b660f5b3958b446f52cc50" dependencies = [ - "redox_syscall", - "wasite", + "wasm-bindgen", "web-sys", ] @@ -5907,17 +5793,7 @@ dependencies = [ "windows-core 0.52.0", "windows-implement", "windows-interface", - "windows-targets 0.52.4", -] - -[[package]] -name = "windows" -version = "0.54.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9252e5725dbed82865af151df558e754e4a3c2c30818359eb17465f1346a1b49" -dependencies = [ - "windows-core 0.54.0", - "windows-targets 0.52.4", + "windows-targets 0.52.0", ] [[package]] @@ -5935,17 +5811,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.4", -] - -[[package]] -name = "windows-core" -version = "0.54.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12661b9c89351d684a50a8a643ce5f608e20243b9fb84687800163429f161d65" -dependencies = [ - "windows-result", - "windows-targets 0.52.4", + "windows-targets 0.52.0", ] [[package]] @@ -5956,7 +5822,7 @@ checksum = "12168c33176773b86799be25e2a2ba07c7aab9968b37541f1094dbd7a60c8946" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] @@ -5967,16 +5833,7 @@ checksum = "9d8dc32e0095a7eeccebd0e3f09e9509365ecb3fc6ac4d6f5f14a3f6392942d1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", -] - -[[package]] -name = "windows-result" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd19df78e5168dfb0aedc343d1d1b8d422ab2db6756d2dc3fef75035402a3f64" -dependencies = [ - "windows-targets 0.52.4", + "syn 2.0.49", ] [[package]] @@ -5994,7 +5851,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.4", + "windows-targets 0.52.0", ] [[package]] @@ -6014,17 +5871,17 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dd37b7e5ab9018759f893a1952c9420d060016fc19a472b4bb20d1bdd694d1b" +checksum = "8a18201040b24831fbb9e4eb208f8892e1f50a37feb53cc7ff887feb8f50e7cd" dependencies = [ - "windows_aarch64_gnullvm 0.52.4", - "windows_aarch64_msvc 0.52.4", - "windows_i686_gnu 0.52.4", - "windows_i686_msvc 0.52.4", - "windows_x86_64_gnu 0.52.4", - "windows_x86_64_gnullvm 0.52.4", - "windows_x86_64_msvc 0.52.4", + "windows_aarch64_gnullvm 0.52.0", + "windows_aarch64_msvc 0.52.0", + "windows_i686_gnu 0.52.0", + "windows_i686_msvc 0.52.0", + "windows_x86_64_gnu 0.52.0", + "windows_x86_64_gnullvm 0.52.0", + "windows_x86_64_msvc 0.52.0", ] [[package]] @@ -6035,9 +5892,9 @@ checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcf46cf4c365c6f2d1cc93ce535f2c8b244591df96ceee75d8e83deb70a9cac9" +checksum = "cb7764e35d4db8a7921e09562a0304bf2f93e0a51bfccee0bd0bb0b666b015ea" [[package]] name = "windows_aarch64_msvc" @@ -6047,9 +5904,9 @@ checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_aarch64_msvc" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da9f259dd3bcf6990b55bffd094c4f7235817ba4ceebde8e6d11cd0c5633b675" +checksum = "bbaa0368d4f1d2aaefc55b6fcfee13f41544ddf36801e793edbbfd7d7df075ef" [[package]] name = "windows_i686_gnu" @@ -6059,9 +5916,9 @@ checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_gnu" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b474d8268f99e0995f25b9f095bc7434632601028cf86590aea5c8a5cb7801d3" +checksum = "a28637cb1fa3560a16915793afb20081aba2c92ee8af57b4d5f28e4b3e7df313" [[package]] name = "windows_i686_msvc" @@ -6071,9 +5928,9 @@ checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_i686_msvc" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1515e9a29e5bed743cb4415a9ecf5dfca648ce85ee42e15873c3cd8610ff8e02" +checksum = "ffe5e8e31046ce6230cc7215707b816e339ff4d4d67c65dffa206fd0f7aa7b9a" [[package]] name = "windows_x86_64_gnu" @@ -6083,9 +5940,9 @@ checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnu" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5eee091590e89cc02ad514ffe3ead9eb6b660aedca2183455434b93546371a03" +checksum = "3d6fa32db2bc4a2f5abeacf2b69f7992cd09dca97498da74a151a3132c26befd" [[package]] name = "windows_x86_64_gnullvm" @@ -6095,9 +5952,9 @@ checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_gnullvm" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77ca79f2451b49fa9e2af39f0747fe999fcda4f5e241b2898624dca97a1f2177" +checksum = "1a657e1e9d3f514745a572a6846d3c7aa7dbe1658c056ed9c3344c4109a6949e" [[package]] name = "windows_x86_64_msvc" @@ -6107,9 +5964,9 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "windows_x86_64_msvc" -version = "0.52.4" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32b752e52a2da0ddfbdbcc6fceadfeede4c939ed16d13e648833a61dfb611ed8" +checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" [[package]] name = "winnow" @@ -6122,9 +5979,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.5" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dffa400e67ed5a4dd237983829e66475f0a4a26938c4b04c21baede6262215b8" +checksum = "d90f4e0f530c4c69f62b80d839e9ef3855edc9cba471a160c4d692deed62b401" dependencies = [ "memchr", ] @@ -6141,9 +5998,9 @@ dependencies = [ [[package]] name = "wmi" -version = "0.13.3" +version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc2f0a4062ca522aad4705a2948fd4061b3857537990202a8ddd5af21607f79a" +checksum = "fff298e96fd8ef6bb55dcb2a7fd2f26969f962bf428ffa6b267457dd804d64d8" dependencies = [ "chrono", "futures", @@ -6193,9 +6050,9 @@ checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" [[package]] name = "yansi" -version = "1.0.0" +version = "1.0.0-rc.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c2861d76f58ec8fc95708b9b1e417f7b12fd72ad33c01fa6886707092dea0d3" +checksum = "1367295b8f788d371ce2dbc842c7b709c73ee1364d30351dd300ec2203b12377" [[package]] name = "yasna" @@ -6229,7 +6086,7 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.52", + "syn 2.0.49", ] [[package]] diff --git a/iroh-dns/Cargo.toml b/iroh-dns/Cargo.toml index f255a25daeb..f451eeff8bd 100644 --- a/iroh-dns/Cargo.toml +++ b/iroh-dns/Cargo.toml @@ -24,7 +24,7 @@ iroh-base = { version = "0.12.0", path = "../iroh-base", default_features = fals iroh-net = { version = "0.12.0", path = "../iroh-net", default_features = false } parking_lot = "0.12.1" pkarr = { version = "1.1.1", features = ["async", "relay"], default_features = false } -reqwest = { version = "0.11.24", features = ["rustls"] } +reqwest = { version = "0.11.24", default_features = false, features = ["rustls-tls"] } ring = "0.16" rustls = "0.21" tokio = { version = "1", features = ["rt", "sync"] } From 6ee0f97334ec7e5b74662d388107894fd8c76811 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 12:40:09 +0100 Subject: [PATCH 24/66] chore: fmt --- iroh-dns/src/discovery.rs | 2 +- iroh-dns/src/publish.rs | 4 ++-- iroh/src/commands/start.rs | 2 +- iroh/src/node.rs | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/iroh-dns/src/discovery.rs b/iroh-dns/src/discovery.rs index 117627bf89b..a99ab9acb1c 100644 --- a/iroh-dns/src/discovery.rs +++ b/iroh-dns/src/discovery.rs @@ -1,4 +1,4 @@ -use std::{sync::Arc}; +use std::sync::Arc; use anyhow::Result; use futures::future::{BoxFuture, FutureExt}; diff --git a/iroh-dns/src/publish.rs b/iroh-dns/src/publish.rs index 8942a4235fd..91aa71c7a08 100644 --- a/iroh-dns/src/publish.rs +++ b/iroh-dns/src/publish.rs @@ -46,7 +46,7 @@ pub struct Publisher { #[debug("PkarrClient")] pkarr_client: PkarrClient, #[debug(skip)] - last_announce: RwLock> + last_announce: RwLock>, } impl Publisher { @@ -59,7 +59,7 @@ impl Publisher { signing_key, pkarr_relay: config.pkarr_relay, pkarr_client, - last_announce: Default::default() + last_announce: Default::default(), } } diff --git a/iroh/src/commands/start.rs b/iroh/src/commands/start.rs index 9c999b2aa29..fd26a554698 100644 --- a/iroh/src/commands/start.rs +++ b/iroh/src/commands/start.rs @@ -14,7 +14,7 @@ use iroh::{ rpc_protocol::{ProviderRequest, ProviderResponse, ProviderService}, util::{fs::load_secret_key, path::IrohPaths}, }; -use iroh_dns::discovery::{DnsDiscovery}; +use iroh_dns::discovery::DnsDiscovery; use iroh_net::{ derp::{DerpMap, DerpMode}, key::SecretKey, diff --git a/iroh/src/node.rs b/iroh/src/node.rs index 723641ea778..63d550c81a0 100644 --- a/iroh/src/node.rs +++ b/iroh/src/node.rs @@ -35,9 +35,9 @@ use iroh_gossip::net::{Gossip, GOSSIP_ALPN}; use iroh_io::AsyncSliceReader; use iroh_net::derp::DerpUrl; use iroh_net::magic_endpoint::get_alpn; +use iroh_net::magicsock::Discovery; use iroh_net::magicsock::LocalEndpointsStream; use iroh_net::util::AbortingJoinHandle; -use iroh_net::magicsock::Discovery; use iroh_net::{ derp::DerpMode, key::{PublicKey, SecretKey}, @@ -289,7 +289,7 @@ where .derp_mode(self.derp_mode); let endpoint = match self.node_discovery { Some(discovery) => endpoint.discovery(discovery), - None => endpoint + None => endpoint, }; let endpoint = match self.peers_data_path { Some(path) => endpoint.peers_data_path(path), From aa4cd942b98f098e14d6767d13d3db4c9c1de0f4 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 12:41:12 +0100 Subject: [PATCH 25/66] chore: fmt and clippy --- iroh-dns/examples/publish.rs | 2 +- iroh-dns/src/packet.rs | 37 ++++++++++++++++-------------------- iroh-dns/src/publish.rs | 4 ++-- iroh-dns/src/resolve.rs | 6 +++--- 4 files changed, 22 insertions(+), 27 deletions(-) diff --git a/iroh-dns/examples/publish.rs b/iroh-dns/examples/publish.rs index 01d6892f16b..5602fb3e3f4 100644 --- a/iroh-dns/examples/publish.rs +++ b/iroh-dns/examples/publish.rs @@ -66,7 +66,7 @@ async fn main() -> Result<()> { let publisher = Publisher::new(config); let info = AddrInfo { - derp_url: Some(args.derp_url), + derp_url: Some(args.derp_url.into()), direct_addresses: Default::default(), }; // let an = NodeAnnounce::new(node_id, Some(args.home_derp), vec![]); diff --git a/iroh-dns/src/packet.rs b/iroh-dns/src/packet.rs index 60d03edf653..af95db509c7 100644 --- a/iroh-dns/src/packet.rs +++ b/iroh-dns/src/packet.rs @@ -6,13 +6,13 @@ use hickory_proto::error::ProtoError; use iroh_net::{AddrInfo, NodeAddr, NodeId}; use url::Url; -pub const IROH_ROOT_ZONE: &'static str = "iroh"; -pub const IROH_NODE_TXT_LABEL: &'static str = "_iroh_node"; +pub const IROH_ROOT_ZONE: &str = "iroh"; +pub const IROH_NODE_TXT_LABEL: &str = "_iroh_node"; pub const DEFAULT_TTL: u32 = 30; -pub const ATTR_DERP: &'static str = "derp"; -pub const ATTR_NODE_ID: &'static str = "node"; -pub const ATTR_DNS: &'static str = "dns"; +pub const ATTR_DERP: &str = "derp"; +pub const ATTR_NODE_ID: &str = "node"; +pub const ATTR_DNS: &str = "dns"; #[derive(derive_more::Debug, Clone, Eq, PartialEq)] pub struct NodeAnnounce { @@ -51,12 +51,12 @@ impl NodeAnnounce { pub fn to_attr_string(&self) -> String { let mut attrs = vec![]; - attrs.push(fmt_attr(ATTR_NODE_ID, &self.node_id)); + attrs.push(fmt_attr(ATTR_NODE_ID, self.node_id)); if let Some(derp) = &self.home_derp { - attrs.push(fmt_attr(ATTR_DERP, &derp)); + attrs.push(fmt_attr(ATTR_DERP, derp)); } for dns in &self.home_dns { - attrs.push(fmt_attr(ATTR_DNS, &dns)); + attrs.push(fmt_attr(ATTR_DNS, dns)); } attrs.join(" ") } @@ -100,7 +100,7 @@ impl NodeAnnounce { ) -> Result { use hickory_proto::rr; let zone = rr::Name::from_str(&self.node_id.to_string())?; - let zone = zone.append_domain(&origin)?; + let zone = zone.append_domain(origin)?; let name = rr::Name::parse(IROH_NODE_TXT_LABEL, Some(&zone))?; let txt_value = self.to_attr_string(); let txt_data = rr::rdata::TXT::new(vec![txt_value]); @@ -114,7 +114,7 @@ impl NodeAnnounce { let mut packet = dns::Packet::new_reply(0); // let name = format!("{}.{}", IROH_NODE_TXT_NAME, self.zone()); let name = IROH_NODE_TXT_LABEL; - let name = dns::Name::new(&name)?.into_owned(); + let name = dns::Name::new(name)?.into_owned(); let txt_value = self.to_attr_string(); let txt_data = rdata::TXT::new().with_string(&txt_value)?.into_owned(); let rdata = rdata::RData::TXT(txt_data); @@ -150,7 +150,7 @@ impl NodeAnnounce { .iter() .find_map(|rr| match &rr.rdata { RData::TXT(txt) => match rr.name.without(&zone) { - Some(name) if &name.to_string() == IROH_NODE_TXT_LABEL => Some(txt), + Some(name) if name.to_string() == IROH_NODE_TXT_LABEL => Some(txt), Some(_) | None => None, }, _ => None, @@ -180,11 +180,7 @@ impl NodeAnnounce { .iter() .find_map(|rr| match rr.data() { Some(rr::RData::TXT(txt)) => { - if let Some(node_id) = is_hickory_node_info_name(rr.name()) { - Some((node_id, txt)) - } else { - None - } + is_hickory_node_info_name(rr.name()).map(|node_id| (node_id, txt)) } _ => None, }) @@ -205,7 +201,7 @@ impl NodeAnnounce { if node.len() != 1 { bail!("more than one node attr is not allowed"); } - let node_id = NodeId::from_str(&node[0])?; + let node_id = NodeId::from_str(node[0])?; let home_derp: Option = attrs .get(ATTR_DERP) .into_iter() @@ -214,8 +210,7 @@ impl NodeAnnounce { let home_dns: Vec = attrs .get(ATTR_DNS) .into_iter() - .map(|x| x.into_iter()) - .flatten() + .flat_map(|x| x.iter()) .map(|s| s.to_string()) .collect(); Ok(Self { @@ -242,9 +237,9 @@ fn is_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { fn parse_attrs<'a>(s: &'a str) -> HashMap<&'a str, Vec<&'a str>> { let mut map: HashMap<&'a str, Vec<&'a str>> = HashMap::new(); - let parts = s.split(" "); + let parts = s.split(' '); for part in parts { - if let Some((name, value)) = part.split_once("=") { + if let Some((name, value)) = part.split_once('=') { map.entry(name).or_default().push(value); } } diff --git a/iroh-dns/src/publish.rs b/iroh-dns/src/publish.rs index 91aa71c7a08..3e19b7cb1e3 100644 --- a/iroh-dns/src/publish.rs +++ b/iroh-dns/src/publish.rs @@ -7,8 +7,8 @@ use url::Url; use crate::packet::NodeAnnounce; -pub const IROH_TEST_PKARR_RELAY: &'static str = "https://testdns.iroh.link/pkarr"; -pub const LOCALHOST_PKARR_RELAY: &'static str = "http://localhost:8080/pkarr"; +pub const IROH_TEST_PKARR_RELAY: &str = "https://testdns.iroh.link/pkarr"; +pub const LOCALHOST_PKARR_RELAY: &str = "http://localhost:8080/pkarr"; /// Publisher config pub struct Config { diff --git a/iroh-dns/src/resolve.rs b/iroh-dns/src/resolve.rs index 973dadd805c..739005aaee6 100644 --- a/iroh-dns/src/resolve.rs +++ b/iroh-dns/src/resolve.rs @@ -13,8 +13,8 @@ use tracing::debug; use crate::packet::{NodeAnnounce, IROH_NODE_TXT_LABEL}; pub const IROH_TEST_DNS_IPV4: Ipv4Addr = Ipv4Addr::new(5, 75, 181, 3); -pub const IROH_TEST_DOMAIN: &'static str = "testdns.iroh.link."; -pub const EXAMPLE_DOMAIN: &'static str = "irohdns.example."; +pub const IROH_TEST_DOMAIN: &str = "testdns.iroh.link."; +pub const EXAMPLE_DOMAIN: &str = "irohdns.example."; pub type HickoryResolver = AsyncResolver>; @@ -87,7 +87,7 @@ impl Resolver { } pub async fn resolve_node_by_domain(&self, domain: &str) -> Result { - let name = Name::from_str(&domain)?; + let name = Name::from_str(domain)?; self.resolve_node(name).await } From 3839856684cfdccc202353c0a2dd0f76f7f54cd4 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 27 Feb 2024 12:46:53 +0100 Subject: [PATCH 26/66] chore: remove obsolete files --- iroh-dns/examples/sig0.rs | 201 -------------------------------------- iroh-dns/src/client.rs | 40 -------- 2 files changed, 241 deletions(-) delete mode 100644 iroh-dns/examples/sig0.rs delete mode 100644 iroh-dns/src/client.rs diff --git a/iroh-dns/examples/sig0.rs b/iroh-dns/examples/sig0.rs deleted file mode 100644 index 1509727fb20..00000000000 --- a/iroh-dns/examples/sig0.rs +++ /dev/null @@ -1,201 +0,0 @@ -fn main() { - unimplemented!() -} -// use anyhow::Result; -// use ed25519_dalek::SigningKey; -// use iroh_dns::{packet::NodeAnnounce}; -// use iroh_net::key::SecretKey; -// use url::Url; -// -// #[tokio::main] -// async fn main() -> Result<()> { -// tracing_subscriber::fmt::init(); -// let node_secret = SecretKey::generate(); -// let signing_key = SigningKey::from_bytes(&node_secret.to_bytes()); -// let node_id = node_secret.public(); -// -// println!("node_id {}", node_id); -// -// let home_derp: Url = "https://derp.example".parse()?; -// let msg = NodeAnnounce { -// node_id, -// home_derp: Some(home_derp.clone()), -// home_dns: Default::default(), -// }; -// -// // let name_server: SocketAddr = "127.0.0.1:5353".parse()?; -// // let res = publish_dns_sig0(name_server, msg, signing_key).await; -// let url: Url = "http://localhost:8080".parse()?; -// let res = sig0::publish_https(url, msg, signing_key).await; -// println!("res {res:?}"); -// res -// } -// mod sig0 { -// -// use std::{net::SocketAddr, str::FromStr, time::UNIX_EPOCH}; -// -// use anyhow::{anyhow, bail, Result}; -// use hickory_client::{ -// client::{Client, ClientHandle, SyncClient}, -// op::ResponseCode, -// proto::rr::dnssec::{Algorithm, KeyPair, SigSigner}, -// rr::{rdata::key::KEY, Name}, -// udp::UdpClientConnection, -// }; -// use hickory_proto::serialize::binary::{BinDecodable, BinEncodable}; -// use reqwest::header::CONTENT_TYPE; -// use ring::signature::Ed25519KeyPair; -// use tracing::debug; -// use url::Url; -// -// use iroh_dns::packet::NodeAnnounce; -// -// pub async fn publish_https( -// mut url: Url, -// announce: NodeAnnounce, -// signing_key: ed25519_dalek::SigningKey, -// ) -> Result<()> { -// let public_key = signing_key.verifying_key(); -// let node_zone = create_node_zone_name(public_key)?; -// let mut message = announce.into_hickory_update_message()?; -// println!("message {message:?}"); -// let signer = create_sig0_signer(signing_key, node_zone.clone())?; -// let ts = get_now_timestamp()?; -// // if signer.should_finalize_message(&message) { -// match message.finalize(&signer, ts) { -// Ok(_answer_verifier) => {} -// Err(e) => { -// debug!("could not sign message: {}", e); -// bail!(e) -// } -// } -// // } -// let body = message.to_bytes()?; -// -// url.set_path("/dns-query"); -// let client = reqwest::Client::new(); -// let res = client -// .post(url) -// .body(body) -// .header(CONTENT_TYPE, "application/dns-message") -// .send() -// .await?; -// let headers = res.headers().clone(); -// let body = res.bytes().await?; -// let parsed = hickory_proto::op::Message::from_bytes(&body)?; -// println!("headers {headers:#?}"); -// println!("body {parsed:#?}"); -// -// Ok(()) -// } -// -// fn get_now_timestamp() -> Result { -// let now = match std::time::SystemTime::now().duration_since(UNIX_EPOCH) { -// Ok(now) => now.as_secs(), -// Err(_) => bail!("Current time is before the Unix epoch."), -// }; -// -// // TODO: truncates u64 to u32, error on overflow? -// let now = now as u32; -// Ok(now) -// } -// -// pub fn create_node_zone_name(public_key: ed25519_dalek::VerifyingKey) -> Result { -// let node_id_str = iroh_base::base32::fmt(public_key.as_bytes()); -// let node_zone = Name::from_str(&format!("{}.", node_id_str))?; -// Ok(node_zone) -// } -// -// pub async fn publish_dns_sig0( -// nameserver: SocketAddr, -// announce: NodeAnnounce, -// signing_key: ed25519_dalek::SigningKey, -// ) -> Result<()> { -// let public_key = signing_key.verifying_key(); -// let record = announce.into_hickory_dns_record()?; -// let node_zone = create_node_zone_name(public_key)?; -// -// let signer = create_sig0_signer(signing_key, node_zone.clone())?; -// -// // TODO: HttpsClientConnection silently ignores the signer, there's a TODO in the -// // hickory-client code... -// // let conn = { -// // let client_config = insecure_client_config(); -// // let name_server: SocketAddr = "127.0.0.1:8443".parse()?; -// // let dns_name = "localhost".to_string(); -// // let conn: HttpsClientConnection> = -// // HttpsClientConnection::new(name_server, dns_name, client_config); -// // conn -// // }; -// -// let conn = UdpClientConnection::new(nameserver)?; -// let mut client = { -// let client = SyncClient::with_signer(conn, signer); -// let (client, bg) = client.new_future().await?; -// tokio::task::spawn(bg); -// client -// }; -// -// // Create the record. -// let result = client.create(record, node_zone).await?; -// match result.response_code() { -// ResponseCode::NoError => Ok(()), -// code @ _ => Err(anyhow!("request failed: {code}")), -// } -// } -// -// fn create_sig0_signer(signing_key: ed25519_dalek::SigningKey, zone: Name) -> Result { -// // Create the Hickory DNS SIG(0) signing facility. Generally the signer_name is the label -// // associated with KEY record in the server. -// let public_key = signing_key.verifying_key(); -// let key = Ed25519KeyPair::from_seed_and_public_key( -// &signing_key.to_bytes(), -// public_key.as_bytes(), -// )?; -// let key = KeyPair::from_ed25519(key); -// // Create the RData KEY associated with the key. This example uses defaults for all the -// // KeyTrust, KeyUsage, UpdateScope, Protocol. Many of these have been deprecated in current -// // DNS RFCs, but are still supported by many servers for auth. See auth docs of the remote -// // server for help in understanding it's requirements and support of these options. -// let sig0key = KEY::new( -// Default::default(), -// Default::default(), -// Default::default(), -// Default::default(), -// Algorithm::ED25519, -// key.to_public_bytes()?, -// ); -// Ok(SigSigner::sig0(sig0key, key, zone)) -// } -// -// // fn insecure_client_config() -> Arc { -// // let crypto = rustls::ClientConfig::builder() -// // .with_safe_defaults() -// // .with_custom_certificate_verifier(SkipServerVerification::new()) -// // .with_no_client_auth(); -// // Arc::new(crypto) -// // } -// // /// Dummy certificate verifier that treats any certificate as valid. -// // /// NOTE, such verification is vulnerable to MITM attacks, but convenient for testing. -// // struct SkipServerVerification; -// // -// // impl SkipServerVerification { -// // fn new() -> Arc { -// // Arc::new(Self) -// // } -// // } -// // -// // impl rustls::client::ServerCertVerifier for SkipServerVerification { -// // fn verify_server_cert( -// // &self, -// // _end_entity: &rustls::Certificate, -// // _intermediates: &[rustls::Certificate], -// // _server_name: &rustls::ServerName, -// // _scts: &mut dyn Iterator, -// // _ocsp_response: &[u8], -// // _now: std::time::SystemTime, -// // ) -> Result { -// // Ok(rustls::client::ServerCertVerified::assertion()) -// // } -// // } -// } diff --git a/iroh-dns/src/client.rs b/iroh-dns/src/client.rs deleted file mode 100644 index 9857c3767d7..00000000000 --- a/iroh-dns/src/client.rs +++ /dev/null @@ -1,40 +0,0 @@ -// // use iroh_net::NodeId -// use anyhow::Result; -// use tokio::task::JoinSet; -// use tokio::sync::mpsc; -// -// use crate::{packet::Announcement, NodeId}; -// -// // pub enum ResolveError { -// // NotFound, -// // F -// // } -// -// pub trait Resolver: Send { -// fn resolve(&self, node_id: NodeId) -> impl Stream> + Send + 'static; -// } -// -// pub trait Publisher: Send { -// fn publish(&self, announcement: Announcement) -> impl Future> + Send + 'static; -// } -// -// pub struct Client { -// tasks: JoinSet, -// // queries: -// } -// -// impl Client { -// fn resolve(&self, node_id: NodeId) -> impl Stream> { -// let (tx, rx) = mpsc::channel(1); -// -// } -// async fn resolve_one(&self, node_id: NodeId) -> Result { -// -// } -// fn add_resolver(&mut self, resolver: impl Resolver) { -// self.tasks.spawn(async move { -// -// -// }) -// } -// } From d257c0bba36f807db81f6f01ff93a674f3b068aa Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 28 Feb 2024 23:33:00 +0100 Subject: [PATCH 27/66] refactor: use z32 encoding for pkarr domain names --- Cargo.lock | 1 + iroh-dns/Cargo.toml | 1 + iroh-dns/examples/publish.rs | 19 ++++++++++--------- iroh-dns/src/lib.rs | 14 ++++++++++++++ iroh-dns/src/packet.rs | 4 +++- iroh-dns/src/resolve.rs | 7 +++++-- 6 files changed, 34 insertions(+), 12 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2040d77a28e..0afe6c70093 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2390,6 +2390,7 @@ dependencies = [ "tracing", "tracing-subscriber", "url", + "z32", ] [[package]] diff --git a/iroh-dns/Cargo.toml b/iroh-dns/Cargo.toml index f451eeff8bd..251d4907a3a 100644 --- a/iroh-dns/Cargo.toml +++ b/iroh-dns/Cargo.toml @@ -30,6 +30,7 @@ rustls = "0.21" tokio = { version = "1", features = ["rt", "sync"] } tracing = "0.1" url = { version = "2", features = ["serde"] } +z32 = "1.0.3" [dev-dependencies] clap = { version = "4.5.1", features = ["derive"] } diff --git a/iroh-dns/examples/publish.rs b/iroh-dns/examples/publish.rs index 5602fb3e3f4..9faf5181524 100644 --- a/iroh-dns/examples/publish.rs +++ b/iroh-dns/examples/publish.rs @@ -9,6 +9,7 @@ use iroh_dns::{ packet::IROH_NODE_TXT_LABEL, publish::{Config, Publisher}, resolve::{EXAMPLE_DOMAIN, IROH_TEST_DOMAIN}, + to_z32, }; #[derive(ValueEnum, Clone, Debug, Default, Copy)] @@ -71,20 +72,20 @@ async fn main() -> Result<()> { }; // let an = NodeAnnounce::new(node_id, Some(args.home_derp), vec![]); publisher.publish_addr_info(&info).await?; - println!("published signed record to {}!", publisher.pkarr_relay()); + println!( + "published signed record to {}! Resolve with ", + publisher.pkarr_relay() + ); match args.env { - Env::IrohTest => println!( - "TXT record resolvable at {}", - node_domain(node_id, IROH_TEST_DOMAIN) - ), + Env::IrohTest => println!("dig {} TXT", node_domain(&node_id, IROH_TEST_DOMAIN)), Env::LocalDev => println!( - "TXT record resolvable at {}", - node_domain(node_id, EXAMPLE_DOMAIN) + "dig @localhost -p 5353 {} TXT", + node_domain(&node_id, EXAMPLE_DOMAIN) ), } Ok(()) } -fn node_domain(node_id: NodeId, origin: &str) -> String { - format!("{}.{}.{}", IROH_NODE_TXT_LABEL, node_id, origin) +fn node_domain(node_id: &NodeId, origin: &str) -> String { + format!("{}.{}.{}", IROH_NODE_TXT_LABEL, to_z32(node_id), origin) } diff --git a/iroh-dns/src/lib.rs b/iroh-dns/src/lib.rs index 20c55182476..915a69e2864 100644 --- a/iroh-dns/src/lib.rs +++ b/iroh-dns/src/lib.rs @@ -1,4 +1,18 @@ +use anyhow::{anyhow, Result}; +use iroh_net::NodeId; + pub mod discovery; pub mod packet; pub mod publish; pub mod resolve; + +pub fn to_z32(node_id: &NodeId) -> String { + z32::encode(node_id.as_bytes()) +} + +pub fn from_z32(s: &str) -> Result { + let bytes = z32::decode(s.as_bytes()).map_err(|_| anyhow!("invalid z32"))?; + let bytes: &[u8; 32] = &bytes.try_into().map_err(|_| anyhow!("not 32 bytes long"))?; + let node_id = NodeId::from_bytes(bytes)?; + Ok(node_id) +} diff --git a/iroh-dns/src/packet.rs b/iroh-dns/src/packet.rs index af95db509c7..aae1aa66ea3 100644 --- a/iroh-dns/src/packet.rs +++ b/iroh-dns/src/packet.rs @@ -6,6 +6,8 @@ use hickory_proto::error::ProtoError; use iroh_net::{AddrInfo, NodeAddr, NodeId}; use url::Url; +use crate::from_z32; + pub const IROH_ROOT_ZONE: &str = "iroh"; pub const IROH_NODE_TXT_LABEL: &str = "_iroh_node"; pub const DEFAULT_TTL: u32 = 30; @@ -231,7 +233,7 @@ fn is_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { return None; } let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; - let node_id = NodeId::from_str(label).ok()?; + let node_id = from_z32(label).ok()?; Some(node_id) } diff --git a/iroh-dns/src/resolve.rs b/iroh-dns/src/resolve.rs index 739005aaee6..47fbe4f8215 100644 --- a/iroh-dns/src/resolve.rs +++ b/iroh-dns/src/resolve.rs @@ -10,7 +10,10 @@ use hickory_resolver::{ use iroh_net::{AddrInfo, NodeAddr, NodeId}; use tracing::debug; -use crate::packet::{NodeAnnounce, IROH_NODE_TXT_LABEL}; +use crate::{ + packet::{NodeAnnounce, IROH_NODE_TXT_LABEL}, + to_z32, +}; pub const IROH_TEST_DNS_IPV4: Ipv4Addr = Ipv4Addr::new(5, 75, 181, 3); pub const IROH_TEST_DOMAIN: &str = "testdns.iroh.link."; @@ -93,7 +96,7 @@ impl Resolver { pub async fn resolve_node_by_id(&self, node_id: NodeId) -> Result { debug!(?node_id, "resolve node by id"); - let name = Name::parse(&node_id.to_string(), Some(&self.default_node_origin))?; + let name = Name::parse(&to_z32(&node_id), Some(&self.default_node_origin))?; let addr = self.resolve_node(name).await; debug!(?node_id, ?addr, "resolved"); let addr = addr?; From b4b85dec273af3f4745ea55f78057af567e1e52d Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 4 Mar 2024 14:27:43 +0100 Subject: [PATCH 28/66] refactor: cleanup --- iroh-dns/src/discovery.rs | 26 ++++++++++++++++++++------ iroh-dns/src/resolve.rs | 11 ++++------- iroh/src/node.rs | 2 +- 3 files changed, 25 insertions(+), 14 deletions(-) diff --git a/iroh-dns/src/discovery.rs b/iroh-dns/src/discovery.rs index a99ab9acb1c..e1ba53d0ec9 100644 --- a/iroh-dns/src/discovery.rs +++ b/iroh-dns/src/discovery.rs @@ -1,10 +1,12 @@ use std::sync::Arc; use anyhow::Result; -use futures::future::{BoxFuture, FutureExt}; -use iroh_net::key::SecretKey; -use iroh_net::magicsock::Discovery; -use iroh_net::{AddrInfo, NodeId}; +use futures::{future::FutureExt, stream::BoxStream, StreamExt}; +use iroh_net::{ + discovery::{Discovery, DiscoveryItem}, + key::SecretKey, + AddrInfo, MagicEndpoint, NodeId, +}; use tracing::warn; use crate::publish::{self, Publisher}; @@ -49,7 +51,19 @@ impl Discovery for DnsDiscovery { } } - fn resolve<'a>(&'a self, node_id: &'a NodeId) -> BoxFuture<'a, Result> { - self.resolver.resolve_node_by_id(*node_id).boxed() + fn resolve<'a>( + &'a self, + _ep: MagicEndpoint, + node_id: NodeId, + ) -> Option>> { + let fut = async move { + let addr_info = self.resolver.resolve_node_by_id(node_id).await?; + Ok(DiscoveryItem { + provenance: "iroh-dns", + last_updated: None, + addr_info, + }) + }; + Some(fut.into_stream().boxed()) } } diff --git a/iroh-dns/src/resolve.rs b/iroh-dns/src/resolve.rs index 47fbe4f8215..4f0ee9ac650 100644 --- a/iroh-dns/src/resolve.rs +++ b/iroh-dns/src/resolve.rs @@ -4,8 +4,7 @@ use anyhow::Result; use hickory_proto::error::ProtoError; use hickory_resolver::{ config::{NameServerConfigGroup, ResolverConfig, ResolverOpts}, - name_server::{GenericConnector, TokioRuntimeProvider}, - AsyncResolver, Name, + AsyncResolver, Name, TokioAsyncResolver, }; use iroh_net::{AddrInfo, NodeAddr, NodeId}; use tracing::debug; @@ -19,8 +18,6 @@ pub const IROH_TEST_DNS_IPV4: Ipv4Addr = Ipv4Addr::new(5, 75, 181, 3); pub const IROH_TEST_DOMAIN: &str = "testdns.iroh.link."; pub const EXAMPLE_DOMAIN: &str = "irohdns.example."; -pub type HickoryResolver = AsyncResolver>; - /// Resolver config pub struct Config { name_servers: NameServerConfigGroup, @@ -67,8 +64,8 @@ impl Config { #[derive(derive_more::Debug, Clone)] pub struct Resolver { default_node_origin: Name, - #[debug("HickoryResolver")] - dns_resolver: HickoryResolver, + #[debug("TokioAsyncResolver")] + dns_resolver: TokioAsyncResolver, } impl Resolver { @@ -85,7 +82,7 @@ impl Resolver { }) } - pub fn resolver(&self) -> &HickoryResolver { + pub fn resolver(&self) -> &TokioAsyncResolver { &self.dns_resolver } diff --git a/iroh/src/node.rs b/iroh/src/node.rs index 63d550c81a0..bba32a82410 100644 --- a/iroh/src/node.rs +++ b/iroh/src/node.rs @@ -35,7 +35,7 @@ use iroh_gossip::net::{Gossip, GOSSIP_ALPN}; use iroh_io::AsyncSliceReader; use iroh_net::derp::DerpUrl; use iroh_net::magic_endpoint::get_alpn; -use iroh_net::magicsock::Discovery; +use iroh_net::discovery::Discovery; use iroh_net::magicsock::LocalEndpointsStream; use iroh_net::util::AbortingJoinHandle; use iroh_net::{ From 1389d449dabcde4c84470b57b6d9bcb98acc66fe Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 7 Mar 2024 17:05:48 +0100 Subject: [PATCH 29/66] refactor: move iroh-dns into iroh-net, add docs, cleanup --- Cargo.lock | 41 +- Cargo.toml | 1 - iroh-dns/Cargo.toml | 38 -- iroh-dns/examples/publish.rs | 91 ----- iroh-dns/examples/resolve.rs | 59 --- iroh-dns/src/discovery.rs | 69 ---- iroh-dns/src/lib.rs | 18 - iroh-dns/src/packet.rs | 364 ------------------ iroh-dns/src/publish.rs | 86 ----- iroh-dns/src/resolve.rs | 117 ------ iroh-net/Cargo.toml | 5 +- iroh-net/src/discovery.rs | 3 + iroh-net/src/discovery/dns.rs | 58 +++ iroh-net/src/discovery/pkarr_relay_publish.rs | 113 ++++++ iroh-net/src/dns.rs | 11 +- iroh-net/src/dns/node_info.rs | 246 ++++++++++++ iroh-net/src/lib.rs | 2 +- iroh/Cargo.toml | 1 - iroh/src/commands/start.rs | 11 +- iroh/src/node.rs | 2 +- 20 files changed, 448 insertions(+), 888 deletions(-) delete mode 100644 iroh-dns/Cargo.toml delete mode 100644 iroh-dns/examples/publish.rs delete mode 100644 iroh-dns/examples/resolve.rs delete mode 100644 iroh-dns/src/discovery.rs delete mode 100644 iroh-dns/src/lib.rs delete mode 100644 iroh-dns/src/packet.rs delete mode 100644 iroh-dns/src/publish.rs delete mode 100644 iroh-dns/src/resolve.rs create mode 100644 iroh-net/src/discovery/dns.rs create mode 100644 iroh-net/src/discovery/pkarr_relay_publish.rs create mode 100644 iroh-net/src/dns/node_info.rs diff --git a/Cargo.lock b/Cargo.lock index 0afe6c70093..582c1730a1a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1784,29 +1784,21 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "091a6fbccf4860009355e3efc52ff4acf37a63489aad7435372d44ceeb6fbbcf" dependencies = [ "async-trait", - "bytes", "cfg-if", "data-encoding", "enum-as-inner", "futures-channel", "futures-io", "futures-util", - "h2", - "http 0.2.11", "idna 0.4.0", "ipnet", "once_cell", "rand", - "ring 0.16.20", - "rustls", - "rustls-pemfile", "thiserror", "tinyvec", "tokio", - "tokio-rustls", "tracing", "url", - "webpki-roots", ] [[package]] @@ -1824,13 +1816,10 @@ dependencies = [ "parking_lot", "rand", "resolv-conf", - "rustls", "smallvec", "thiserror", "tokio", - "tokio-rustls", "tracing", - "webpki-roots", ] [[package]] @@ -2237,7 +2226,6 @@ dependencies = [ "indicatif", "iroh-base", "iroh-bytes", - "iroh-dns", "iroh-gossip", "iroh-io", "iroh-metrics", @@ -2367,32 +2355,6 @@ dependencies = [ "tracing-subscriber", ] -[[package]] -name = "iroh-dns" -version = "0.12.0" -dependencies = [ - "anyhow", - "clap", - "derive_more", - "ed25519-dalek", - "futures", - "hex", - "hickory-proto", - "hickory-resolver", - "iroh-base", - "iroh-net", - "parking_lot", - "pkarr", - "reqwest", - "ring 0.16.20", - "rustls", - "tokio", - "tracing", - "tracing-subscriber", - "url", - "z32", -] - [[package]] name = "iroh-gossip" version = "0.12.0" @@ -2478,6 +2440,7 @@ dependencies = [ "futures", "governor", "hex", + "hickory-proto", "hickory-resolver", "hostname", "http 1.0.0", @@ -2496,6 +2459,7 @@ dependencies = [ "num_enum", "once_cell", "parking_lot", + "pkarr", "postcard", "pretty_assertions", "proptest", @@ -2539,6 +2503,7 @@ dependencies = [ "windows 0.51.1", "wmi", "x509-parser", + "z32", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index d1cdd26c07b..751f092c909 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -3,7 +3,6 @@ members = [ "iroh", "iroh-bytes", "iroh-base", - "iroh-dns", "iroh-gossip", "iroh-metrics", "iroh-net", diff --git a/iroh-dns/Cargo.toml b/iroh-dns/Cargo.toml deleted file mode 100644 index 251d4907a3a..00000000000 --- a/iroh-dns/Cargo.toml +++ /dev/null @@ -1,38 +0,0 @@ -[package] -name = "iroh-dns" -version = "0.12.0" -edition = "2021" -readme = "README.md" -description = "DNS resolver and pkarr publisher for iroh" -license = "MIT OR Apache-2.0" -authors = ["n0 team"] -repository = "https://github.com/n0-computer/iroh" -keywords = ["networking", "p2p", "holepunching", "ipfs"] - -# Sadly this also needs to be updated in .github/workflows/ci.yml -rust-version = "1.75" - -[dependencies] -anyhow = "1.0.80" -derive_more = { version = "1.0.0-beta.1", features = ["debug", "display"] } -ed25519-dalek = { version = "2.1.1", features = ["pkcs8"] } -futures = "0.3.30" -hex = "0.4.3" -hickory-proto = { version = "0.24.0", features = ["dnssec", "ring"] } -hickory-resolver = { version = "0.24.0", features = ["dns-over-https", "dns-over-tls", "tokio-rustls", "webpki-roots", "dns-over-rustls", "dns-over-https-rustls"] } -iroh-base = { version = "0.12.0", path = "../iroh-base", default_features = false, features = ["base32"] } -iroh-net = { version = "0.12.0", path = "../iroh-net", default_features = false } -parking_lot = "0.12.1" -pkarr = { version = "1.1.1", features = ["async", "relay"], default_features = false } -reqwest = { version = "0.11.24", default_features = false, features = ["rustls-tls"] } -ring = "0.16" -rustls = "0.21" -tokio = { version = "1", features = ["rt", "sync"] } -tracing = "0.1" -url = { version = "2", features = ["serde"] } -z32 = "1.0.3" - -[dev-dependencies] -clap = { version = "4.5.1", features = ["derive"] } -tokio = { version = "1", features = ["full"] } -tracing-subscriber = "0.3" diff --git a/iroh-dns/examples/publish.rs b/iroh-dns/examples/publish.rs deleted file mode 100644 index 9faf5181524..00000000000 --- a/iroh-dns/examples/publish.rs +++ /dev/null @@ -1,91 +0,0 @@ -use std::str::FromStr; - -use anyhow::{bail, Result}; -use clap::{Parser, ValueEnum}; -use iroh_net::{key::SecretKey, AddrInfo, NodeId}; -use url::Url; - -use iroh_dns::{ - packet::IROH_NODE_TXT_LABEL, - publish::{Config, Publisher}, - resolve::{EXAMPLE_DOMAIN, IROH_TEST_DOMAIN}, - to_z32, -}; - -#[derive(ValueEnum, Clone, Debug, Default, Copy)] -pub enum Env { - /// Use the irohdns test server at testdns.iroh.link - #[default] - IrohTest, - /// Use a relay listening at localhost:8080 - LocalDev, -} - -/// Publish a record to an irohdns server. -/// -/// You have to set the IROH_SECRET environment variable to the node secret for which to publish. -#[derive(Parser, Debug)] -struct Cli { - /// Environment to publish to. - #[clap(value_enum, short, long, default_value_t = Env::IrohTest)] - env: Env, - /// Relay URL. If set, the --env option will be ignored. - #[clap(short, long, conflicts_with = "env")] - relay: Option, - /// Home Derp server to publish for this node - #[clap(short, long)] - derp_url: Url, - /// Create a new node secret if IROH_SECRET is unset. Only for development / debugging. - #[clap(short, long)] - create: bool, -} - -#[tokio::main] -async fn main() -> Result<()> { - tracing_subscriber::fmt::init(); - let args = Cli::parse(); - let secret_key = match std::env::var("IROH_SECRET") { - Ok(s) => SecretKey::from_str(&s)?, - Err(_) if args.create => { - let s = SecretKey::generate(); - println!("Generated a new node secret. To reuse, set"); - println!("IROH_SECRET={s}"); - s - } - Err(_) => { - bail!("Environtment variable IROH_SECRET is not set. To create a new secret, use the --create option.") - } - }; - let node_id = secret_key.public(); - println!("node: {node_id}"); - println!("derp: {}", args.derp_url); - let config = match (args.relay, args.env) { - (Some(pkarr_relay), _) => Config::new(secret_key, pkarr_relay), - (None, Env::IrohTest) => Config::with_iroh_test(secret_key), - (None, Env::LocalDev) => Config::localhost_dev(secret_key), - }; - let publisher = Publisher::new(config); - - let info = AddrInfo { - derp_url: Some(args.derp_url.into()), - direct_addresses: Default::default(), - }; - // let an = NodeAnnounce::new(node_id, Some(args.home_derp), vec![]); - publisher.publish_addr_info(&info).await?; - println!( - "published signed record to {}! Resolve with ", - publisher.pkarr_relay() - ); - match args.env { - Env::IrohTest => println!("dig {} TXT", node_domain(&node_id, IROH_TEST_DOMAIN)), - Env::LocalDev => println!( - "dig @localhost -p 5353 {} TXT", - node_domain(&node_id, EXAMPLE_DOMAIN) - ), - } - Ok(()) -} - -fn node_domain(node_id: &NodeId, origin: &str) -> String { - format!("{}.{}.{}", IROH_NODE_TXT_LABEL, to_z32(node_id), origin) -} diff --git a/iroh-dns/examples/resolve.rs b/iroh-dns/examples/resolve.rs deleted file mode 100644 index 91a7fdcd3e2..00000000000 --- a/iroh-dns/examples/resolve.rs +++ /dev/null @@ -1,59 +0,0 @@ -use clap::Parser; -use clap::ValueEnum; -use iroh_dns::resolve::{Config, Resolver}; -use iroh_net::NodeId; - -#[derive(ValueEnum, Clone, Debug, Default)] -pub enum Env { - /// Use cloudflare and the irohdns test server at testdns.iroh.link - #[default] - IrohTest, - /// Use a localhost domain server listening on port 5353 - LocalDev, -} - -#[derive(Debug, Parser)] -struct Cli { - #[clap(value_enum, short, long, default_value_t = Env::IrohTest)] - env: Env, - #[clap(subcommand)] - command: Command, -} - -#[derive(Debug, Parser)] -enum Command { - /// Resolve node info by node id. - Node { node_id: NodeId }, - /// Resolve node info by domain. - Domain { domain: String }, -} - -#[tokio::main] -async fn main() -> anyhow::Result<()> { - let args = Cli::parse(); - let config = match args.env { - Env::IrohTest => Config::with_cloudflare_and_iroh_test(), - Env::LocalDev => Config::localhost_dev(), - }; - let resolver = Resolver::new(config)?; - match args.command { - Command::Node { node_id } => { - let addr = resolver.resolve_node_by_id(node_id).await?; - let derp_url = addr.derp_url.map(|u| u.to_string()).unwrap_or_default(); - println!("node_id: {node_id}"); - println!("derp_url: {derp_url}"); - } - Command::Domain { domain } => { - let addr = resolver.resolve_node_by_domain(&domain).await?; - let node_id = addr.node_id; - let derp_url = addr - .info - .derp_url - .map(|u| u.to_string()) - .unwrap_or_default(); - println!("node_id: {node_id}"); - println!("derp_url: {derp_url}"); - } - } - Ok(()) -} diff --git a/iroh-dns/src/discovery.rs b/iroh-dns/src/discovery.rs deleted file mode 100644 index e1ba53d0ec9..00000000000 --- a/iroh-dns/src/discovery.rs +++ /dev/null @@ -1,69 +0,0 @@ -use std::sync::Arc; - -use anyhow::Result; -use futures::{future::FutureExt, stream::BoxStream, StreamExt}; -use iroh_net::{ - discovery::{Discovery, DiscoveryItem}, - key::SecretKey, - AddrInfo, MagicEndpoint, NodeId, -}; -use tracing::warn; - -use crate::publish::{self, Publisher}; -use crate::resolve::{self, Resolver}; - -#[derive(Debug)] -pub struct DnsDiscovery { - publisher: Option>, - resolver: Resolver, -} - -impl DnsDiscovery { - pub fn new(resolver: Resolver, publisher: Option>) -> Self { - Self { - resolver, - publisher, - } - } - pub fn with_iroh_test(secret_key: Option) -> Result { - let publisher = - secret_key.map(|k| Arc::new(Publisher::new(publish::Config::with_iroh_test(k)))); - let resolver = Resolver::new(resolve::Config::with_cloudflare_and_iroh_test())?; - Ok(Self::new(resolver, publisher)) - } - pub fn localhost_dev(secret_key: Option) -> Result { - let publisher = - secret_key.map(|k| Arc::new(Publisher::new(publish::Config::localhost_dev(k)))); - let resolver = Resolver::new(resolve::Config::localhost_dev())?; - Ok(Self::new(resolver, publisher)) - } -} - -impl Discovery for DnsDiscovery { - fn publish(&self, info: &AddrInfo) { - if let Some(publisher) = self.publisher.clone() { - let info = info.clone(); - tokio::task::spawn(async move { - if let Err(err) = publisher.publish_addr_info(&info).await { - warn!("failed to publish address update: {err:?}"); - } - }); - } - } - - fn resolve<'a>( - &'a self, - _ep: MagicEndpoint, - node_id: NodeId, - ) -> Option>> { - let fut = async move { - let addr_info = self.resolver.resolve_node_by_id(node_id).await?; - Ok(DiscoveryItem { - provenance: "iroh-dns", - last_updated: None, - addr_info, - }) - }; - Some(fut.into_stream().boxed()) - } -} diff --git a/iroh-dns/src/lib.rs b/iroh-dns/src/lib.rs deleted file mode 100644 index 915a69e2864..00000000000 --- a/iroh-dns/src/lib.rs +++ /dev/null @@ -1,18 +0,0 @@ -use anyhow::{anyhow, Result}; -use iroh_net::NodeId; - -pub mod discovery; -pub mod packet; -pub mod publish; -pub mod resolve; - -pub fn to_z32(node_id: &NodeId) -> String { - z32::encode(node_id.as_bytes()) -} - -pub fn from_z32(s: &str) -> Result { - let bytes = z32::decode(s.as_bytes()).map_err(|_| anyhow!("invalid z32"))?; - let bytes: &[u8; 32] = &bytes.try_into().map_err(|_| anyhow!("not 32 bytes long"))?; - let node_id = NodeId::from_bytes(bytes)?; - Ok(node_id) -} diff --git a/iroh-dns/src/packet.rs b/iroh-dns/src/packet.rs deleted file mode 100644 index aae1aa66ea3..00000000000 --- a/iroh-dns/src/packet.rs +++ /dev/null @@ -1,364 +0,0 @@ -use std::{collections::HashMap, fmt::Display, str::FromStr}; - -// use hickory_proto::rr::Name; -use anyhow::{anyhow, bail, Result}; -use hickory_proto::error::ProtoError; -use iroh_net::{AddrInfo, NodeAddr, NodeId}; -use url::Url; - -use crate::from_z32; - -pub const IROH_ROOT_ZONE: &str = "iroh"; -pub const IROH_NODE_TXT_LABEL: &str = "_iroh_node"; -pub const DEFAULT_TTL: u32 = 30; - -pub const ATTR_DERP: &str = "derp"; -pub const ATTR_NODE_ID: &str = "node"; -pub const ATTR_DNS: &str = "dns"; - -#[derive(derive_more::Debug, Clone, Eq, PartialEq)] -pub struct NodeAnnounce { - pub node_id: NodeId, - #[debug("{:?}", self.home_derp.as_ref().map(|s| s.to_string()))] - pub home_derp: Option, - pub home_dns: Vec, -} - -impl From for NodeAddr { - fn from(value: NodeAnnounce) -> Self { - NodeAddr { - node_id: value.node_id, - info: value.into(), - } - } -} - -impl From for AddrInfo { - fn from(value: NodeAnnounce) -> Self { - AddrInfo { - derp_url: value.home_derp.map(|u| u.into()), - direct_addresses: Default::default(), - } - } -} - -impl NodeAnnounce { - pub fn new(node_id: NodeId, derp: Option, dns: Vec) -> Self { - Self { - node_id, - home_derp: derp, - home_dns: dns, - } - } - - pub fn to_attr_string(&self) -> String { - let mut attrs = vec![]; - attrs.push(fmt_attr(ATTR_NODE_ID, self.node_id)); - if let Some(derp) = &self.home_derp { - attrs.push(fmt_attr(ATTR_DERP, derp)); - } - for dns in &self.home_dns { - attrs.push(fmt_attr(ATTR_DNS, dns)); - } - attrs.join(" ") - } - - pub fn zone(&self, absolute: bool) -> String { - match absolute { - true => format!("{}.{}.", self.node_id, IROH_ROOT_ZONE), - false => format!("{}.{}", self.node_id, IROH_ROOT_ZONE), - } - } - - pub fn hickory_zone(&self, absolute: bool) -> Result { - hickory_proto::rr::Name::from_str(&self.zone(absolute)) - } - - pub fn into_hickory_answers_message(&self) -> Result { - use hickory_proto::op; - let record = self.into_hickory_dns_record()?; - let mut packet = op::Message::new(); - packet.answers_mut().push(record); - Ok(packet) - } - - pub fn into_hickory_update_message(&self) -> Result { - use hickory_proto::{op, rr}; - let record = self.into_hickory_dns_record()?; - let zone = rr::Name::from_str(&self.zone(true))?; - let message = op::update_message::create(record.into(), zone, false); - Ok(message) - } - - pub fn into_hickory_dns_record(&self) -> Result { - use hickory_proto::rr; - let origin = rr::Name::from_str(IROH_ROOT_ZONE)?; - self.into_hickory_dns_record_with_origin(&origin) - } - - pub fn into_hickory_dns_record_with_origin( - &self, - origin: &hickory_proto::rr::Name, - ) -> Result { - use hickory_proto::rr; - let zone = rr::Name::from_str(&self.node_id.to_string())?; - let zone = zone.append_domain(origin)?; - let name = rr::Name::parse(IROH_NODE_TXT_LABEL, Some(&zone))?; - let txt_value = self.to_attr_string(); - let txt_data = rr::rdata::TXT::new(vec![txt_value]); - let rdata = rr::RData::TXT(txt_data); - let record = rr::Record::from_rdata(name, DEFAULT_TTL, rdata); - Ok(record) - } - - pub fn into_pkarr_dns_packet(&self) -> Result> { - use pkarr::dns::{self, rdata}; - let mut packet = dns::Packet::new_reply(0); - // let name = format!("{}.{}", IROH_NODE_TXT_NAME, self.zone()); - let name = IROH_NODE_TXT_LABEL; - let name = dns::Name::new(name)?.into_owned(); - let txt_value = self.to_attr_string(); - let txt_data = rdata::TXT::new().with_string(&txt_value)?.into_owned(); - let rdata = rdata::RData::TXT(txt_data); - packet.answers.push(dns::ResourceRecord::new( - name, - dns::CLASS::IN, - DEFAULT_TTL, - rdata, - )); - Ok(packet) - } - - pub fn into_pkarr_signed_packet( - &self, - signing_key: &ed25519_dalek::SigningKey, - ) -> Result { - // TODO: PR to pkarr for impl From for pkarr::Keypair - let keypair = pkarr::Keypair::from_secret_key(&signing_key.to_bytes()); - let packet = self.into_pkarr_dns_packet()?; - let signed_packet = pkarr::SignedPacket::from_packet(&keypair, &packet)?; - Ok(signed_packet) - } - - pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { - use pkarr::dns::{self, rdata::RData}; - let pubkey = packet.public_key(); - let pubkey_z32 = pubkey.to_z32(); - let node_id = NodeId::from(*pubkey.verifying_key()); - let zone = dns::Name::new(&pubkey_z32)?; - let inner = packet.packet(); - let txt_record = inner - .answers - .iter() - .find_map(|rr| match &rr.rdata { - RData::TXT(txt) => match rr.name.without(&zone) { - Some(name) if name.to_string() == IROH_NODE_TXT_LABEL => Some(txt), - Some(_) | None => None, - }, - _ => None, - }) - .ok_or_else(|| anyhow!("missing _iroh_node txt record"))?; - - let txt_record = txt_record.to_owned(); - let txt = String::try_from(txt_record)?; - let an = Self::parse_from_attributes(&txt)?; - if an.node_id != node_id { - bail!("node id mismatch between record name and TXT value"); - } - Ok(an) - } - - pub fn from_hickory_answers_message(message: &hickory_proto::op::Message) -> Result { - Self::from_hickory_records(message.answers()) - } - - pub fn from_hickory_lookup(lookup: &hickory_resolver::lookup::Lookup) -> Result { - Self::from_hickory_records(lookup.records()) - } - - pub fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { - use hickory_proto::rr; - let (node_id, txt) = records - .iter() - .find_map(|rr| match rr.data() { - Some(rr::RData::TXT(txt)) => { - is_hickory_node_info_name(rr.name()).map(|node_id| (node_id, txt)) - } - _ => None, - }) - .ok_or_else(|| anyhow!("no TXT record with name _iroh_node.b32encodedpubkey found"))?; - let attr_str = txt.to_string(); - let an = Self::parse_from_attributes(&attr_str)?; - if an.node_id != node_id { - bail!("node id mismatch between record name and TXT value"); - } - Ok(an) - } - - pub fn parse_from_attributes(attrs: &str) -> Result { - let attrs = parse_attrs(attrs); - let Some(node) = attrs.get(ATTR_NODE_ID) else { - bail!("missing required node attr"); - }; - if node.len() != 1 { - bail!("more than one node attr is not allowed"); - } - let node_id = NodeId::from_str(node[0])?; - let home_derp: Option = attrs - .get(ATTR_DERP) - .into_iter() - .flatten() - .find_map(|x| Url::parse(x).ok()); - let home_dns: Vec = attrs - .get(ATTR_DNS) - .into_iter() - .flat_map(|x| x.iter()) - .map(|s| s.to_string()) - .collect(); - Ok(Self { - node_id, - home_derp, - home_dns, - }) - } -} - -fn is_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { - if name.num_labels() < 2 { - return None; - } - let mut labels = name.iter(); - let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; - if label != IROH_NODE_TXT_LABEL { - return None; - } - let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; - let node_id = from_z32(label).ok()?; - Some(node_id) -} - -fn parse_attrs<'a>(s: &'a str) -> HashMap<&'a str, Vec<&'a str>> { - let mut map: HashMap<&'a str, Vec<&'a str>> = HashMap::new(); - let parts = s.split(' '); - for part in parts { - if let Some((name, value)) = part.split_once('=') { - map.entry(name).or_default().push(value); - } - } - map -} - -fn fmt_attr(label: &str, value: impl Display) -> String { - format!("{label}={value}") -} - -// fn simple_dns_to_hickory( -// signed_packet: &pkarr::SignedPacket, -// ) -> anyhow::Result { -// let encoded = signed_packet.encoded_packet(); -// let parsed1 = pkarr::dns::Packet::parse(&encoded)?; -// println!("simple_dns {parsed1:#?}"); -// let parsed2 = hickory_proto::op::Message::from_bytes(&encoded)?; -// println!("hickory {parsed2:#?}"); -// Ok(parsed2) -// } - -#[cfg(test)] -mod tests { - // TODO: The tests are not comprehensive in any way, more like examples while getting things to - // work - - use std::str::FromStr; - - use hickory_proto::serialize::binary::{BinDecodable, BinEncodable}; - use url::Url; - - use super::*; - - #[test] - fn create_signed_packet() -> Result<()> { - let signing_key = iroh_net::key::SecretKey::generate(); - let node_id = signing_key.public(); - let home_derp: Url = "https://derp.example/".parse()?; - let an = NodeAnnounce { - node_id, - home_derp: Some(home_derp), - home_dns: vec![], - }; - let signing_key = ed25519_dalek::SigningKey::from_bytes(&signing_key.to_bytes()); - let sp = an.into_pkarr_signed_packet(&signing_key)?; - println!("sp {sp:#?}"); - println!("packet {:#?}", sp.packet()); - let an2 = NodeAnnounce::from_pkarr_signed_packet(&sp)?; - assert_eq!(an, an2); - let _p = an.into_hickory_answers_message()?; - Ok(()) - } - - #[test] - fn convert2() -> anyhow::Result<()> { - let key = iroh_net::key::SecretKey::generate(); - let node_id = key.public(); - let home_derp: Url = "https://derp.example".parse()?; - let a = NodeAnnounce { - node_id, - home_derp: Some(home_derp), - home_dns: Default::default(), - }; - let packet_simpdns = a.into_hickory_answers_message()?; - let packet_hickory = a.into_hickory_answers_message()?; - let buf_simpdns = packet_simpdns.to_bytes()?; - let buf_hickory = packet_hickory.to_bytes()?; - println!( - "simple_dns {} {}", - buf_simpdns.len(), - hex::encode(&buf_simpdns) - ); - println!( - "hickory {} {}", - buf_hickory.len(), - hex::encode(&buf_hickory) - ); - let _simpdns_from_hickory = pkarr::dns::Packet::parse(&buf_hickory)?; - let _hickory_form_simpdns = hickory_proto::op::Message::from_bytes(&buf_simpdns)?; - - Ok(()) - } - - #[test] - fn convert3() -> anyhow::Result<()> { - use hickory_proto as proto; - use pkarr::dns; - let ttl = 300; - let (packet1, bytes1) = { - use dns::rdata; - let mut packet = dns::Packet::new_reply(0); - let name = dns::Name::new("foo")?; - let rdata = rdata::RData::TXT(rdata::TXT::new().with_string("bar")?); - let record = dns::ResourceRecord::new(name, dns::CLASS::IN, ttl, rdata); - packet.answers.push(record); - let bytes = packet.build_bytes_vec()?; - (packet, bytes) - }; - let (packet2, bytes2) = { - use proto::rr; - use proto::serialize::binary::BinEncodable; - let mut packet = proto::op::Message::new(); - let name = rr::Name::from_str("foo")?; - let rdata = rr::RData::TXT(rr::rdata::TXT::new(vec!["bar".to_string()])); - let mut record = rr::Record::with(name, rr::RecordType::TXT, ttl); - record.set_data(Some(rdata)); - packet.answers_mut().push(record); - let bytes = packet.to_bytes()?; - (packet, bytes) - }; - println!("simple_dns deb {:#?}", packet1); - println!("hickory deb {:#?}", packet2); - println!("simple_dns len {}", bytes1.len()); - println!("hickory len {}", bytes2.len()); - println!("simple_dns hex {}", hex::encode(&bytes1)); - println!("hickory hex {}", hex::encode(&bytes2)); - - Ok(()) - } -} diff --git a/iroh-dns/src/publish.rs b/iroh-dns/src/publish.rs deleted file mode 100644 index 3e19b7cb1e3..00000000000 --- a/iroh-dns/src/publish.rs +++ /dev/null @@ -1,86 +0,0 @@ -use anyhow::Result; -use ed25519_dalek::SigningKey; -use iroh_net::{key::SecretKey, AddrInfo, NodeId}; -use parking_lot::RwLock; -use pkarr::PkarrClient; -use url::Url; - -use crate::packet::NodeAnnounce; - -pub const IROH_TEST_PKARR_RELAY: &str = "https://testdns.iroh.link/pkarr"; -pub const LOCALHOST_PKARR_RELAY: &str = "http://localhost:8080/pkarr"; - -/// Publisher config -pub struct Config { - pub secret_key: SecretKey, - pub pkarr_relay: Url, -} - -impl Config { - pub fn new(secret_key: SecretKey, pkarr_relay: Url) -> Self { - Self { - secret_key, - pkarr_relay, - } - } - - pub fn with_iroh_test(secret_key: SecretKey) -> Self { - let pkarr_relay: Url = IROH_TEST_PKARR_RELAY.parse().expect("url is valid"); - Self::new(secret_key, pkarr_relay) - } - - pub fn localhost_dev(secret_key: SecretKey) -> Self { - let pkarr_relay: Url = LOCALHOST_PKARR_RELAY.parse().expect("url is valid"); - Self::new(secret_key, pkarr_relay) - } -} - -/// Publish node announces to a pkarr relay. -#[derive(derive_more::Debug)] -pub struct Publisher { - node_id: NodeId, - #[debug("SigningKey")] - signing_key: SigningKey, - #[debug("{}", self.pkarr_relay)] - pkarr_relay: Url, - #[debug("PkarrClient")] - pkarr_client: PkarrClient, - #[debug(skip)] - last_announce: RwLock>, -} - -impl Publisher { - pub fn new(config: Config) -> Self { - let pkarr_client = PkarrClient::builder().build(); - let node_id = config.secret_key.public(); - let signing_key = ed25519_dalek::SigningKey::from_bytes(&config.secret_key.to_bytes()); - Self { - node_id, - signing_key, - pkarr_relay: config.pkarr_relay, - pkarr_client, - last_announce: Default::default(), - } - } - - pub async fn publish_addr_info(&self, info: &AddrInfo) -> Result<()> { - let an = NodeAnnounce::new( - self.node_id, - info.derp_url.as_ref().map(|u| u.clone().into()), - Default::default(), - ); - if self.last_announce.read().as_ref() == Some(&an) { - return Ok(()); - } - let _ = self.last_announce.write().insert(an.clone()); - let signed_packet = an.into_pkarr_signed_packet(&self.signing_key)?; - self.pkarr_client - .relay_put(&self.pkarr_relay, &signed_packet) - .await?; - Ok(()) - } - - pub fn pkarr_relay(&self) -> &Url { - &self.pkarr_relay - } -} diff --git a/iroh-dns/src/resolve.rs b/iroh-dns/src/resolve.rs deleted file mode 100644 index 4f0ee9ac650..00000000000 --- a/iroh-dns/src/resolve.rs +++ /dev/null @@ -1,117 +0,0 @@ -use std::{net::Ipv4Addr, str::FromStr}; - -use anyhow::Result; -use hickory_proto::error::ProtoError; -use hickory_resolver::{ - config::{NameServerConfigGroup, ResolverConfig, ResolverOpts}, - AsyncResolver, Name, TokioAsyncResolver, -}; -use iroh_net::{AddrInfo, NodeAddr, NodeId}; -use tracing::debug; - -use crate::{ - packet::{NodeAnnounce, IROH_NODE_TXT_LABEL}, - to_z32, -}; - -pub const IROH_TEST_DNS_IPV4: Ipv4Addr = Ipv4Addr::new(5, 75, 181, 3); -pub const IROH_TEST_DOMAIN: &str = "testdns.iroh.link."; -pub const EXAMPLE_DOMAIN: &str = "irohdns.example."; - -/// Resolver config -pub struct Config { - name_servers: NameServerConfigGroup, - default_node_origin: String, -} - -impl Config { - // TODO: Add with_system_and_iroh_test() - - pub fn with_cloudflare_and_iroh_test() -> Self { - let cloudflare_dns = NameServerConfigGroup::cloudflare(); - let cloudflare_https = NameServerConfigGroup::cloudflare_https(); - let iroh_test_https = NameServerConfigGroup::from_ips_https( - &[IROH_TEST_DNS_IPV4.into()], - 443, - IROH_TEST_DOMAIN.to_string(), - true, - ); - let iroh_test_dns = - NameServerConfigGroup::from_ips_clear(&[IROH_TEST_DNS_IPV4.into()], 53, false); - - let mut name_servers = NameServerConfigGroup::new(); - name_servers.merge(cloudflare_https); - name_servers.merge(cloudflare_dns); - name_servers.merge(iroh_test_https); - name_servers.merge(iroh_test_dns); - Self { - name_servers, - default_node_origin: IROH_TEST_DOMAIN.to_string(), - } - } - - pub fn localhost_dev() -> Self { - let name_servers = - NameServerConfigGroup::from_ips_clear(&[Ipv4Addr::LOCALHOST.into()], 5353, true); - Self { - name_servers, - default_node_origin: EXAMPLE_DOMAIN.to_string(), - } - } -} - -/// Resolve iroh nodes through DNS -#[derive(derive_more::Debug, Clone)] -pub struct Resolver { - default_node_origin: Name, - #[debug("TokioAsyncResolver")] - dns_resolver: TokioAsyncResolver, -} - -impl Resolver { - pub fn new(config: Config) -> Result { - let default_node_origin = Name::from_str(&config.default_node_origin)?; - // TODO: If we add our default node origin as search domain, we can resolve just node IDs! - // let domain = Some(config.default_node_origin); - let domain = None; - let resolv_conf = ResolverConfig::from_parts(domain, vec![], config.name_servers); - let dns_resolver = AsyncResolver::tokio(resolv_conf, ResolverOpts::default()); - Ok(Self { - dns_resolver, - default_node_origin, - }) - } - - pub fn resolver(&self) -> &TokioAsyncResolver { - &self.dns_resolver - } - - pub async fn resolve_node_by_domain(&self, domain: &str) -> Result { - let name = Name::from_str(domain)?; - self.resolve_node(name).await - } - - pub async fn resolve_node_by_id(&self, node_id: NodeId) -> Result { - debug!(?node_id, "resolve node by id"); - let name = Name::parse(&to_z32(&node_id), Some(&self.default_node_origin))?; - let addr = self.resolve_node(name).await; - debug!(?node_id, ?addr, "resolved"); - let addr = addr?; - Ok(addr.info) - } - - async fn resolve_node(&self, name: Name) -> Result { - let name = with_iroh_node_txt_label(name)?; - let lookup = self.dns_resolver.txt_lookup(name).await?; - let an = NodeAnnounce::from_hickory_lookup(lookup.as_lookup())?; - Ok(an.into()) - } -} - -fn with_iroh_node_txt_label(name: Name) -> Result { - if name.iter().next() == Some(IROH_NODE_TXT_LABEL.as_bytes()) { - Ok(name) - } else { - Name::parse(IROH_NODE_TXT_LABEL, Some(&name)) - } -} diff --git a/iroh-net/Cargo.toml b/iroh-net/Cargo.toml index d4edebf5946..d8395d226cb 100644 --- a/iroh-net/Cargo.toml +++ b/iroh-net/Cargo.toml @@ -29,6 +29,8 @@ flume = "0.11" futures = "0.3.25" governor = "0.6.0" hex = "0.4.3" +hickory-proto = "0.24.0" +hickory-resolver = "0.24.0" hostname = "0.3.1" http = "1" http-body-util = "0.1.0" @@ -40,6 +42,7 @@ libc = "0.2.139" num_enum = "0.7" once_cell = "1.18.0" parking_lot = "0.12.1" +pkarr = { version = "1.1.3", default-features = false, features = ["async", "relay"] } postcard = { version = "1", default-features = false, features = ["alloc", "use-std", "experimental-derive"] } quinn = "0.10" quinn-proto = "0.10.5" @@ -65,12 +68,12 @@ tokio-rustls = { version = "0.24" } tokio-rustls-acme = { version = "0.2" } tokio-util = { version = "0.7", features = ["io-util", "io", "codec"] } tracing = "0.1" -hickory-resolver = "0.24.0" url = { version = "2.4", features = ["serde"] } watchable = "1.1.2" webpki = { package = "rustls-webpki", version = "0.101.4", features = ["std"] } webpki-roots = "0.25" x509-parser = "0.15" +z32 = "1.0.3" # derper clap = { version = "4", features = ["derive"], optional = true } diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 3a5b87152c0..ed58249c1db 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -10,6 +10,9 @@ use tracing::{debug, error_span, warn, Instrument}; use crate::{AddrInfo, MagicEndpoint, NodeId}; +pub mod dns; +pub mod pkarr_relay_publish; + /// Node discovery for [`super::MagicEndpoint`]. /// /// The purpose of this trait is to hook up a node discovery mechanism that diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs new file mode 100644 index 00000000000..89d9cd44e38 --- /dev/null +++ b/iroh-net/src/discovery/dns.rs @@ -0,0 +1,58 @@ +//! DNS node discovery for iroh-net + +use crate::{ + discovery::{Discovery, DiscoveryItem}, + MagicEndpoint, NodeId, +}; +use anyhow::Result; +use futures::{future::FutureExt, stream::BoxStream, StreamExt}; + +use crate::dns; + +/// DNS node discovery. +/// +/// The DNS discovery looks up node addressing information over the Domain Name System. +/// Node information is resolved via an _iroh_node.z32encodednodeid TXT record. +/// +/// The content of this record is expected to be a DNS attribute string, with a required +/// `node=` attribute containing the base32 encoded node id and a derp_url attribute containing the +/// node's home Derp server. +/// +/// The discovery has to be configured with a `node_origin`, which is the domain name under which +/// lookups for nodes will be made. +/// With a origin of mydns.example, a node info record would be searched at +/// _iroh_node.z32encodednodeid.mydns.example TXT +#[derive(Debug)] +pub struct DnsDiscovery { + node_origin: String, +} + +impl DnsDiscovery { + /// Create a new DNS discovery with `node_origin` appended to all lookups. + pub fn new(node_origin: String) -> Self { + Self { node_origin } + } + + /// Create a new DNS discovery which uses the n0 testdns origin. + pub fn with_n0_testdns() -> Self { + Self::new("testdns.iroh.link".to_string()) + } +} + +impl Discovery for DnsDiscovery { + fn resolve( + &self, + _ep: MagicEndpoint, + node_id: NodeId, + ) -> Option>> { + let fut = async move { + let node_addr = dns::node_info::lookup_by_id(&node_id, &self.node_origin).await?; + Ok(DiscoveryItem { + provenance: "iroh-dns", + last_updated: None, + addr_info: node_addr.info, + }) + }; + Some(fut.into_stream().boxed()) + } +} diff --git a/iroh-net/src/discovery/pkarr_relay_publish.rs b/iroh-net/src/discovery/pkarr_relay_publish.rs new file mode 100644 index 00000000000..44d154e7759 --- /dev/null +++ b/iroh-net/src/discovery/pkarr_relay_publish.rs @@ -0,0 +1,113 @@ +//! A discovery service which publishes node information to a [Pkarr] relay. +//! +//! This service only implements the [`Discovery::publish`] method and does not provide discovery. +//! It encodes the node information into a DNS packet in the format resolvable by the +//! [`super::dns::DnsDiscovery`], which means a single _iroh_node TXT record, under the z32 encoded +//! node id as origin domain. +//! +//! [pkarr]: https://pkarr.org + +// TODO: Decide what to do with this module once publishing over Derpers land. Either remove, or +// leave in the repo but do not enable it by default in the iroh node. + +use std::sync::Arc; + +use anyhow::Result; +use parking_lot::RwLock; +use pkarr::PkarrClient; +use tracing::warn; +use url::Url; + +use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, AddrInfo}; + +/// URL of the n0 testdns server +pub const IROH_TEST_PKARR_RELAY: &str = "https://testdns.iroh.link/pkarr"; + +/// Default TTL for the _iroh_node TXT record in the pkarr signed packet +const DEFAULT_PKARR_TTL: u32 = 30; + +/// Publish node info to a pkarr relay. +#[derive(derive_more::Debug, Clone)] +pub struct Publisher { + config: Config, + #[debug("PkarrClient")] + pkarr_client: PkarrClient, + last_published: Arc>>, +} + +/// Publisher config +#[derive(derive_more::Debug, Clone)] +pub struct Config { + #[debug("SecretKey")] + secret_key: SecretKey, + #[debug("{}", self.pkarr_relay)] + pkarr_relay: Url, + ttl: u32, +} + +impl Config { + /// Create a new config with a secret key and a pkarr relay URL. + pub fn new(secret_key: SecretKey, pkarr_relay: Url) -> Self { + Self { + secret_key, + pkarr_relay, + ttl: DEFAULT_PKARR_TTL, + } + } + + /// Create a config that publishes to the n0 testdns server. + pub fn n0_testdns(secret_key: SecretKey) -> Self { + let pkarr_relay: Url = IROH_TEST_PKARR_RELAY.parse().expect("url is valid"); + Self::new(secret_key, pkarr_relay) + } + + /// Set the TTL for pkarr packets, in seconds. + /// + /// Default value is 30 seconds. + pub fn ttl(mut self, ttl: u32) -> Self { + self.ttl = ttl; + self + } +} + +impl Publisher { + /// Create a new publisher with a [`Config`]. + pub fn new(config: Config) -> Self { + let pkarr_client = PkarrClient::builder().build(); + Self { + config, + pkarr_client, + last_published: Default::default(), + } + } + + /// Publish [`AddrInfo`] about this node to a pkarr relay. + pub async fn publish_addr_info(&self, info: &AddrInfo) -> Result<()> { + let info = NodeInfo::new( + self.config.secret_key.public(), + info.derp_url.clone().map(Url::from), + ); + if self.last_published.read().as_ref() == Some(&info) { + return Ok(()); + } + let _ = self.last_published.write().insert(info.clone()); + let signed_packet = + info.to_pkarr_signed_packet(&self.config.secret_key, self.config.ttl)?; + self.pkarr_client + .relay_put(&self.config.pkarr_relay, &signed_packet) + .await?; + Ok(()) + } +} + +impl Discovery for Publisher { + fn publish(&self, info: &AddrInfo) { + let this = self.clone(); + let info = info.clone(); + tokio::task::spawn(async move { + if let Err(err) = this.publish_addr_info(&info).await { + warn!("failed to publish address update: {err:?}"); + } + }); + } +} diff --git a/iroh-net/src/dns.rs b/iroh-net/src/dns.rs index 441fb364dec..c15c3be0ed9 100644 --- a/iroh-net/src/dns.rs +++ b/iroh-net/src/dns.rs @@ -1,3 +1,5 @@ +//! DNS resolver and discovery for iroh-net + use std::net::IpAddr; use std::time::Duration; @@ -5,9 +7,16 @@ use anyhow::Result; use hickory_resolver::{AsyncResolver, IntoName, TokioAsyncResolver, TryParseIp}; use once_cell::sync::Lazy; -pub static DNS_RESOLVER: Lazy = +pub mod node_info; + +pub(crate) static DNS_RESOLVER: Lazy = Lazy::new(|| get_resolver().expect("unable to create DNS resolver")); +/// Get the DNS resolver used within iroh-net. +pub fn resolver() -> &'static TokioAsyncResolver { + Lazy::force(&DNS_RESOLVER) +} + /// Get resolver to query MX records. /// /// We first try to read the system's resolver from `/etc/resolv.conf`. diff --git a/iroh-net/src/dns/node_info.rs b/iroh-net/src/dns/node_info.rs new file mode 100644 index 00000000000..89270812a03 --- /dev/null +++ b/iroh-net/src/dns/node_info.rs @@ -0,0 +1,246 @@ +//! This module contains functions and structs to lookup node information from DNS +//! and to encode node information in Pkarr signed packets. + +use std::{collections::HashMap, fmt, str::FromStr}; + +use anyhow::{anyhow, bail, Result}; +use hickory_proto::error::ProtoError; +use hickory_resolver::Name; +use url::Url; + +use crate::{key::SecretKey, AddrInfo, NodeAddr, NodeId}; + +const ATTR_DERP: &str = "derp"; +const ATTR_NODE_ID: &str = "node"; + +/// The label for the node info TXT record +pub const IROH_NODE_TXT_LABEL: &str = "_iroh_node"; + +/// Lookup node info by domain name +/// +/// The domain name must either contain an _iroh_node TXT record or be a CNAME record that leads to +/// an _iroh_node TXT record. +pub async fn lookup_by_domain(domain: &str) -> Result { + let name = Name::from_str(domain)?; + let info = lookup_node_info(name).await?; + Ok(info.into()) +} + +/// Lookup node info by node id and origin domain name. +pub async fn lookup_by_id(node_id: &NodeId, origin: &str) -> Result { + let domain = format!("{}.{}", to_z32(node_id), origin); + lookup_by_domain(&domain).await +} + +async fn lookup_node_info(name: Name) -> Result { + let name = ensure_iroh_node_txt_label(name)?; + let lookup = super::resolver().txt_lookup(name).await?; + NodeInfo::from_hickory_lookup(lookup.as_lookup()) +} + +fn ensure_iroh_node_txt_label(name: Name) -> Result { + if name.iter().next() == Some(IROH_NODE_TXT_LABEL.as_bytes()) { + Ok(name) + } else { + Name::parse(IROH_NODE_TXT_LABEL, Some(&name)) + } +} + +/// Encode a [`NodeId`] in [`z-base-32`] encoding. +/// +/// [z-base-32]: https://philzimmermann.com/docs/human-oriented-base-32-encoding.txt +pub fn to_z32(node_id: &NodeId) -> String { + z32::encode(node_id.as_bytes()) +} + +/// Parse a [`NodeId`] from [`z-base-32`] encoding. +/// +/// [z-base-32]: https://philzimmermann.com/docs/human-oriented-base-32-encoding.txt +pub fn from_z32(s: &str) -> Result { + let bytes = z32::decode(s.as_bytes()).map_err(|_| anyhow!("invalid z32"))?; + let bytes: &[u8; 32] = &bytes.try_into().map_err(|_| anyhow!("not 32 bytes long"))?; + let node_id = NodeId::from_bytes(bytes)?; + Ok(node_id) +} + +/// Node info contained in a DNS _iroh_node TXT record. +#[derive(derive_more::Debug, Clone, Eq, PartialEq)] +pub struct NodeInfo { + /// The node id + pub node_id: NodeId, + /// Home Derp server for this node + #[debug("{:?}", self.derp_url.as_ref().map(|s| s.to_string()))] + pub derp_url: Option, +} + +impl From for NodeAddr { + fn from(value: NodeInfo) -> Self { + NodeAddr { + node_id: value.node_id, + info: value.into(), + } + } +} + +impl From for AddrInfo { + fn from(value: NodeInfo) -> Self { + AddrInfo { + derp_url: value.derp_url.map(|u| u.into()), + direct_addresses: Default::default(), + } + } +} + +impl NodeInfo { + /// Create a new [`NodeInfo`] from its parts. + pub fn new(node_id: NodeId, derp_url: Option) -> Self { + Self { node_id, derp_url } + } + + /// Convert this node info into a DNS attribute string. + /// + /// It will look like this: + /// `node=b32encodednodeid derp=https://myderp.example` + pub fn to_attribute_string(&self) -> String { + let mut attrs = vec![]; + attrs.push(fmt_attr(ATTR_NODE_ID, self.node_id)); + if let Some(derp) = &self.derp_url { + attrs.push(fmt_attr(ATTR_DERP, derp)); + } + attrs.join(" ") + } + + /// Try to parse a [`NodeInfo`] from the lookup result of our DNS resolver. + pub fn from_hickory_lookup(lookup: &hickory_resolver::lookup::Lookup) -> Result { + Self::from_hickory_records(lookup.records()) + } + + /// Try to parse a [`NodeInfo`] from a set of DNS records. + pub fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { + use hickory_proto::rr; + let (node_id, txt) = records + .iter() + .find_map(|rr| match rr.data() { + Some(rr::RData::TXT(txt)) => { + parse_hickory_node_info_name(rr.name()).map(|node_id| (node_id, txt)) + } + _ => None, + }) + .ok_or_else(|| anyhow!("no TXT record with name _iroh_node.b32encodedpubkey found"))?; + let node_info = Self::parse_from_attributes(&txt.to_string())?; + if node_info.node_id != node_id { + bail!("node id mismatch between record name and TXT value"); + } + Ok(node_info) + } + + /// Parse the [`NodeInfo`] from an attribute string. + /// + /// See [Self::to_attribute_string] for the expected format. + pub fn parse_from_attributes(attrs: &str) -> Result { + let attrs = parse_attrs(attrs); + let Some(node) = attrs.get(ATTR_NODE_ID) else { + bail!("missing required node attribute"); + }; + if node.len() != 1 { + bail!("more than one node attribute is not allowed"); + } + let node_id = NodeId::from_str(node[0])?; + let home_derp: Option = attrs + .get(ATTR_DERP) + .into_iter() + .flatten() + .find_map(|x| Url::parse(x).ok()); + Ok(Self { + node_id, + derp_url: home_derp, + }) + } + + /// Create a [`pkarr::SignedPacket`] by constructing a DNS packet and + /// signing it with a [`SecretKey`]. + pub fn to_pkarr_signed_packet( + &self, + secret_key: &SecretKey, + ttl: u32, + ) -> Result { + let packet = self.to_pkarr_dns_packet(ttl)?; + let keypair = pkarr::Keypair::from_secret_key(&secret_key.to_bytes()); + let signed_packet = pkarr::SignedPacket::from_packet(&keypair, &packet)?; + Ok(signed_packet) + } + + fn to_pkarr_dns_packet(&self, ttl: u32) -> Result> { + use pkarr::dns::{self, rdata}; + let name = dns::Name::new(IROH_NODE_TXT_LABEL)?.into_owned(); + let rdata = { + let value = self.to_attribute_string(); + let txt = rdata::TXT::new().with_string(&value)?.into_owned(); + rdata::RData::TXT(txt) + }; + + let mut packet = dns::Packet::new_reply(0); + packet + .answers + .push(dns::ResourceRecord::new(name, dns::CLASS::IN, ttl, rdata)); + Ok(packet) + } + + /// Try to parse a [`NodeInfo`] from a [`pkarr::SignedPacket`]. + pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { + use pkarr::dns::{self, rdata::RData}; + let pubkey = packet.public_key(); + let pubkey_z32 = pubkey.to_z32(); + let node_id = NodeId::from(*pubkey.verifying_key()); + let zone = dns::Name::new(&pubkey_z32)?; + let inner = packet.packet(); + let txt_record = inner + .answers + .iter() + .find_map(|rr| match &rr.rdata { + RData::TXT(txt) => match rr.name.without(&zone) { + Some(name) if name.to_string() == IROH_NODE_TXT_LABEL => Some(txt), + Some(_) | None => None, + }, + _ => None, + }) + .ok_or_else(|| anyhow!("missing _iroh_node txt record"))?; + + let txt_record = txt_record.to_owned(); + let txt = String::try_from(txt_record)?; + let info = Self::parse_from_attributes(&txt)?; + if info.node_id != node_id { + bail!("node id mismatch between record name and TXT value"); + } + Ok(info) + } +} + +fn parse_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { + if name.num_labels() < 2 { + return None; + } + let mut labels = name.iter(); + let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; + if label != IROH_NODE_TXT_LABEL { + return None; + } + let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; + let node_id = from_z32(label).ok()?; + Some(node_id) +} + +fn fmt_attr(label: &str, value: impl fmt::Display) -> String { + format!("{label}={value}") +} + +fn parse_attrs<'a>(s: &'a str) -> HashMap<&'a str, Vec<&'a str>> { + let mut map: HashMap<&'a str, Vec<&'a str>> = HashMap::new(); + let parts = s.split(' '); + for part in parts { + if let Some((name, value)) = part.split_once('=') { + map.entry(name).or_default().push(value); + } + } + map +} diff --git a/iroh-net/src/lib.rs b/iroh-net/src/lib.rs index 95362a2ca18..5db44541bb2 100644 --- a/iroh-net/src/lib.rs +++ b/iroh-net/src/lib.rs @@ -16,7 +16,7 @@ pub mod derp; pub mod dialer; mod disco; pub mod discovery; -mod dns; +pub mod dns; pub mod magic_endpoint; pub mod magicsock; pub mod metrics; diff --git a/iroh/Cargo.toml b/iroh/Cargo.toml index b359351fb3a..e7f92eddda2 100644 --- a/iroh/Cargo.toml +++ b/iroh/Cargo.toml @@ -29,7 +29,6 @@ hashlink = "0.8.4" hex = { version = "0.4.3" } iroh-bytes = { version = "0.12.0", path = "../iroh-bytes", features = ["downloader"] } iroh-base = { version = "0.12.0", path = "../iroh-base", features = ["key"] } -iroh-dns = { version = "0.12.0", path = "../iroh-dns" } iroh-io = { version = "0.4.0", features = ["stats"] } iroh-metrics = { version = "0.12.0", path = "../iroh-metrics", optional = true } iroh-net = { version = "0.12.0", path = "../iroh-net" } diff --git a/iroh/src/commands/start.rs b/iroh/src/commands/start.rs index fd26a554698..90b8c0d8156 100644 --- a/iroh/src/commands/start.rs +++ b/iroh/src/commands/start.rs @@ -14,9 +14,9 @@ use iroh::{ rpc_protocol::{ProviderRequest, ProviderResponse, ProviderService}, util::{fs::load_secret_key, path::IrohPaths}, }; -use iroh_dns::discovery::DnsDiscovery; use iroh_net::{ derp::{DerpMap, DerpMode}, + discovery::{dns::DnsDiscovery, pkarr_relay_publish, ConcurrentDiscovery}, key::SecretKey, }; use quic_rpc::{transport::quinn::QuinnServerEndpoint, ServiceEndpoint}; @@ -220,7 +220,14 @@ pub(crate) async fn start_node( Some(derp_map) => DerpMode::Custom(derp_map), }; - let discovery = DnsDiscovery::with_iroh_test(Some(secret_key.clone()))?; + let mut discovery = ConcurrentDiscovery::new(); + let dns_discovery = DnsDiscovery::with_n0_testdns(); + discovery.add(dns_discovery); + // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. + let pkarr_publish = pkarr_relay_publish::Publisher::new( + pkarr_relay_publish::Config::n0_testdns(secret_key.clone()), + ); + discovery.add(pkarr_publish); Node::builder(bao_store, doc_store) .derp_mode(derp_mode) diff --git a/iroh/src/node.rs b/iroh/src/node.rs index bba32a82410..755fc8de3f0 100644 --- a/iroh/src/node.rs +++ b/iroh/src/node.rs @@ -34,8 +34,8 @@ use iroh_bytes::{protocol::Closed, provider::AddProgress, BlobFormat, Hash, Hash use iroh_gossip::net::{Gossip, GOSSIP_ALPN}; use iroh_io::AsyncSliceReader; use iroh_net::derp::DerpUrl; -use iroh_net::magic_endpoint::get_alpn; use iroh_net::discovery::Discovery; +use iroh_net::magic_endpoint::get_alpn; use iroh_net::magicsock::LocalEndpointsStream; use iroh_net::util::AbortingJoinHandle; use iroh_net::{ From 9f21f409464ec3f9a6807d1a126919298be7a1ad Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 18 Mar 2024 15:35:18 +0100 Subject: [PATCH 30/66] fix: make NodeDiscoveryConfig pub --- iroh/src/node.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh/src/node.rs b/iroh/src/node.rs index b84ad28fa30..445caec0d18 100644 --- a/iroh/src/node.rs +++ b/iroh/src/node.rs @@ -42,7 +42,7 @@ mod builder; mod rpc; mod rpc_status; -pub use builder::{Builder, GcPolicy, StorageConfig}; +pub use builder::{Builder, GcPolicy, StorageConfig, NodeDiscoveryConfig}; pub use rpc_status::RpcStatus; type EventCallback = Box BoxFuture<'static, ()> + 'static + Sync + Send>; From 44c5cc6bfc9bdb9c9ec769af328e567464bb3335 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 18 Mar 2024 15:36:05 +0100 Subject: [PATCH 31/66] chore: fmt --- iroh/src/node.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh/src/node.rs b/iroh/src/node.rs index 445caec0d18..b6dd5ea6e6c 100644 --- a/iroh/src/node.rs +++ b/iroh/src/node.rs @@ -42,7 +42,7 @@ mod builder; mod rpc; mod rpc_status; -pub use builder::{Builder, GcPolicy, StorageConfig, NodeDiscoveryConfig}; +pub use builder::{Builder, GcPolicy, NodeDiscoveryConfig, StorageConfig}; pub use rpc_status::RpcStatus; type EventCallback = Box BoxFuture<'static, ()> + 'static + Sync + Send>; From be08b9b5d92debed7c07dcc9ba43c3fc19bd3081 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 18 Mar 2024 15:55:27 +0100 Subject: [PATCH 32/66] cleanup: better namings --- iroh-net/src/discovery/dns.rs | 7 +++++-- iroh-net/src/discovery/pkarr_relay_publish.rs | 6 +++--- iroh/src/node/builder.rs | 2 +- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs index 89d9cd44e38..c2608aa35c6 100644 --- a/iroh-net/src/discovery/dns.rs +++ b/iroh-net/src/discovery/dns.rs @@ -9,6 +9,9 @@ use futures::{future::FutureExt, stream::BoxStream, StreamExt}; use crate::dns; +/// The n0 testing DNS node origin +pub const N0_TESTDNS_NODE_ORIGIN: &str = "testdns.iroh.link"; + /// DNS node discovery. /// /// The DNS discovery looks up node addressing information over the Domain Name System. @@ -34,8 +37,8 @@ impl DnsDiscovery { } /// Create a new DNS discovery which uses the n0 testdns origin. - pub fn with_n0_testdns() -> Self { - Self::new("testdns.iroh.link".to_string()) + pub fn n0_testdns() -> Self { + Self::new(N0_TESTDNS_NODE_ORIGIN.to_string()) } } diff --git a/iroh-net/src/discovery/pkarr_relay_publish.rs b/iroh-net/src/discovery/pkarr_relay_publish.rs index 44d154e7759..66a4cfbbf22 100644 --- a/iroh-net/src/discovery/pkarr_relay_publish.rs +++ b/iroh-net/src/discovery/pkarr_relay_publish.rs @@ -20,8 +20,8 @@ use url::Url; use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, AddrInfo}; -/// URL of the n0 testdns server -pub const IROH_TEST_PKARR_RELAY: &str = "https://testdns.iroh.link/pkarr"; +/// The n0 testing pkarr relay +pub const N0_TESTDNS_PKARR_RELAY: &str = "https://testdns.iroh.link/pkarr"; /// Default TTL for the _iroh_node TXT record in the pkarr signed packet const DEFAULT_PKARR_TTL: u32 = 30; @@ -57,7 +57,7 @@ impl Config { /// Create a config that publishes to the n0 testdns server. pub fn n0_testdns(secret_key: SecretKey) -> Self { - let pkarr_relay: Url = IROH_TEST_PKARR_RELAY.parse().expect("url is valid"); + let pkarr_relay: Url = N0_TESTDNS_PKARR_RELAY.parse().expect("url is valid"); Self::new(secret_key, pkarr_relay) } diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs index 76f810fdad2..181d2dcf812 100644 --- a/iroh/src/node/builder.rs +++ b/iroh/src/node/builder.rs @@ -312,7 +312,7 @@ where NodeDiscoveryConfig::Custom(discovery) => Some(discovery), NodeDiscoveryConfig::Default => { let mut discovery = ConcurrentDiscovery::new(); - let dns_discovery = DnsDiscovery::with_n0_testdns(); + let dns_discovery = DnsDiscovery::n0_testdns(); discovery.add(dns_discovery); // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. let pkarr_publish = pkarr_relay_publish::Publisher::new( From e0012b5318b763713de0f5a625f924093f72047a Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 18 Mar 2024 20:08:05 +0100 Subject: [PATCH 33/66] fix Cargo.lock after rebase --- Cargo.lock | 442 ++++++++++++++++++++++++++++------------------------- 1 file changed, 238 insertions(+), 204 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0f15f1f4abd..62c7863aa1c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -30,9 +30,9 @@ dependencies = [ [[package]] name = "ahash" -version = "0.8.8" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42cd52102d3df161c77a887b608d7a4897d7cc112886a9537b738a887a03aaff" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" dependencies = [ "cfg-if", "once_cell", @@ -84,9 +84,9 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstream" -version = "0.6.12" +version = "0.6.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96b09b5178381e0874812a9b157f7fe84982617e48f71f4e3235482775e5b540" +checksum = "d96bd03f33fe50a863e394ee9718a706f988b9079b20c3784fb726e7678b62fb" dependencies = [ "anstyle", "anstyle-parse", @@ -206,7 +206,7 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -217,7 +217,7 @@ checksum = "c980ee35e870bd1a4d2c8294d4c04d0499e67bca1e4b5cefcc693c2fa00caea9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -235,7 +235,7 @@ version = "0.16.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdb8867f378f33f78a811a8eb9bf108ad99430d7aad43315dd9319c827ef6247" dependencies = [ - "http 0.2.11", + "http 0.2.12", "log", "url", "wildmatch", @@ -258,7 +258,7 @@ dependencies = [ "bitflags 1.3.2", "bytes", "futures-util", - "http 0.2.11", + "http 0.2.12", "http-body 0.4.6", "hyper 0.14.28", "itoa", @@ -284,7 +284,7 @@ dependencies = [ "async-trait", "bytes", "futures-util", - "http 0.2.11", + "http 0.2.12", "http-body 0.4.6", "mime", "rustversion", @@ -424,9 +424,9 @@ checksum = "78a6932c88f1d2c29533a3b8a5f5a2f84cc19c3339b431677c3160c5c2e6ca85" [[package]] name = "bumpalo" -version = "3.15.1" +version = "3.15.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c764d619ca78fccbf3069b37bd7af92577f044bb15236036662d79b6559f25b7" +checksum = "8ea184aa71bb362a1157c896979544cc23974e08fd265f29ea96b59f0b4a555b" [[package]] name = "byteorder" @@ -482,12 +482,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.83" +version = "1.0.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" -dependencies = [ - "libc", -] +checksum = "a0ba8f7aaa012f30d5b2861462f6708eccd49c3c39863fe083a308035f63d723" [[package]] name = "cfg-if" @@ -508,9 +505,9 @@ dependencies = [ [[package]] name = "chrono" -version = "0.4.34" +version = "0.4.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bc015644b92d5890fab7489e49d21f879d5c990186827d42ec511919404f38b" +checksum = "8eaf5903dcbc0a39312feb77df2ff4c76387d591b9fc7b04a238dcf8bb62639a" dependencies = [ "android-tzdata", "iana-time-zone", @@ -518,7 +515,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-targets 0.52.0", + "windows-targets 0.52.4", ] [[package]] @@ -590,7 +587,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -845,9 +842,9 @@ checksum = "7059fff8937831a9ae6f0fe4d658ffabf58f2ca96aa9dec1c889f936f705f216" [[package]] name = "crossbeam-channel" -version = "0.5.11" +version = "0.5.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "176dc175b78f56c0f321911d9c8eb2b77a78a4860b9c19db83835fea1a46649b" +checksum = "ab3db02a9c5b5121e1e42fbdb1aeb65f5e02624cc58c43f2884c6ccac0b82f95" dependencies = [ "crossbeam-utils", ] @@ -985,14 +982,14 @@ checksum = "f46882e17999c6cc590af592290432be3bce0428cb0d5f8b6715e4dc7b383eb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] name = "darling" -version = "0.20.6" +version = "0.20.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c376d08ea6aa96aafe61237c7200d1241cb177b7d3a542d791f2d118e9cbb955" +checksum = "54e36fcd13ed84ffdfda6f5be89b31287cbb80c439841fe69e04841435464391" dependencies = [ "darling_core", "darling_macro", @@ -1000,27 +997,27 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.20.6" +version = "0.20.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33043dcd19068b8192064c704b3f83eb464f91f1ff527b44a4e2b08d9cdb8855" +checksum = "9c2cf1c23a687a1feeb728783b993c4e1ad83d99f351801977dd809b48d0a70f" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim 0.10.0", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] name = "darling_macro" -version = "0.20.6" +version = "0.20.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5a91391accf613803c2a9bf9abccdbaa07c54b4244a5b64883f9c3c137c86be" +checksum = "a668eda54683121533a393014d8692171709ff57a7d61f187b6e782719f8933f" dependencies = [ "darling_core", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -1112,7 +1109,7 @@ checksum = "5fe87ce4529967e0ba1dcf8450bab64d97dfd5010a6256187ffe2e43e6f0e049" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -1142,7 +1139,7 @@ checksum = "2bba3e9872d7c58ce7ef0fcf1844fcc3e23ef2a58377b50df35dd98e42a5726e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", "unicode-xid", ] @@ -1225,7 +1222,7 @@ checksum = "487585f4d0c6655fe74905e2504d8ad6908e4db67f744eb140876906c2f3175d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -1370,7 +1367,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -1383,7 +1380,7 @@ dependencies = [ "num-traits", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -1570,7 +1567,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -1587,9 +1584,9 @@ checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" [[package]] name = "futures-timer" -version = "3.0.2" +version = "3.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" +checksum = "f288b0a4f20f9a56b5d1da57e2227c661b7b16168e2f72365f57b63326e29b24" [[package]] name = "futures-util" @@ -1718,8 +1715,8 @@ dependencies = [ "futures-core", "futures-sink", "futures-util", - "http 0.2.11", - "indexmap 2.2.3", + "http 0.2.12", + "indexmap 2.2.5", "slab", "tokio", "tokio-util", @@ -1728,9 +1725,9 @@ dependencies = [ [[package]] name = "half" -version = "2.3.1" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc52e53916c08643f1b56ec082790d1e86a32e58dc5268f897f313fbae7b4872" +checksum = "b5eceaaeec696539ddaf7b333340f1af35a5aa87ae3e4f3ead0532f72affab2e" dependencies = [ "cfg-if", "crunchy", @@ -1805,9 +1802,9 @@ checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" [[package]] name = "hermit-abi" -version = "0.3.6" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd5256b483761cd23699d0da46cc6fd2ee3be420bbe6d020ae4a091e70b7e9fd" +checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" [[package]] name = "hex" @@ -1912,9 +1909,9 @@ checksum = "f558a64ac9af88b5ba400d99b579451af0d39c6d360980045b91aac966d705e2" [[package]] name = "http" -version = "0.2.11" +version = "0.2.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8947b1a6fad4393052c7ba1f4cd97bed3e953a95c79c92ad9b051a04611d9fbb" +checksum = "601cbb57e577e2f5ef5be8e7b83f0f63994f25aa94d673e54a92d5c516d101f1" dependencies = [ "bytes", "fnv", @@ -1923,9 +1920,9 @@ dependencies = [ [[package]] name = "http" -version = "1.0.0" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b32afd38673a8016f7c9ae69e5af41a58f81b1d31689040f2f1959594ce194ea" +checksum = "21b9ddb458710bc376481b842f5da65cdf31522de232c1ca8146abce2a358258" dependencies = [ "bytes", "fnv", @@ -1939,7 +1936,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7ceab25649e9960c0311ea418d17bee82c0dcec1bd053b5f9a66e265a693bed2" dependencies = [ "bytes", - "http 0.2.11", + "http 0.2.12", "pin-project-lite", ] @@ -1950,7 +1947,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1cac85db508abc24a2e48553ba12a996e87244a0395ce011e62b37158745d643" dependencies = [ "bytes", - "http 1.0.0", + "http 1.1.0", ] [[package]] @@ -1961,7 +1958,7 @@ checksum = "41cb79eb393015dadd30fc252023adb0b2400a0caee0fa2a077e6e21a551e840" dependencies = [ "bytes", "futures-util", - "http 1.0.0", + "http 1.1.0", "http-body 1.0.0", "pin-project-lite", ] @@ -2014,7 +2011,7 @@ dependencies = [ "futures-core", "futures-util", "h2", - "http 0.2.11", + "http 0.2.12", "http-body 0.4.6", "httparse", "httpdate", @@ -2029,19 +2026,20 @@ dependencies = [ [[package]] name = "hyper" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb5aa53871fc917b1a9ed87b683a5d86db645e23acb32c2e0785a353e522fb75" +checksum = "186548d73ac615b32a73aafe38fb4f56c0d340e110e5a200bcadbaf2e199263a" dependencies = [ "bytes", "futures-channel", "futures-util", - "http 1.0.0", + "http 1.1.0", "http-body 1.0.0", "httparse", "httpdate", "itoa", "pin-project-lite", + "smallvec", "tokio", "want", ] @@ -2053,7 +2051,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", - "http 0.2.11", + "http 0.2.12", "hyper 0.14.28", "rustls", "tokio", @@ -2080,9 +2078,9 @@ checksum = "ca38ef113da30126bbff9cd1705f9273e15d45498615d138b0c20279ac7a76aa" dependencies = [ "bytes", "futures-util", - "http 1.0.0", + "http 1.1.0", "http-body 1.0.0", - "hyper 1.1.0", + "hyper 1.2.0", "pin-project-lite", "socket2", "tokio", @@ -2146,7 +2144,7 @@ dependencies = [ "attohttpc", "bytes", "futures", - "http 0.2.11", + "http 0.2.12", "hyper 0.14.28", "log", "rand", @@ -2168,9 +2166,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.2.3" +version = "2.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "233cf39063f058ea2caae4091bf4a3ef70a653afbc026f5c4a4135d114e3c177" +checksum = "7b0b929d511467233429c45a44ac1dcaa21ba0f5ba11e4879e6ed28ddb4f9df4" dependencies = [ "equivalent", "hashbrown 0.14.3", @@ -2438,7 +2436,7 @@ dependencies = [ "ed25519-dalek", "futures", "genawaiter", - "indexmap 2.2.3", + "indexmap 2.2.5", "iroh-base", "iroh-blake3", "iroh-metrics", @@ -2478,7 +2476,7 @@ dependencies = [ "anyhow", "erased_set", "http-body-util", - "hyper 1.1.0", + "hyper 1.2.0", "hyper-util", "once_cell", "prometheus-client", @@ -2514,9 +2512,9 @@ dependencies = [ "hickory-proto", "hickory-resolver", "hostname", - "http 1.0.0", + "http 1.1.0", "http-body-util", - "hyper 1.1.0", + "hyper 1.2.0", "hyper-util", "igd", "iroh-base", @@ -2691,9 +2689,9 @@ checksum = "b1a46d1a171d865aa5f83f92695765caa047a9b4cbae2cbf37dbd613a793fd4c" [[package]] name = "js-sys" -version = "0.3.68" +version = "0.3.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "406cda4b368d531c842222cf9d2600a9a4acce8d29423695379c6868a143a9ee" +checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" dependencies = [ "wasm-bindgen", ] @@ -2754,15 +2752,15 @@ dependencies = [ [[package]] name = "log" -version = "0.4.20" +version = "0.4.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" +checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" [[package]] name = "lru" -version = "0.12.2" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db2c024b41519440580066ba82aab04092b333e09066a5eb86c7c4890df31f22" +checksum = "d3262e75e648fce39813cb56ac41f3c3e3f65217ebf3844d818d1f9398cfb0dc" dependencies = [ "hashbrown 0.14.3", ] @@ -2838,9 +2836,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.10" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f3d0b296e374a4e6f3c7b0a1f5a51d748a0d34c85e7dc48fc3fa9a87657fe09" +checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c" dependencies = [ "libc", "wasi", @@ -3135,7 +3133,7 @@ dependencies = [ "proc-macro-crate 3.1.0", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -3185,9 +3183,9 @@ checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" [[package]] name = "opaque-debug" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" +checksum = "c08d65885ee38876c4f86fa503fb49d7b507c2b62552df7c70b2fce627e06381" [[package]] name = "openssl-probe" @@ -3233,7 +3231,7 @@ dependencies = [ "proc-macro2", "proc-macro2-diagnostics", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -3358,9 +3356,9 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "pest" -version = "2.7.7" +version = "2.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "219c0dcc30b6a27553f9cc242972b67f75b60eb0db71f0b5462f38b058c41546" +checksum = "56f8023d0fb78c8e03784ea1c7f3fa36e68a723138990b8d5a47d916b651e7a8" dependencies = [ "memchr", "thiserror", @@ -3369,9 +3367,9 @@ dependencies = [ [[package]] name = "pest_derive" -version = "2.7.7" +version = "2.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22e1288dbd7786462961e69bfd4df7848c1e37e8b74303dbdab82c3a9cdd2809" +checksum = "b0d24f72393fd16ab6ac5738bc33cdb6a9aa73f8b902e8fe29cf4e67d7dd1026" dependencies = [ "pest", "pest_generator", @@ -3379,22 +3377,22 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.7.7" +version = "2.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1381c29a877c6d34b8c176e734f35d7f7f5b3adaefe940cb4d1bb7af94678e2e" +checksum = "fdc17e2a6c7d0a492f0158d7a4bd66cc17280308bbaff78d5bef566dca35ab80" dependencies = [ "pest", "pest_meta", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] name = "pest_meta" -version = "2.7.7" +version = "2.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0934d6907f148c22a3acbda520c7eed243ad7487a30f51f6ce52b58b7077a8a" +checksum = "934cd7631c050f4674352a6e835d5f6711ffbfb9345c2fc0107155ac495ae293" dependencies = [ "once_cell", "pest", @@ -3403,22 +3401,22 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.4" +version = "1.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0302c4a0442c456bd56f841aee5c3bfd17967563f6fadc9ceb9f9c23cf3807e0" +checksum = "b6bf43b791c5b9e34c3d182969b4abb522f9343702850a2e57f460d00d09b4b3" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.4" +version = "1.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "266c042b60c9c76b8d53061e52b2e0d1116abc57cefc8c5cd671619a56ac3690" +checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -3507,39 +3505,39 @@ dependencies = [ [[package]] name = "pnet_base" -version = "0.33.0" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "872e46346144ebf35219ccaa64b1dffacd9c6f188cd7d012bd6977a2a838f42e" +checksum = "fe4cf6fb3ab38b68d01ab2aea03ed3d1132b4868fa4e06285f29f16da01c5f4c" dependencies = [ "no-std-net", ] [[package]] name = "pnet_macros" -version = "0.33.0" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a780e80005c2e463ec25a6e9f928630049a10b43945fea83207207d4a7606f4" +checksum = "688b17499eee04a0408aca0aa5cba5fc86401d7216de8a63fdf7a4c227871804" dependencies = [ "proc-macro2", "quote", "regex", - "syn 1.0.109", + "syn 2.0.52", ] [[package]] name = "pnet_macros_support" -version = "0.33.0" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6d932134f32efd7834eb8b16d42418dac87086347d1bc7d142370ef078582bc" +checksum = "eea925b72f4bd37f8eab0f221bbe4c78b63498350c983ffa9dd4bcde7e030f56" dependencies = [ "pnet_base", ] [[package]] name = "pnet_packet" -version = "0.33.0" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bde678bbd85cb1c2d99dc9fc596e57f03aa725f84f3168b0eaf33eeccb41706" +checksum = "a9a005825396b7fe7a38a8e288dbc342d5034dac80c15212436424fef8ea90ba" dependencies = [ "glob", "pnet_base", @@ -3732,9 +3730,9 @@ checksum = "af066a9c399a26e020ada66a034357a868728e72cd426f3adcd35f80d88d88c8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", "version_check", - "yansi 1.0.0-rc.1", + "yansi 1.0.0", ] [[package]] @@ -3757,7 +3755,7 @@ checksum = "440f724eba9f6996b75d63681b0a92b06947f1457076d503a4d2e2c8f56442b8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -3800,7 +3798,7 @@ dependencies = [ "itertools 0.11.0", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -3991,9 +3989,9 @@ dependencies = [ [[package]] name = "rayon" -version = "1.8.1" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa7237101a77a10773db45d62004a272517633fbcc3df19d96455ede1122e051" +checksum = "e4963ed1bc86e4f3ee217022bd855b297cef07fb9eac5dfa1f788b220b49b3bd" dependencies = [ "either", "rayon-core", @@ -4079,18 +4077,18 @@ checksum = "5fddb4f8d99b0a2ebafc65a87a69a7b9875e4b1ae1f00db265d300ef7f28bccc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] name = "reflink-copy" -version = "0.1.14" +version = "0.1.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "767be24c0da52e7448d495b8d162506a9aa125426651d547d545d6c2b4b65b62" +checksum = "52b1349400e2ffd64a9fb5ed9008e33c0b8ef86bd5bae8f73080839c7082f1d5" dependencies = [ "cfg-if", "rustix", - "windows 0.52.0", + "windows 0.54.0", ] [[package]] @@ -4101,7 +4099,7 @@ checksum = "b62dbe01f0b06f9d8dc7d49e05a0785f153b00b2c227856282f671e0318c9b15" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.5", + "regex-automata 0.4.6", "regex-syntax 0.8.2", ] @@ -4116,9 +4114,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bb987efffd3c6d0d8f5f89510bb458559eab11e4f869acb20bf845e016259cd" +checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea" dependencies = [ "aho-corasick", "memchr", @@ -4155,7 +4153,7 @@ dependencies = [ "futures-core", "futures-util", "h2", - "http 0.2.11", + "http 0.2.12", "http-body 0.4.6", "hyper 0.14.28", "hyper-rustls", @@ -4548,16 +4546,16 @@ checksum = "7eb0b34b42edc17f6b7cac84a52a1c5f0e1bb2227e997ca9011ea3dd34e8610b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] name = "serde_json" -version = "1.0.113" +version = "1.0.114" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69801b70b1c3dac963ecb03a364ba0ceda9cf60c71cfe475e99864759c8b8a79" +checksum = "c5f09b1bd632ef549eaa9f60a1f8de742bdbc698e6cee2095fc84dde5f549ae0" dependencies = [ - "indexmap 2.2.3", + "indexmap 2.2.5", "itoa", "ryu", "serde", @@ -4603,7 +4601,7 @@ dependencies = [ "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.2.3", + "indexmap 2.2.5", "serde", "serde_derive", "serde_json", @@ -4620,7 +4618,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -4732,12 +4730,12 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.5" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" +checksum = "05ffd9c0a93b7543e062e759284fcf5f5e3b098501104bfbdde4d404db792871" dependencies = [ "libc", - "windows-sys 0.48.0", + "windows-sys 0.52.0", ] [[package]] @@ -4866,7 +4864,7 @@ dependencies = [ "proc-macro2", "quote", "struct_iterable_internal", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -4884,7 +4882,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta-derive", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -4895,7 +4893,7 @@ checksum = "a60bcaff7397072dca0017d1db428e30d5002e00b6847703e2e42005c95fbe00" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -4926,7 +4924,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -4939,7 +4937,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -4972,9 +4970,9 @@ checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" [[package]] name = "surge-ping" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af341b2be485d647b5dc4cfb2da99efac35b5c95748a08fb7233480fedc5ead3" +checksum = "efbf95ce4c7c5b311d2ce3f088af2b93edef0f09727fa50fbe03c7a979afce77" dependencies = [ "hex", "parking_lot", @@ -4999,9 +4997,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.49" +version = "2.0.52" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "915aea9e586f80826ee59f8453c1101f9d1c4b3964cd2460185ee8e299ada496" +checksum = "b699d15b36d1f02c3e7c69f8ffef53de37aefae075d8488d4ba1a7788d574a07" dependencies = [ "proc-macro2", "quote", @@ -5093,7 +5091,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -5127,14 +5125,14 @@ checksum = "c61f3ba182994efc43764a46c018c347bc492c79f024e705f46567b418f6d4f7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] name = "thread_local" -version = "1.1.7" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdd6f064ccff2d6567adcb3873ca630700f00b5ad3f060c25b5dcfd9a4ce152" +checksum = "8b9ef9bad013ada3808854ceac7b46812a6465ba368859a37e2100283d2d719c" dependencies = [ "cfg-if", "once_cell", @@ -5236,7 +5234,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -5356,7 +5354,7 @@ version = "0.19.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" dependencies = [ - "indexmap 2.2.3", + "indexmap 2.2.5", "toml_datetime", "winnow 0.5.40", ] @@ -5367,7 +5365,7 @@ version = "0.21.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8534fd7f78b5405e860340ad6575217ce99f38d4d5c8f2442cb5ecb50090e1" dependencies = [ - "indexmap 2.2.3", + "indexmap 2.2.5", "toml_datetime", "winnow 0.5.40", ] @@ -5378,11 +5376,11 @@ version = "0.22.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2c1b5fd4128cc8d3e0cb74d4ed9a9cc7c7284becd4df68f5f940e1ad123606f6" dependencies = [ - "indexmap 2.2.3", + "indexmap 2.2.5", "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.1", + "winnow 0.6.5", ] [[package]] @@ -5397,7 +5395,7 @@ dependencies = [ "base64 0.21.7", "bytes", "h2", - "http 0.2.11", + "http 0.2.12", "http-body 0.4.6", "hyper 0.14.28", "hyper-timeout", @@ -5464,7 +5462,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -5573,9 +5571,9 @@ checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" [[package]] name = "unicode-normalization" -version = "0.1.22" +version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" +checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" dependencies = [ "tinyvec", ] @@ -5661,9 +5659,9 @@ dependencies = [ [[package]] name = "walkdir" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d71d857dc86794ca4c280d616f7da00d2dbfd8cd788846559a6813e6aa4b54ee" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" dependencies = [ "same-file", "winapi-util", @@ -5684,11 +5682,17 @@ version = "0.11.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" +[[package]] +name = "wasite" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" + [[package]] name = "wasm-bindgen" -version = "0.2.91" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1e124130aee3fb58c5bdd6b639a0509486b0338acaaae0c84a5124b0f588b7f" +checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -5696,24 +5700,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.91" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9e7e1900c352b609c8488ad12639a311045f40a35491fb69ba8c12f758af70b" +checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.41" +version = "0.4.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "877b9c3f61ceea0e56331985743b13f3d25c406a7098d45180fb5f09bc19ed97" +checksum = "76bc14366121efc8dbb487ab05bcc9d346b3b5ec0eaa76e46594cabbe51762c0" dependencies = [ "cfg-if", "js-sys", @@ -5723,9 +5727,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.91" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b30af9e2d358182b5c7449424f017eba305ed32a7010509ede96cdc4696c46ed" +checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -5733,22 +5737,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.91" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "642f325be6301eb8107a83d12a8ac6c1e1c54345a7ef1a9261962dfefda09e66" +checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.91" +version = "0.2.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f186bd2dcf04330886ce82d6f33dd75a7bfcf69ecf5763b89fcde53b6ac9838" +checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" [[package]] name = "watchable" @@ -5764,9 +5768,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.68" +version = "0.3.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96565907687f7aceb35bc5fc03770a8a0471d82e479f25832f54a0e3f4b28446" +checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" dependencies = [ "js-sys", "wasm-bindgen", @@ -5780,11 +5784,12 @@ checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1" [[package]] name = "whoami" -version = "1.4.1" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22fc3756b8a9133049b26c7f61ab35416c130e8c09b660f5b3958b446f52cc50" +checksum = "0fec781d48b41f8163426ed18e8fc2864c12937df9ce54c88ede7bd47270893e" dependencies = [ - "wasm-bindgen", + "redox_syscall", + "wasite", "web-sys", ] @@ -5859,7 +5864,17 @@ dependencies = [ "windows-core 0.52.0", "windows-implement", "windows-interface", - "windows-targets 0.52.0", + "windows-targets 0.52.4", +] + +[[package]] +name = "windows" +version = "0.54.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9252e5725dbed82865af151df558e754e4a3c2c30818359eb17465f1346a1b49" +dependencies = [ + "windows-core 0.54.0", + "windows-targets 0.52.4", ] [[package]] @@ -5877,7 +5892,17 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.0", + "windows-targets 0.52.4", +] + +[[package]] +name = "windows-core" +version = "0.54.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12661b9c89351d684a50a8a643ce5f608e20243b9fb84687800163429f161d65" +dependencies = [ + "windows-result", + "windows-targets 0.52.4", ] [[package]] @@ -5888,7 +5913,7 @@ checksum = "12168c33176773b86799be25e2a2ba07c7aab9968b37541f1094dbd7a60c8946" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] @@ -5899,7 +5924,16 @@ checksum = "9d8dc32e0095a7eeccebd0e3f09e9509365ecb3fc6ac4d6f5f14a3f6392942d1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", +] + +[[package]] +name = "windows-result" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd19df78e5168dfb0aedc343d1d1b8d422ab2db6756d2dc3fef75035402a3f64" +dependencies = [ + "windows-targets 0.52.4", ] [[package]] @@ -5917,7 +5951,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.0", + "windows-targets 0.52.4", ] [[package]] @@ -5937,17 +5971,17 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a18201040b24831fbb9e4eb208f8892e1f50a37feb53cc7ff887feb8f50e7cd" +checksum = "7dd37b7e5ab9018759f893a1952c9420d060016fc19a472b4bb20d1bdd694d1b" dependencies = [ - "windows_aarch64_gnullvm 0.52.0", - "windows_aarch64_msvc 0.52.0", - "windows_i686_gnu 0.52.0", - "windows_i686_msvc 0.52.0", - "windows_x86_64_gnu 0.52.0", - "windows_x86_64_gnullvm 0.52.0", - "windows_x86_64_msvc 0.52.0", + "windows_aarch64_gnullvm 0.52.4", + "windows_aarch64_msvc 0.52.4", + "windows_i686_gnu 0.52.4", + "windows_i686_msvc 0.52.4", + "windows_x86_64_gnu 0.52.4", + "windows_x86_64_gnullvm 0.52.4", + "windows_x86_64_msvc 0.52.4", ] [[package]] @@ -5958,9 +5992,9 @@ checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb7764e35d4db8a7921e09562a0304bf2f93e0a51bfccee0bd0bb0b666b015ea" +checksum = "bcf46cf4c365c6f2d1cc93ce535f2c8b244591df96ceee75d8e83deb70a9cac9" [[package]] name = "windows_aarch64_msvc" @@ -5970,9 +6004,9 @@ checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_aarch64_msvc" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbaa0368d4f1d2aaefc55b6fcfee13f41544ddf36801e793edbbfd7d7df075ef" +checksum = "da9f259dd3bcf6990b55bffd094c4f7235817ba4ceebde8e6d11cd0c5633b675" [[package]] name = "windows_i686_gnu" @@ -5982,9 +6016,9 @@ checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_gnu" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a28637cb1fa3560a16915793afb20081aba2c92ee8af57b4d5f28e4b3e7df313" +checksum = "b474d8268f99e0995f25b9f095bc7434632601028cf86590aea5c8a5cb7801d3" [[package]] name = "windows_i686_msvc" @@ -5994,9 +6028,9 @@ checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_i686_msvc" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffe5e8e31046ce6230cc7215707b816e339ff4d4d67c65dffa206fd0f7aa7b9a" +checksum = "1515e9a29e5bed743cb4415a9ecf5dfca648ce85ee42e15873c3cd8610ff8e02" [[package]] name = "windows_x86_64_gnu" @@ -6006,9 +6040,9 @@ checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnu" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6fa32db2bc4a2f5abeacf2b69f7992cd09dca97498da74a151a3132c26befd" +checksum = "5eee091590e89cc02ad514ffe3ead9eb6b660aedca2183455434b93546371a03" [[package]] name = "windows_x86_64_gnullvm" @@ -6018,9 +6052,9 @@ checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_gnullvm" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a657e1e9d3f514745a572a6846d3c7aa7dbe1658c056ed9c3344c4109a6949e" +checksum = "77ca79f2451b49fa9e2af39f0747fe999fcda4f5e241b2898624dca97a1f2177" [[package]] name = "windows_x86_64_msvc" @@ -6030,9 +6064,9 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "windows_x86_64_msvc" -version = "0.52.0" +version = "0.52.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dff9641d1cd4be8d1a070daf9e3773c5f67e78b4d9d42263020c057706765c04" +checksum = "32b752e52a2da0ddfbdbcc6fceadfeede4c939ed16d13e648833a61dfb611ed8" [[package]] name = "winnow" @@ -6045,9 +6079,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.1" +version = "0.6.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d90f4e0f530c4c69f62b80d839e9ef3855edc9cba471a160c4d692deed62b401" +checksum = "dffa400e67ed5a4dd237983829e66475f0a4a26938c4b04c21baede6262215b8" dependencies = [ "memchr", ] @@ -6064,9 +6098,9 @@ dependencies = [ [[package]] name = "wmi" -version = "0.13.2" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fff298e96fd8ef6bb55dcb2a7fd2f26969f962bf428ffa6b267457dd804d64d8" +checksum = "fc2f0a4062ca522aad4705a2948fd4061b3857537990202a8ddd5af21607f79a" dependencies = [ "chrono", "futures", @@ -6116,9 +6150,9 @@ checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" [[package]] name = "yansi" -version = "1.0.0-rc.1" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1367295b8f788d371ce2dbc842c7b709c73ee1364d30351dd300ec2203b12377" +checksum = "6c2861d76f58ec8fc95708b9b1e417f7b12fd72ad33c01fa6886707092dea0d3" [[package]] name = "yasna" @@ -6131,9 +6165,9 @@ dependencies = [ [[package]] name = "z32" -version = "1.0.3" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5c2c4e1757d25c3bfa9e137b15e458bc028bf569c2c22e59ecf9f62545d45fd" +checksum = "edb37266251c28b03d08162174a91c3a092e3bd4f476f8205ee1c507b78b7bdc" [[package]] name = "zerocopy" @@ -6152,7 +6186,7 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.49", + "syn 2.0.52", ] [[package]] From 0c47f280ce3a76caf09ffb74820c4a3450068340 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 21 Mar 2024 15:32:05 +0100 Subject: [PATCH 34/66] tests: add tests for DNS discovery --- Cargo.lock | 98 ++++++++++- iroh-net/Cargo.toml | 1 + iroh-net/src/discovery.rs | 306 ++++++++++++++++++++++++++++++++++ iroh-net/src/discovery/dns.rs | 3 +- iroh-net/src/dns/node_info.rs | 40 ++++- 5 files changed, 433 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3402673ad7a..d1f22a24eda 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -254,7 +254,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" dependencies = [ "async-trait", - "axum-core", + "axum-core 0.3.4", "bitflags 1.3.2", "bytes", "futures-util", @@ -275,6 +275,40 @@ dependencies = [ "tower-service", ] +[[package]] +name = "axum" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1236b4b292f6c4d6dc34604bb5120d85c3fe1d1aa596bd5cc52ca054d13e7b9e" +dependencies = [ + "async-trait", + "axum-core 0.4.3", + "bytes", + "futures-util", + "http 1.1.0", + "http-body 1.0.0", + "http-body-util", + "hyper 1.2.0", + "hyper-util", + "itoa", + "matchit", + "memchr", + "mime", + "percent-encoding", + "pin-project-lite", + "rustversion", + "serde", + "serde_json", + "serde_path_to_error", + "serde_urlencoded", + "sync_wrapper", + "tokio", + "tower", + "tower-layer", + "tower-service", + "tracing", +] + [[package]] name = "axum-core" version = "0.3.4" @@ -292,6 +326,27 @@ dependencies = [ "tower-service", ] +[[package]] +name = "axum-core" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a15c63fd72d41492dc4f497196f5da1fb04fb7529e631d73630d1b491e47a2e3" +dependencies = [ + "async-trait", + "bytes", + "futures-util", + "http 1.1.0", + "http-body 1.0.0", + "http-body-util", + "mime", + "pin-project-lite", + "rustversion", + "sync_wrapper", + "tower-layer", + "tower-service", + "tracing", +] + [[package]] name = "backoff" version = "0.4.0" @@ -1750,6 +1805,25 @@ dependencies = [ "tracing", ] +[[package]] +name = "h2" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51ee2dd2e4f378392eeff5d51618cd9a63166a2513846bbc55f21cfacd9199d4" +dependencies = [ + "bytes", + "fnv", + "futures-core", + "futures-sink", + "futures-util", + "http 1.1.0", + "indexmap 2.2.5", + "slab", + "tokio", + "tokio-util", + "tracing", +] + [[package]] name = "half" version = "2.4.0" @@ -2044,7 +2118,7 @@ dependencies = [ "futures-channel", "futures-core", "futures-util", - "h2", + "h2 0.3.25", "http 0.2.12", "http-body 0.4.6", "httparse", @@ -2067,6 +2141,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", + "h2 0.4.3", "http 1.1.0", "http-body 1.0.0", "httparse", @@ -2530,6 +2605,7 @@ version = "0.12.0" dependencies = [ "aead", "anyhow", + "axum 0.7.4", "backoff", "bytes", "clap", @@ -4178,7 +4254,7 @@ dependencies = [ "encoding_rs", "futures-core", "futures-util", - "h2", + "h2 0.3.25", "http 0.2.12", "http-body 0.4.6", "hyper 0.14.28", @@ -4587,6 +4663,16 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_path_to_error" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af99884400da37c88f5e9146b7f1fd0fbcae8f6eec4e9da38b67d05486f814a6" +dependencies = [ + "itoa", + "serde", +] + [[package]] name = "serde_spanned" version = "0.6.5" @@ -4738,7 +4824,7 @@ version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "01607fe2e61894468c6dc0b26103abb073fb08b79a3d9e4b6d76a1a341549958" dependencies = [ - "bitflags 2.4.2", + "bitflags 2.5.0", ] [[package]] @@ -5413,10 +5499,10 @@ checksum = "d560933a0de61cf715926b9cac824d4c883c2c43142f787595e48280c40a1d0e" dependencies = [ "async-stream", "async-trait", - "axum", + "axum 0.6.20", "base64 0.21.7", "bytes", - "h2", + "h2 0.3.25", "http 0.2.12", "http-body 0.4.6", "hyper 0.14.28", diff --git a/iroh-net/Cargo.toml b/iroh-net/Cargo.toml index d8395d226cb..f0d5566bc50 100644 --- a/iroh-net/Cargo.toml +++ b/iroh-net/Cargo.toml @@ -109,6 +109,7 @@ tokio = { version = "1", features = ["io-util", "sync", "rt", "net", "fs", "macr tracing-subscriber = { version = "0.3", features = ["env-filter"] } iroh-test = { path = "../iroh-test" } serde_json = "1.0.107" +axum = "0.7.4" [[bench]] name = "key" diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index ed58249c1db..bffff4240ca 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -535,3 +535,309 @@ mod tests { .as_micros() as u64 } } + +/// This module contains end-to-end tests for DNS node discovery. +/// +/// The tests use a minimal test DNS server to resolve against, and a minimal pkarr relay to +/// publish to. +#[cfg(test)] +mod test_dns_pkarr { + use std::net::SocketAddr; + + use anyhow::Result; + use hickory_resolver::{config::NameServerConfig, AsyncResolver, TokioAsyncResolver}; + use iroh_base::key::SecretKey; + use pkarr::SignedPacket; + use tokio_util::sync::CancellationToken; + use url::Url; + + use crate::{ + discovery::pkarr_relay_publish, + dns::node_info::{lookup_by_id, NodeInfo}, + AddrInfo, NodeAddr, + }; + + use self::state::State; + + #[tokio::test] + async fn dns_resolve() -> Result<()> { + let _logging_guard = iroh_test::logging::setup(); + let state = State::default(); + let cancel = CancellationToken::new(); + let origin = "testdns.example".to_string(); + let (dns_addr, dns_task) = + dns_server::spawn(state.clone(), origin.clone(), cancel.clone()).await?; + + let node_secret = SecretKey::generate(); + let (node_info, signed_packet) = generate_node_info(&node_secret); + state.upsert(signed_packet)?; + + let resolver = get_test_resolver(dns_addr)?; + let resolved = lookup_by_id(&resolver, &node_info.node_id, &origin).await?; + + assert_eq!(resolved, node_info.into()); + + cancel.cancel(); + dns_task.await??; + Ok(()) + } + + #[tokio::test] + async fn pkarr_publish_dns_resolve() -> Result<()> { + let _logging_guard = iroh_test::logging::setup(); + let state = State::default(); + let cancel = CancellationToken::new(); + let origin = "testdns.example".to_string(); + let (dns_addr, dns_task) = + dns_server::spawn(state.clone(), origin.clone(), cancel.clone()).await?; + + let (pkarr_url, pkarr_task) = pkarr_relay::spawn(state.clone(), cancel.clone()).await?; + + let secret_key = SecretKey::generate(); + let node_id = secret_key.public(); + let publisher = pkarr_relay_publish::Publisher::new(pkarr_relay_publish::Config::new( + secret_key, pkarr_url, + )); + + let addr_info = AddrInfo { + derp_url: Some("https://derp.example".parse().unwrap()), + ..Default::default() + }; + publisher.publish_addr_info(&addr_info).await?; + + let resolver = get_test_resolver(dns_addr)?; + let resolved = lookup_by_id(&resolver, &node_id, &origin).await?; + + let expected = NodeAddr { + info: addr_info, + node_id, + }; + + assert_eq!(resolved, expected); + + cancel.cancel(); + dns_task.await??; + pkarr_task.await??; + Ok(()) + } + + fn get_test_resolver(name_server: SocketAddr) -> Result { + let mut config = hickory_resolver::config::ResolverConfig::new(); + let nameserver_config = + NameServerConfig::new(name_server, hickory_resolver::config::Protocol::Udp); + config.add_name_server(nameserver_config); + let resolver = AsyncResolver::tokio(config, Default::default()); + Ok(resolver) + } + + fn generate_node_info(secret: &SecretKey) -> (NodeInfo, SignedPacket) { + let node_id = secret.public(); + let derp_url: Url = "https://derp.example".parse().expect("valid url"); + let node_info = NodeInfo { + node_id, + derp_url: Some(derp_url.clone()), + }; + let signed_packet = node_info + .to_pkarr_signed_packet(&secret, 30) + .expect("valid packet"); + (node_info, signed_packet) + } + + mod state { + use crate::NodeId; + use parking_lot::{Mutex, MutexGuard}; + use pkarr::SignedPacket; + use std::{ + collections::{hash_map, HashMap}, + ops::Deref, + sync::Arc, + }; + + #[derive(Default, Debug, Clone)] + pub struct State(Arc>>); + + impl State { + pub fn upsert(&self, signed_packet: SignedPacket) -> anyhow::Result { + let node_id = NodeId::from_bytes(&signed_packet.public_key().to_bytes())?; + let mut state = self.0.lock(); + let updated = match state.entry(node_id) { + hash_map::Entry::Vacant(e) => { + e.insert(signed_packet); + true + } + hash_map::Entry::Occupied(mut e) => { + if signed_packet.more_recent_than(e.get()) { + e.insert(signed_packet); + true + } else { + false + } + } + }; + Ok(updated) + } + pub fn get(&self, node_id: &NodeId) -> Option + '_> { + let state = self.0.lock(); + if state.contains_key(node_id) { + let guard = MutexGuard::map(state, |state| state.get_mut(node_id).unwrap()); + Some(guard) + } else { + None + } + } + } + } + + mod pkarr_relay { + use std::net::{Ipv4Addr, SocketAddr}; + + use anyhow::Result; + use axum::{ + extract::{Path, State}, + response::IntoResponse, + routing::put, + Router, + }; + use bytes::Bytes; + use tokio::task::JoinHandle; + use tokio_util::sync::CancellationToken; + use tracing::warn; + use url::Url; + + use super::State as AppState; + + pub async fn spawn( + state: AppState, + cancel: CancellationToken, + ) -> Result<(Url, JoinHandle>)> { + let bind_addr = SocketAddr::from((Ipv4Addr::LOCALHOST, 0)); + let app = Router::new() + .route("/pkarr/:key", put(pkarr_put)) + .with_state(state); + let listener = tokio::net::TcpListener::bind(bind_addr).await?; + let bound_addr = listener.local_addr()?; + let url: Url = format!("http://{bound_addr}/pkarr") + .parse() + .expect("valid url"); + let join_handle = tokio::task::spawn(async move { + let serve = axum::serve(listener, app); + let serve = serve.with_graceful_shutdown(cancel.cancelled_owned()); + serve.await?; + Ok(()) + }); + Ok((url, join_handle)) + } + + async fn pkarr_put( + State(state): State, + Path(key): Path, + body: Bytes, + ) -> Result { + let key = pkarr::PublicKey::try_from(key.as_str())?; + let signed_packet = pkarr::SignedPacket::from_relay_response(key, body)?; + let _updated = state.upsert(signed_packet)?; + Ok(http::StatusCode::NO_CONTENT) + } + + #[derive(Debug)] + struct AppError(anyhow::Error); + impl> From for AppError { + fn from(value: T) -> Self { + Self(value.into()) + } + } + impl IntoResponse for AppError { + fn into_response(self) -> axum::response::Response { + warn!(err = ?self, "request failed"); + (http::StatusCode::INTERNAL_SERVER_ERROR, self.0.to_string()).into_response() + } + } + } + + mod dns_server { + use std::net::{Ipv4Addr, SocketAddr}; + + use anyhow::{ensure, Result}; + use hickory_proto::{ + op::{header::MessageType, Message}, + serialize::binary::BinDecodable, + }; + use tokio::{net::UdpSocket, task::JoinHandle}; + use tokio_util::sync::CancellationToken; + use tracing::{debug, warn}; + + use crate::dns::node_info::{parse_hickory_node_info_name, NodeInfo}; + + use super::State; + + pub async fn spawn( + state: State, + origin: String, + cancel: CancellationToken, + ) -> Result<(SocketAddr, JoinHandle>)> { + let bind_addr = SocketAddr::from((Ipv4Addr::LOCALHOST, 0)); + let socket = UdpSocket::bind(bind_addr).await?; + let bound_addr = socket.local_addr()?; + let s = TestDnsServer { + socket, + cancel, + state, + origin, + }; + let join_handle = tokio::task::spawn(async move { s.run().await }); + Ok((bound_addr, join_handle)) + } + + struct TestDnsServer { + state: State, + socket: UdpSocket, + cancel: CancellationToken, + origin: String, + } + + impl TestDnsServer { + async fn run(self) -> Result<()> { + let mut buf = [0; 1450]; + loop { + tokio::select! { + _ = self.cancel.cancelled() => break, + res = self.socket.recv_from(&mut buf) => { + let (len, from) = res?; + if let Err(err) = self.handle_datagram(from, &buf[..len]).await { + warn!(?err, %from, "failed to handle datagram"); + } + } + }; + } + Ok(()) + } + + async fn handle_datagram(&self, from: SocketAddr, buf: &[u8]) -> Result<()> { + const TTL: u32 = 30; + let packet = Message::from_bytes(&buf)?; + let queries = packet.queries(); + debug!(?queries, %from, "received query"); + + let mut reply = packet.clone(); + reply.set_message_type(MessageType::Response); + for query in queries { + let Some(node_id) = parse_hickory_node_info_name(query.name()) else { + continue; + }; + let packet = self.state.get(&node_id); + let Some(packet) = packet.as_ref() else { + continue; + }; + let node_info = NodeInfo::from_pkarr_signed_packet(packet)?; + let record = node_info.to_hickory_record(&self.origin, TTL)?; + reply.add_answer(record); + } + debug!(?reply, %from, "send reply"); + let buf = reply.to_vec()?; + let len = self.socket.send_to(&buf, from).await?; + ensure!(len == buf.len(), "failed to send complete packet"); + Ok(()) + } + } + } +} diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs index c2608aa35c6..de8e49ebf6c 100644 --- a/iroh-net/src/discovery/dns.rs +++ b/iroh-net/src/discovery/dns.rs @@ -49,7 +49,8 @@ impl Discovery for DnsDiscovery { node_id: NodeId, ) -> Option>> { let fut = async move { - let node_addr = dns::node_info::lookup_by_id(&node_id, &self.node_origin).await?; + let node_addr = + dns::node_info::lookup_by_id(dns::resolver(), &node_id, &self.node_origin).await?; Ok(DiscoveryItem { provenance: "iroh-dns", last_updated: None, diff --git a/iroh-net/src/dns/node_info.rs b/iroh-net/src/dns/node_info.rs index 89270812a03..57ad8637e3f 100644 --- a/iroh-net/src/dns/node_info.rs +++ b/iroh-net/src/dns/node_info.rs @@ -5,7 +5,7 @@ use std::{collections::HashMap, fmt, str::FromStr}; use anyhow::{anyhow, bail, Result}; use hickory_proto::error::ProtoError; -use hickory_resolver::Name; +use hickory_resolver::{Name, TokioAsyncResolver}; use url::Url; use crate::{key::SecretKey, AddrInfo, NodeAddr, NodeId}; @@ -20,21 +20,28 @@ pub const IROH_NODE_TXT_LABEL: &str = "_iroh_node"; /// /// The domain name must either contain an _iroh_node TXT record or be a CNAME record that leads to /// an _iroh_node TXT record. -pub async fn lookup_by_domain(domain: &str) -> Result { +pub async fn lookup_by_domain(resolver: &TokioAsyncResolver, domain: &str) -> Result { let name = Name::from_str(domain)?; - let info = lookup_node_info(name).await?; + let info = lookup_node_info(resolver, name).await?; Ok(info.into()) } /// Lookup node info by node id and origin domain name. -pub async fn lookup_by_id(node_id: &NodeId, origin: &str) -> Result { +pub async fn lookup_by_id( + resolver: &TokioAsyncResolver, + node_id: &NodeId, + origin: &str, +) -> Result { let domain = format!("{}.{}", to_z32(node_id), origin); - lookup_by_domain(&domain).await + lookup_by_domain(resolver, &domain).await } -async fn lookup_node_info(name: Name) -> Result { +pub(crate) async fn lookup_node_info( + resolver: &TokioAsyncResolver, + name: Name, +) -> Result { let name = ensure_iroh_node_txt_label(name)?; - let lookup = super::resolver().txt_lookup(name).await?; + let lookup = resolver.txt_lookup(name).await?; NodeInfo::from_hickory_lookup(lookup.as_lookup()) } @@ -186,6 +193,23 @@ impl NodeInfo { Ok(packet) } + /// Convert into a [`hickory_proto::rr::Record`] DNS record. + pub fn to_hickory_record(&self, origin: &str, ttl: u32) -> Result { + use hickory_proto::rr; + let name = format!( + "{}.{}.{}", + IROH_NODE_TXT_LABEL, + to_z32(&self.node_id), + origin + ); + let name = rr::Name::from_utf8(&name)?; + let value = self.to_attribute_string(); + let txt = rr::rdata::TXT::new(vec![value]); + let rdata = rr::RData::TXT(txt); + let record = rr::Record::from_rdata(name, ttl, rdata); + Ok(record) + } + /// Try to parse a [`NodeInfo`] from a [`pkarr::SignedPacket`]. pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { use pkarr::dns::{self, rdata::RData}; @@ -216,7 +240,7 @@ impl NodeInfo { } } -fn parse_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { +pub(crate) fn parse_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { if name.num_labels() < 2 { return None; } From 3531449bc33247eb2f8b49db598b6ccfdf68feb1 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 21 Mar 2024 15:47:31 +0100 Subject: [PATCH 35/66] refactor: cleanup dns tests --- iroh-net/src/discovery.rs | 32 +++++++++++++++++++++++--------- iroh-net/src/discovery/dns.rs | 22 ++++++++++++++++++++-- 2 files changed, 43 insertions(+), 11 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index bffff4240ca..60d122345a9 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -538,8 +538,8 @@ mod tests { /// This module contains end-to-end tests for DNS node discovery. /// -/// The tests use a minimal test DNS server to resolve against, and a minimal pkarr relay to -/// publish to. +/// The tests run a minimal test DNS server to resolve against, and a minimal pkarr relay to +/// publish to. The relay and DNS servers share their state. #[cfg(test)] mod test_dns_pkarr { use std::net::SocketAddr; @@ -548,6 +548,7 @@ mod test_dns_pkarr { use hickory_resolver::{config::NameServerConfig, AsyncResolver, TokioAsyncResolver}; use iroh_base::key::SecretKey; use pkarr::SignedPacket; + use tokio::task::JoinHandle; use tokio_util::sync::CancellationToken; use url::Url; @@ -585,13 +586,11 @@ mod test_dns_pkarr { #[tokio::test] async fn pkarr_publish_dns_resolve() -> Result<()> { let _logging_guard = iroh_test::logging::setup(); - let state = State::default(); + let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); - let (dns_addr, dns_task) = - dns_server::spawn(state.clone(), origin.clone(), cancel.clone()).await?; - - let (pkarr_url, pkarr_task) = pkarr_relay::spawn(state.clone(), cancel.clone()).await?; + let (dns_addr, pkarr_url, task) = + spawn_dns_and_pkarr(origin.clone(), cancel.clone()).await?; let secret_key = SecretKey::generate(); let node_id = secret_key.public(); @@ -616,8 +615,7 @@ mod test_dns_pkarr { assert_eq!(resolved, expected); cancel.cancel(); - dns_task.await??; - pkarr_task.await??; + task.await??; Ok(()) } @@ -643,6 +641,22 @@ mod test_dns_pkarr { (node_info, signed_packet) } + async fn spawn_dns_and_pkarr( + origin: String, + cancel: CancellationToken, + ) -> Result<(SocketAddr, Url, JoinHandle>)> { + let state = State::default(); + let (dns_addr, dns_task) = + dns_server::spawn(state.clone(), origin.clone(), cancel.clone()).await?; + let (pkarr_url, pkarr_task) = pkarr_relay::spawn(state, cancel.clone()).await?; + let join_handle = tokio::task::spawn(async move { + dns_task.await??; + pkarr_task.await??; + Ok(()) + }); + Ok((dns_addr, pkarr_url, join_handle)) + } + mod state { use crate::NodeId; use parking_lot::{Mutex, MutexGuard}; diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs index de8e49ebf6c..939972b220b 100644 --- a/iroh-net/src/discovery/dns.rs +++ b/iroh-net/src/discovery/dns.rs @@ -6,6 +6,7 @@ use crate::{ }; use anyhow::Result; use futures::{future::FutureExt, stream::BoxStream, StreamExt}; +use hickory_resolver::TokioAsyncResolver; use crate::dns; @@ -28,12 +29,25 @@ pub const N0_TESTDNS_NODE_ORIGIN: &str = "testdns.iroh.link"; #[derive(Debug)] pub struct DnsDiscovery { node_origin: String, + resolver: Option, } impl DnsDiscovery { /// Create a new DNS discovery with `node_origin` appended to all lookups. pub fn new(node_origin: String) -> Self { - Self { node_origin } + Self { + node_origin, + resolver: None, + } + } + + /// Create a new DNS discovery with `node_origin` appended to all lookups, and a custom DNS + /// resolver instance + pub fn with_resolver(resolver: TokioAsyncResolver, node_origin: String) -> Self { + Self { + node_origin, + resolver: Some(resolver), + } } /// Create a new DNS discovery which uses the n0 testdns origin. @@ -49,8 +63,12 @@ impl Discovery for DnsDiscovery { node_id: NodeId, ) -> Option>> { let fut = async move { + let resolver = match &self.resolver { + Some(resolver) => resolver, + None => dns::resolver(), + }; let node_addr = - dns::node_info::lookup_by_id(dns::resolver(), &node_id, &self.node_origin).await?; + dns::node_info::lookup_by_id(resolver, &node_id, &self.node_origin).await?; Ok(DiscoveryItem { provenance: "iroh-dns", last_updated: None, From 1932c5c5f12522dde56d8e56305acc3230100750 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 21 Mar 2024 15:48:24 +0100 Subject: [PATCH 36/66] chore: clippy --- iroh-net/src/dns/node_info.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/dns/node_info.rs b/iroh-net/src/dns/node_info.rs index 57ad8637e3f..a46cf2ab21d 100644 --- a/iroh-net/src/dns/node_info.rs +++ b/iroh-net/src/dns/node_info.rs @@ -202,7 +202,7 @@ impl NodeInfo { to_z32(&self.node_id), origin ); - let name = rr::Name::from_utf8(&name)?; + let name = rr::Name::from_utf8(name)?; let value = self.to_attribute_string(); let txt = rr::rdata::TXT::new(vec![value]); let rdata = rr::RData::TXT(txt); From 443285bcbff90c71cd5ca8bab78b36a7591deed6 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 21 Mar 2024 16:05:29 +0100 Subject: [PATCH 37/66] refactor: make pkarr relay client reusable for derper --- iroh-net/src/discovery/pkarr_relay_publish.rs | 69 ++++++++++--------- 1 file changed, 35 insertions(+), 34 deletions(-) diff --git a/iroh-net/src/discovery/pkarr_relay_publish.rs b/iroh-net/src/discovery/pkarr_relay_publish.rs index 66a4cfbbf22..a35de11f9ef 100644 --- a/iroh-net/src/discovery/pkarr_relay_publish.rs +++ b/iroh-net/src/discovery/pkarr_relay_publish.rs @@ -14,7 +14,7 @@ use std::sync::Arc; use anyhow::Result; use parking_lot::RwLock; -use pkarr::PkarrClient; +use pkarr::SignedPacket; use tracing::warn; use url::Url; @@ -29,29 +29,23 @@ const DEFAULT_PKARR_TTL: u32 = 30; /// Publish node info to a pkarr relay. #[derive(derive_more::Debug, Clone)] pub struct Publisher { - config: Config, - #[debug("PkarrClient")] - pkarr_client: PkarrClient, - last_published: Arc>>, -} - -/// Publisher config -#[derive(derive_more::Debug, Clone)] -pub struct Config { #[debug("SecretKey")] secret_key: SecretKey, - #[debug("{}", self.pkarr_relay)] - pkarr_relay: Url, + #[debug("PkarrClient")] + pkarr_client: PkarrRelayClient, + last_published: Arc>>, ttl: u32, } -impl Config { +impl Publisher { /// Create a new config with a secret key and a pkarr relay URL. pub fn new(secret_key: SecretKey, pkarr_relay: Url) -> Self { + let pkarr_client = PkarrRelayClient::new(pkarr_relay); Self { secret_key, - pkarr_relay, + pkarr_client, ttl: DEFAULT_PKARR_TTL, + last_published: Default::default(), } } @@ -64,38 +58,22 @@ impl Config { /// Set the TTL for pkarr packets, in seconds. /// /// Default value is 30 seconds. - pub fn ttl(mut self, ttl: u32) -> Self { + pub fn set_ttl(&mut self, ttl: u32) { self.ttl = ttl; - self - } -} - -impl Publisher { - /// Create a new publisher with a [`Config`]. - pub fn new(config: Config) -> Self { - let pkarr_client = PkarrClient::builder().build(); - Self { - config, - pkarr_client, - last_published: Default::default(), - } } /// Publish [`AddrInfo`] about this node to a pkarr relay. pub async fn publish_addr_info(&self, info: &AddrInfo) -> Result<()> { let info = NodeInfo::new( - self.config.secret_key.public(), + self.secret_key.public(), info.derp_url.clone().map(Url::from), ); if self.last_published.read().as_ref() == Some(&info) { return Ok(()); } let _ = self.last_published.write().insert(info.clone()); - let signed_packet = - info.to_pkarr_signed_packet(&self.config.secret_key, self.config.ttl)?; - self.pkarr_client - .relay_put(&self.config.pkarr_relay, &signed_packet) - .await?; + let signed_packet = info.to_pkarr_signed_packet(&self.secret_key, self.ttl)?; + self.pkarr_client.publish(&signed_packet).await?; Ok(()) } } @@ -111,3 +89,26 @@ impl Discovery for Publisher { }); } } + +/// A pkarr client to publish [`pkarr::SignedPackets`] to a pkarr relay. +#[derive(Debug, Clone)] +pub(crate) struct PkarrRelayClient { + inner: pkarr::PkarrClient, + relay_url: Url, +} + +impl PkarrRelayClient { + /// Create a new client. + pub fn new(relay_url: Url) -> Self { + Self { + inner: pkarr::PkarrClient::builder().build(), + relay_url, + } + } + + /// Publish a [`SignedPacket`] + pub async fn publish(&self, signed_packet: &SignedPacket) -> anyhow::Result<()> { + self.inner.relay_put(&self.relay_url, signed_packet).await?; + Ok(()) + } +} From e519f8d5bfb86ee2aff1d68410e2db9fcd97ed27 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Fri, 22 Mar 2024 00:19:15 +0100 Subject: [PATCH 38/66] rebase fixup --- iroh-net/src/discovery.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index d8eb6909189..a0f9a55b327 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -594,12 +594,10 @@ mod test_dns_pkarr { let secret_key = SecretKey::generate(); let node_id = secret_key.public(); - let publisher = pkarr_relay_publish::Publisher::new(pkarr_relay_publish::Config::new( - secret_key, pkarr_url, - )); + let publisher = pkarr_relay_publish::Publisher::new(secret_key, pkarr_url); let addr_info = AddrInfo { - derp_url: Some("https://derp.example".parse().unwrap()), + relay_url: Some("https://relay.example".parse().unwrap()), ..Default::default() }; publisher.publish_addr_info(&addr_info).await?; @@ -630,10 +628,10 @@ mod test_dns_pkarr { fn generate_node_info(secret: &SecretKey) -> (NodeInfo, SignedPacket) { let node_id = secret.public(); - let derp_url: Url = "https://derp.example".parse().expect("valid url"); + let relay_url: Url = "https://relay.example".parse().expect("valid url"); let node_info = NodeInfo { node_id, - derp_url: Some(derp_url.clone()), + relay_url: Some(relay_url.clone()), }; let signed_packet = node_info .to_pkarr_signed_packet(&secret, 30) From 7ccae5945234c25177c7449bfa4d1e391450bfe5 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Fri, 22 Mar 2024 00:32:25 +0100 Subject: [PATCH 39/66] chore: clippy & doclinks --- iroh-net/src/discovery.rs | 4 ++-- iroh-net/src/discovery/pkarr_relay_publish.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index a0f9a55b327..3555e4ec8f0 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -634,7 +634,7 @@ mod test_dns_pkarr { relay_url: Some(relay_url.clone()), }; let signed_packet = node_info - .to_pkarr_signed_packet(&secret, 30) + .to_pkarr_signed_packet(secret, 30) .expect("valid packet"); (node_info, signed_packet) } @@ -826,7 +826,7 @@ mod test_dns_pkarr { async fn handle_datagram(&self, from: SocketAddr, buf: &[u8]) -> Result<()> { const TTL: u32 = 30; - let packet = Message::from_bytes(&buf)?; + let packet = Message::from_bytes(buf)?; let queries = packet.queries(); debug!(?queries, %from, "received query"); diff --git a/iroh-net/src/discovery/pkarr_relay_publish.rs b/iroh-net/src/discovery/pkarr_relay_publish.rs index 03cfcdcdf40..f1d03ec9052 100644 --- a/iroh-net/src/discovery/pkarr_relay_publish.rs +++ b/iroh-net/src/discovery/pkarr_relay_publish.rs @@ -90,7 +90,7 @@ impl Discovery for Publisher { } } -/// A pkarr client to publish [`pkarr::SignedPackets`] to a pkarr relay. +/// A pkarr client to publish [`pkarr::SignedPacket`]s to a pkarr relay. #[derive(Debug, Clone)] pub(crate) struct PkarrRelayClient { inner: pkarr::PkarrClient, From f798084511b98bf9ee1e7a848603151ee5ddcde8 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 12:53:02 +0100 Subject: [PATCH 40/66] refactor: use DNS resolver from MagicEndpoint --- iroh-net/src/discovery/dns.rs | 25 ++++--------------------- iroh-net/src/magic_endpoint.rs | 5 +++++ iroh-net/src/magicsock.rs | 5 +++++ 3 files changed, 14 insertions(+), 21 deletions(-) diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs index 939972b220b..ada51944d57 100644 --- a/iroh-net/src/discovery/dns.rs +++ b/iroh-net/src/discovery/dns.rs @@ -6,7 +6,6 @@ use crate::{ }; use anyhow::Result; use futures::{future::FutureExt, stream::BoxStream, StreamExt}; -use hickory_resolver::TokioAsyncResolver; use crate::dns; @@ -29,25 +28,12 @@ pub const N0_TESTDNS_NODE_ORIGIN: &str = "testdns.iroh.link"; #[derive(Debug)] pub struct DnsDiscovery { node_origin: String, - resolver: Option, } impl DnsDiscovery { /// Create a new DNS discovery with `node_origin` appended to all lookups. pub fn new(node_origin: String) -> Self { - Self { - node_origin, - resolver: None, - } - } - - /// Create a new DNS discovery with `node_origin` appended to all lookups, and a custom DNS - /// resolver instance - pub fn with_resolver(resolver: TokioAsyncResolver, node_origin: String) -> Self { - Self { - node_origin, - resolver: Some(resolver), - } + Self { node_origin } } /// Create a new DNS discovery which uses the n0 testdns origin. @@ -59,16 +45,13 @@ impl DnsDiscovery { impl Discovery for DnsDiscovery { fn resolve( &self, - _ep: MagicEndpoint, + ep: MagicEndpoint, node_id: NodeId, ) -> Option>> { + let resolver = ep.dns_resolver().clone(); let fut = async move { - let resolver = match &self.resolver { - Some(resolver) => resolver, - None => dns::resolver(), - }; let node_addr = - dns::node_info::lookup_by_id(resolver, &node_id, &self.node_origin).await?; + dns::node_info::lookup_by_id(&resolver, &node_id, &self.node_origin).await?; Ok(DiscoveryItem { provenance: "iroh-dns", last_updated: None, diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index 6301fd695df..4a2c6f3d292 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -505,6 +505,11 @@ impl MagicEndpoint { Ok(()) } + /// Get a reference to the DNS resolver used in this [`MagicEndpoint`]. + pub fn dns_resolver(&self) -> &DnsResolver { + &self.msock.dns_resolver() + } + /// Close the QUIC endpoint and the magic socket. /// /// This will close all open QUIC connections with the provided error_code and reason. See diff --git a/iroh-net/src/magicsock.rs b/iroh-net/src/magicsock.rs index b6302b44dbb..40f2897b773 100644 --- a/iroh-net/src/magicsock.rs +++ b/iroh-net/src/magicsock.rs @@ -1362,6 +1362,11 @@ impl MagicSock { self.inner.node_map.add_node_addr(addr); } + /// Get a reference to the DNS resolver used in this [`MagicSock`]. + pub fn dns_resolver(&self) -> &DnsResolver { + &self.inner.dns_resolver + } + /// Closes the connection. /// /// Only the first close does anything. Any later closes return nil. From ce48317a82393860190bcc5ec5f129011825bdd7 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 16:43:09 +0100 Subject: [PATCH 41/66] tests: add end-to-end test --- iroh-base/src/node_addr.rs | 8 +- iroh-net/src/discovery.rs | 262 ++++++++++-------- iroh-net/src/discovery/pkarr_relay_publish.rs | 6 + iroh-net/src/dns.rs | 93 ++++++- iroh/src/node/builder.rs | 14 +- 5 files changed, 258 insertions(+), 125 deletions(-) diff --git a/iroh-base/src/node_addr.rs b/iroh-base/src/node_addr.rs index 05c86f78692..5a291223820 100644 --- a/iroh-base/src/node_addr.rs +++ b/iroh-base/src/node_addr.rs @@ -4,7 +4,7 @@ use anyhow::Context; use serde::{Deserialize, Serialize}; use url::Url; -use crate::key::PublicKey; +use crate::key::{NodeId, PublicKey}; /// A peer and it's addressing information. #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] @@ -63,6 +63,12 @@ impl From<(PublicKey, Option, &[SocketAddr])> for NodeAddr { } } +impl From for NodeAddr { + fn from(node_id: NodeId) -> Self { + NodeAddr::new(node_id) + } +} + /// Addressing information to connect to a peer. #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Default)] pub struct AddrInfo { diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 3555e4ec8f0..63df3e2f160 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -70,11 +70,16 @@ pub struct ConcurrentDiscovery { } impl ConcurrentDiscovery { - /// Create a new [`ConcurrentDiscovery`]. - pub fn new() -> Self { + /// Create a empty [`ConcurrentDiscovery`]. + pub fn empty() -> Self { Self::default() } + /// Create a new [`ConcurrentDiscovery`]. + pub fn new(services: Vec>) -> Self { + Self { services } + } + /// Add a [`Discovery`] service. pub fn add(&mut self, service: impl Discovery + 'static) { self.services.push(Box::new(service)); @@ -421,7 +426,7 @@ mod tests { let secret = SecretKey::generate(); let disco1 = EmptyDiscovery; let disco2 = disco_shared.create_discovery(secret.public()); - let mut disco = ConcurrentDiscovery::new(); + let mut disco = ConcurrentDiscovery::empty(); disco.add(disco1); disco.add(disco2); new_endpoint(secret, disco).await @@ -450,7 +455,7 @@ mod tests { let disco1 = EmptyDiscovery; let disco2 = disco_shared.create_lying_discovery(secret.public()); let disco3 = disco_shared.create_discovery(secret.public()); - let mut disco = ConcurrentDiscovery::new(); + let mut disco = ConcurrentDiscovery::empty(); disco.add(disco1); disco.add(disco2); disco.add(disco3); @@ -476,8 +481,7 @@ mod tests { let ep2 = { let secret = SecretKey::generate(); let disco1 = disco_shared.create_lying_discovery(secret.public()); - let mut disco = ConcurrentDiscovery::new(); - disco.add(disco1); + let disco = ConcurrentDiscovery::new(vec![Box::new(disco1)]); new_endpoint(secret, disco).await }; let ep1_addr = NodeAddr::new(ep1.node_id()); @@ -542,6 +546,7 @@ mod tests { /// publish to. The relay and DNS servers share their state. #[cfg(test)] mod test_dns_pkarr { + use std::future::Future; use std::net::SocketAddr; use anyhow::Result; @@ -554,26 +559,32 @@ mod test_dns_pkarr { use crate::{ discovery::pkarr_relay_publish, - dns::node_info::{lookup_by_id, NodeInfo}, - AddrInfo, NodeAddr, + dns::{ + node_info::{lookup_by_id, parse_hickory_node_info_name, NodeInfo}, + tests::dns_server::{self, Resolver}, + }, + relay::{RelayMap, RelayMode}, + test_utils::run_relay_server, + AddrInfo, MagicEndpoint, NodeAddr, }; use self::state::State; + use super::{dns::DnsDiscovery, ConcurrentDiscovery}; + #[tokio::test] async fn dns_resolve() -> Result<()> { let _logging_guard = iroh_test::logging::setup(); - let state = State::default(); let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); - let (dns_addr, dns_task) = - dns_server::spawn(state.clone(), origin.clone(), cancel.clone()).await?; + let state = State::new(origin.clone()); + let (dns_addr, dns_task) = dns_server::spawn(state.clone(), cancel.clone()).await?; let node_secret = SecretKey::generate(); let (node_info, signed_packet) = generate_node_info(&node_secret); state.upsert(signed_packet)?; - let resolver = get_test_resolver(dns_addr)?; + let resolver = dns_resolver(dns_addr)?; let resolved = lookup_by_id(&resolver, &node_info.node_id, &origin).await?; assert_eq!(resolved, node_info.into()); @@ -589,7 +600,7 @@ mod test_dns_pkarr { let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); - let (dns_addr, pkarr_url, task) = + let (dns_addr, pkarr_url, _state, task) = spawn_dns_and_pkarr(origin.clone(), cancel.clone()).await?; let secret_key = SecretKey::generate(); @@ -602,7 +613,7 @@ mod test_dns_pkarr { }; publisher.publish_addr_info(&addr_info).await?; - let resolver = get_test_resolver(dns_addr)?; + let resolver = dns_resolver(dns_addr)?; let resolved = lookup_by_id(&resolver, &node_id, &origin).await?; let expected = NodeAddr { @@ -617,10 +628,70 @@ mod test_dns_pkarr { Ok(()) } - fn get_test_resolver(name_server: SocketAddr) -> Result { + const TEST_ALPN: &[u8] = b"TEST"; + + #[tokio::test] + async fn pkarr_publish_dns_discover() -> Result<()> { + let _logging_guard = iroh_test::logging::setup(); + + let cancel = CancellationToken::new(); + let origin = "testdns.example".to_string(); + let (nameserver, pkarr_url, state, task) = + spawn_dns_and_pkarr(origin.clone(), cancel.clone()).await?; + + let (relay_map, _relay_url, _relay_guard) = run_relay_server().await.unwrap(); + let ep1 = ep_with_discovery(relay_map.clone(), nameserver, &origin, &pkarr_url).await?; + println!("ep1 {}", ep1.node_id()); + let ep2 = ep_with_discovery(relay_map, nameserver, &origin, &pkarr_url).await?; + println!("ep2 {}", ep2.node_id()); + + // wait for out address to be updated and thus published at least once + ep2.my_addr().await?; + // wait a bit longer for the publish to be done + // TODO: add event on state for this + // tokio::time::sleep(std::time::Duration::from_millis(500)).await; + state.on_update().await; + + // we connect only by node id! + let ep2_node_id = ep2.node_id(); + println!("connect ep1 {} -> ep2 {}", ep1.node_id(), ep2.node_id()); + let res = ep1.connect(ep2_node_id.into(), TEST_ALPN).await; + assert!(res.is_ok(), "connection established"); + cancel.cancel(); + task.await??; + Ok(()) + } + + async fn ep_with_discovery( + relay_map: RelayMap, + nameserver: SocketAddr, + node_origin: &str, + pkarr_relay: &Url, + ) -> Result { + let secret_key = SecretKey::generate(); + let resolver = dns_resolver(nameserver)?; + let discovery = ConcurrentDiscovery::new(vec![ + Box::new(DnsDiscovery::new(node_origin.to_string())), + Box::new(pkarr_relay_publish::Publisher::new( + secret_key.clone(), + pkarr_relay.clone(), + )), + ]); + let ep = MagicEndpoint::builder() + .relay_mode(RelayMode::Custom(relay_map)) + .secret_key(secret_key) + .dns_resolver(resolver) + .alpns(vec![TEST_ALPN.to_vec()]) + .discovery(Box::new(discovery)) + .bind(0) + .await?; + Ok(ep) + } + + fn dns_resolver(nameserver: SocketAddr) -> Result { let mut config = hickory_resolver::config::ResolverConfig::new(); let nameserver_config = - NameServerConfig::new(name_server, hickory_resolver::config::Protocol::Udp); + NameServerConfig::new(nameserver, hickory_resolver::config::Protocol::Udp); config.add_name_server(nameserver_config); let resolver = AsyncResolver::tokio(config, Default::default()); Ok(resolver) @@ -642,17 +713,16 @@ mod test_dns_pkarr { async fn spawn_dns_and_pkarr( origin: String, cancel: CancellationToken, - ) -> Result<(SocketAddr, Url, JoinHandle>)> { - let state = State::default(); - let (dns_addr, dns_task) = - dns_server::spawn(state.clone(), origin.clone(), cancel.clone()).await?; - let (pkarr_url, pkarr_task) = pkarr_relay::spawn(state, cancel.clone()).await?; + ) -> Result<(SocketAddr, Url, State, JoinHandle>)> { + let state = State::new(origin); + let (dns_addr, dns_task) = dns_server::spawn(state.clone(), cancel.clone()).await?; + let (pkarr_url, pkarr_task) = pkarr_relay::spawn(state.clone(), cancel.clone()).await?; let join_handle = tokio::task::spawn(async move { dns_task.await??; pkarr_task.await??; Ok(()) }); - Ok((dns_addr, pkarr_url, join_handle)) + Ok((dns_addr, pkarr_url, state, join_handle)) } mod state { @@ -665,14 +735,30 @@ mod test_dns_pkarr { sync::Arc, }; - #[derive(Default, Debug, Clone)] - pub struct State(Arc>>); + #[derive(Debug, Clone)] + pub struct State { + packets: Arc>>, + pub origin: String, + notify: Arc, + } impl State { + pub fn new(origin: String) -> Self { + Self { + packets: Default::default(), + origin, + notify: Arc::new(tokio::sync::Notify::new()), + } + } + + pub fn on_update(&self) -> tokio::sync::futures::Notified<'_> { + self.notify.notified() + } + pub fn upsert(&self, signed_packet: SignedPacket) -> anyhow::Result { let node_id = NodeId::from_bytes(&signed_packet.public_key().to_bytes())?; - let mut state = self.0.lock(); - let updated = match state.entry(node_id) { + let mut map = self.packets.lock(); + let updated = match map.entry(node_id) { hash_map::Entry::Vacant(e) => { e.insert(signed_packet); true @@ -686,12 +772,16 @@ mod test_dns_pkarr { } } }; + if updated { + self.notify.notify_waiters(); + } Ok(updated) } pub fn get(&self, node_id: &NodeId) -> Option + '_> { - let state = self.0.lock(); - if state.contains_key(node_id) { - let guard = MutexGuard::map(state, |state| state.get_mut(node_id).unwrap()); + println!("GET {node_id}"); + let map = self.packets.lock(); + if map.contains_key(node_id) { + let guard = MutexGuard::map(map, |state| state.get_mut(node_id).unwrap()); Some(guard) } else { None @@ -700,6 +790,33 @@ mod test_dns_pkarr { } } + impl Resolver for State { + fn resolve( + &self, + query: &hickory_proto::op::Message, + reply: &mut hickory_proto::op::Message, + ) -> impl Future> + Send { + const TTL: u32 = 30; + let this = self.clone(); + async move { + println!("QUERY {:?}", query.queries()); + for query in query.queries() { + let Some(node_id) = parse_hickory_node_info_name(query.name()) else { + continue; + }; + let packet = this.get(&node_id); + let Some(packet) = packet.as_ref() else { + continue; + }; + let node_info = NodeInfo::from_pkarr_signed_packet(packet)?; + let record = node_info.to_hickory_record(&this.origin, TTL)?; + reply.add_answer(record); + } + Ok(()) + } + } + } + mod pkarr_relay { use std::net::{Ipv4Addr, SocketAddr}; @@ -765,91 +882,4 @@ mod test_dns_pkarr { } } } - - mod dns_server { - use std::net::{Ipv4Addr, SocketAddr}; - - use anyhow::{ensure, Result}; - use hickory_proto::{ - op::{header::MessageType, Message}, - serialize::binary::BinDecodable, - }; - use tokio::{net::UdpSocket, task::JoinHandle}; - use tokio_util::sync::CancellationToken; - use tracing::{debug, warn}; - - use crate::dns::node_info::{parse_hickory_node_info_name, NodeInfo}; - - use super::State; - - pub async fn spawn( - state: State, - origin: String, - cancel: CancellationToken, - ) -> Result<(SocketAddr, JoinHandle>)> { - let bind_addr = SocketAddr::from((Ipv4Addr::LOCALHOST, 0)); - let socket = UdpSocket::bind(bind_addr).await?; - let bound_addr = socket.local_addr()?; - let s = TestDnsServer { - socket, - cancel, - state, - origin, - }; - let join_handle = tokio::task::spawn(async move { s.run().await }); - Ok((bound_addr, join_handle)) - } - - struct TestDnsServer { - state: State, - socket: UdpSocket, - cancel: CancellationToken, - origin: String, - } - - impl TestDnsServer { - async fn run(self) -> Result<()> { - let mut buf = [0; 1450]; - loop { - tokio::select! { - _ = self.cancel.cancelled() => break, - res = self.socket.recv_from(&mut buf) => { - let (len, from) = res?; - if let Err(err) = self.handle_datagram(from, &buf[..len]).await { - warn!(?err, %from, "failed to handle datagram"); - } - } - }; - } - Ok(()) - } - - async fn handle_datagram(&self, from: SocketAddr, buf: &[u8]) -> Result<()> { - const TTL: u32 = 30; - let packet = Message::from_bytes(buf)?; - let queries = packet.queries(); - debug!(?queries, %from, "received query"); - - let mut reply = packet.clone(); - reply.set_message_type(MessageType::Response); - for query in queries { - let Some(node_id) = parse_hickory_node_info_name(query.name()) else { - continue; - }; - let packet = self.state.get(&node_id); - let Some(packet) = packet.as_ref() else { - continue; - }; - let node_info = NodeInfo::from_pkarr_signed_packet(packet)?; - let record = node_info.to_hickory_record(&self.origin, TTL)?; - reply.add_answer(record); - } - debug!(?reply, %from, "send reply"); - let buf = reply.to_vec()?; - let len = self.socket.send_to(&buf, from).await?; - ensure!(len == buf.len(), "failed to send complete packet"); - Ok(()) - } - } - } } diff --git a/iroh-net/src/discovery/pkarr_relay_publish.rs b/iroh-net/src/discovery/pkarr_relay_publish.rs index f1d03ec9052..e39e37b1054 100644 --- a/iroh-net/src/discovery/pkarr_relay_publish.rs +++ b/iroh-net/src/discovery/pkarr_relay_publish.rs @@ -73,6 +73,12 @@ impl Publisher { } let _ = self.last_published.write().insert(info.clone()); let signed_packet = info.to_pkarr_signed_packet(&self.secret_key, self.ttl)?; + println!( + "PUBLISH!!! self {} sp pk {}, nid from sp {}", + self.secret_key.public(), + signed_packet.public_key(), + crate::NodeId::from_bytes(signed_packet.public_key().as_bytes()).unwrap() + ); self.pkarr_client.publish(&signed_packet).await?; Ok(()) } diff --git a/iroh-net/src/dns.rs b/iroh-net/src/dns.rs index 39068cac2b7..b1a5d649a96 100644 --- a/iroh-net/src/dns.rs +++ b/iroh-net/src/dns.rs @@ -149,7 +149,7 @@ pub async fn lookup_ipv4_ipv6( } #[cfg(test)] -mod tests { +pub(crate) mod tests { use crate::defaults::NA_RELAY_HOSTNAME; use super::*; @@ -176,4 +176,95 @@ mod tests { assert!(!res.is_empty()); dbg!(res); } + + pub(crate) mod dns_server { + use std::net::{Ipv4Addr, SocketAddr}; + + use anyhow::{ensure, Result}; + use futures::{future::BoxFuture, Future}; + use hickory_proto::{ + op::{header::MessageType, Message}, + serialize::binary::BinDecodable, + }; + use tokio::{net::UdpSocket, task::JoinHandle}; + use tokio_util::sync::CancellationToken; + use tracing::{debug, warn}; + + pub trait Resolver: Send + Sync + 'static { + fn resolve( + &self, + query: &Message, + reply: &mut Message, + ) -> impl Future> + Send; + } + + pub type ResolveCallback = Box< + dyn Fn(&Message, &mut Message) -> BoxFuture<'static, Result<()>> + + Send + + Sync + + 'static, + >; + impl Resolver for ResolveCallback { + fn resolve( + &self, + query: &Message, + reply: &mut Message, + ) -> impl Future> + Send { + (&self)(query, reply) + } + } + + pub async fn spawn( + resolver: impl Resolver, + cancel: CancellationToken, + ) -> Result<(SocketAddr, JoinHandle>)> { + let bind_addr = SocketAddr::from((Ipv4Addr::LOCALHOST, 0)); + let socket = UdpSocket::bind(bind_addr).await?; + let bound_addr = socket.local_addr()?; + let s = TestDnsServer { + socket, + cancel, + resolver, + }; + let join_handle = tokio::task::spawn(async move { s.run().await }); + Ok((bound_addr, join_handle)) + } + + struct TestDnsServer { + resolver: R, + socket: UdpSocket, + cancel: CancellationToken, + } + + impl TestDnsServer { + async fn run(self) -> Result<()> { + let mut buf = [0; 1450]; + loop { + tokio::select! { + _ = self.cancel.cancelled() => break, + res = self.socket.recv_from(&mut buf) => { + let (len, from) = res?; + if let Err(err) = self.handle_datagram(from, &buf[..len]).await { + warn!(?err, %from, "failed to handle incoming datagram"); + } + } + }; + } + Ok(()) + } + + async fn handle_datagram(&self, from: SocketAddr, buf: &[u8]) -> Result<()> { + let packet = Message::from_bytes(buf)?; + debug!(queries = ?packet.queries(), %from, "received query"); + let mut reply = packet.clone(); + reply.set_message_type(MessageType::Response); + self.resolver.resolve(&packet, &mut reply).await?; + debug!(?reply, %from, "send reply"); + let buf = reply.to_vec()?; + let len = self.socket.send_to(&buf, from).await?; + ensure!(len == buf.len(), "failed to send complete packet"); + Ok(()) + } + } + } } diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs index 668ab9b6099..c7fc4d7daf5 100644 --- a/iroh/src/node/builder.rs +++ b/iroh/src/node/builder.rs @@ -331,13 +331,13 @@ where NodeDiscoveryConfig::None => None, NodeDiscoveryConfig::Custom(discovery) => Some(discovery), NodeDiscoveryConfig::Default => { - let mut discovery = ConcurrentDiscovery::new(); - let dns_discovery = DnsDiscovery::n0_testdns(); - discovery.add(dns_discovery); - // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. - let pkarr_publish = - pkarr_relay_publish::Publisher::n0_testdns(self.secret_key.clone()); - discovery.add(pkarr_publish); + let discovery = ConcurrentDiscovery::new(vec![ + // Enable DNS discovery by default + DnsDiscovery::n0_testdns(), + // Enable pkarr publishing by default + // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. + pkarr_relay_publish::Publisher::n0_testdns(self.secret_key.clone()), + ]); Some(Box::new(discovery)) } }; From d1465f75c8a6c82a9e47e4ad54599a13eb858b74 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 16:44:51 +0100 Subject: [PATCH 42/66] fixup --- iroh-net/src/discovery.rs | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 63df3e2f160..81445a835b3 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -641,20 +641,13 @@ mod test_dns_pkarr { let (relay_map, _relay_url, _relay_guard) = run_relay_server().await.unwrap(); let ep1 = ep_with_discovery(relay_map.clone(), nameserver, &origin, &pkarr_url).await?; - println!("ep1 {}", ep1.node_id()); let ep2 = ep_with_discovery(relay_map, nameserver, &origin, &pkarr_url).await?; - println!("ep2 {}", ep2.node_id()); - // wait for out address to be updated and thus published at least once - ep2.my_addr().await?; - // wait a bit longer for the publish to be done - // TODO: add event on state for this - // tokio::time::sleep(std::time::Duration::from_millis(500)).await; + // wait until our shared state received the update from pkarr publishing state.on_update().await; // we connect only by node id! let ep2_node_id = ep2.node_id(); - println!("connect ep1 {} -> ep2 {}", ep1.node_id(), ep2.node_id()); let res = ep1.connect(ep2_node_id.into(), TEST_ALPN).await; assert!(res.is_ok(), "connection established"); cancel.cancel(); From 536e3bd6244812440110bbe9582c74f37bd4a6c1 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 16:49:08 +0100 Subject: [PATCH 43/66] cleanup --- iroh-net/src/discovery.rs | 8 ++++---- .../{pkarr_relay_publish.rs => pkarr_publish.rs} | 0 iroh/src/node/builder.rs | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) rename iroh-net/src/discovery/{pkarr_relay_publish.rs => pkarr_publish.rs} (100%) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 81445a835b3..d851de5e48a 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -11,7 +11,7 @@ use tracing::{debug, error_span, warn, Instrument}; use crate::{AddrInfo, MagicEndpoint, NodeId}; pub mod dns; -pub mod pkarr_relay_publish; +pub mod pkarr_publish; /// Node discovery for [`super::MagicEndpoint`]. /// @@ -558,7 +558,7 @@ mod test_dns_pkarr { use url::Url; use crate::{ - discovery::pkarr_relay_publish, + discovery::pkarr_publish, dns::{ node_info::{lookup_by_id, parse_hickory_node_info_name, NodeInfo}, tests::dns_server::{self, Resolver}, @@ -605,7 +605,7 @@ mod test_dns_pkarr { let secret_key = SecretKey::generate(); let node_id = secret_key.public(); - let publisher = pkarr_relay_publish::Publisher::new(secret_key, pkarr_url); + let publisher = pkarr_publish::Publisher::new(secret_key, pkarr_url); let addr_info = AddrInfo { relay_url: Some("https://relay.example".parse().unwrap()), @@ -665,7 +665,7 @@ mod test_dns_pkarr { let resolver = dns_resolver(nameserver)?; let discovery = ConcurrentDiscovery::new(vec![ Box::new(DnsDiscovery::new(node_origin.to_string())), - Box::new(pkarr_relay_publish::Publisher::new( + Box::new(pkarr_publish::Publisher::new( secret_key.clone(), pkarr_relay.clone(), )), diff --git a/iroh-net/src/discovery/pkarr_relay_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs similarity index 100% rename from iroh-net/src/discovery/pkarr_relay_publish.rs rename to iroh-net/src/discovery/pkarr_publish.rs diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs index 50c90003e2c..d7bcd6c5128 100644 --- a/iroh/src/node/builder.rs +++ b/iroh/src/node/builder.rs @@ -16,7 +16,7 @@ use iroh_bytes::{ }; use iroh_gossip::net::{Gossip, GOSSIP_ALPN}; use iroh_net::{ - discovery::{dns::DnsDiscovery, pkarr_relay_publish, ConcurrentDiscovery, Discovery}, + discovery::{dns::DnsDiscovery, pkarr_publish, ConcurrentDiscovery, Discovery}, magic_endpoint::get_alpn, relay::RelayMode, util::AbortingJoinHandle, @@ -122,7 +122,7 @@ impl Default for Builder { relay_mode: RelayMode::Default, rpc_endpoint: Default::default(), gc_policy: GcPolicy::Disabled, - docs_store: iroh_sync::store::Store::memory(), + docs_store: iroh_sync::store::Store::memory(), node_discovery: Default::default(), } } @@ -336,7 +336,7 @@ where Box::new(DnsDiscovery::n0_testdns()), // Enable pkarr publishing by default // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. - Box::new(pkarr_relay_publish::Publisher::n0_testdns( + Box::new(pkarr_publish::Publisher::n0_testdns( self.secret_key.clone(), )), ]); From 9e5759d0a822b18f9456fbac7c9e9d5d28b25bee Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 16:56:51 +0100 Subject: [PATCH 44/66] cleanup --- iroh-net/src/discovery.rs | 29 ++++++------ iroh-net/src/dns.rs | 91 -------------------------------------- iroh-net/src/test_utils.rs | 88 ++++++++++++++++++++++++++++++++++++ 3 files changed, 102 insertions(+), 106 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index d851de5e48a..6f19cc014d9 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -559,16 +559,16 @@ mod test_dns_pkarr { use crate::{ discovery::pkarr_publish, - dns::{ - node_info::{lookup_by_id, parse_hickory_node_info_name, NodeInfo}, - tests::dns_server::{self, Resolver}, - }, + dns::node_info::{lookup_by_id, parse_hickory_node_info_name, NodeInfo}, relay::{RelayMap, RelayMode}, - test_utils::run_relay_server, + test_utils::{ + dns_server::{run_dns_server, Resolver}, + run_relay_server, + }, AddrInfo, MagicEndpoint, NodeAddr, }; - use self::state::State; + use self::{pkarr_relay::run_pkarr_relay, state::State}; use super::{dns::DnsDiscovery, ConcurrentDiscovery}; @@ -578,13 +578,13 @@ mod test_dns_pkarr { let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); let state = State::new(origin.clone()); - let (dns_addr, dns_task) = dns_server::spawn(state.clone(), cancel.clone()).await?; + let (nameserver, dns_task) = run_dns_server(state.clone(), cancel.clone()).await?; let node_secret = SecretKey::generate(); let (node_info, signed_packet) = generate_node_info(&node_secret); state.upsert(signed_packet)?; - let resolver = dns_resolver(dns_addr)?; + let resolver = dns_resolver(nameserver)?; let resolved = lookup_by_id(&resolver, &node_info.node_id, &origin).await?; assert_eq!(resolved, node_info.into()); @@ -600,7 +600,7 @@ mod test_dns_pkarr { let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); - let (dns_addr, pkarr_url, _state, task) = + let (nameserver, pkarr_url, _state, task) = spawn_dns_and_pkarr(origin.clone(), cancel.clone()).await?; let secret_key = SecretKey::generate(); @@ -613,7 +613,7 @@ mod test_dns_pkarr { }; publisher.publish_addr_info(&addr_info).await?; - let resolver = dns_resolver(dns_addr)?; + let resolver = dns_resolver(nameserver)?; let resolved = lookup_by_id(&resolver, &node_id, &origin).await?; let expected = NodeAddr { @@ -708,14 +708,14 @@ mod test_dns_pkarr { cancel: CancellationToken, ) -> Result<(SocketAddr, Url, State, JoinHandle>)> { let state = State::new(origin); - let (dns_addr, dns_task) = dns_server::spawn(state.clone(), cancel.clone()).await?; - let (pkarr_url, pkarr_task) = pkarr_relay::spawn(state.clone(), cancel.clone()).await?; + let (nameserver, dns_task) = run_dns_server(state.clone(), cancel.clone()).await?; + let (pkarr_url, pkarr_task) = run_pkarr_relay(state.clone(), cancel.clone()).await?; let join_handle = tokio::task::spawn(async move { dns_task.await??; pkarr_task.await??; Ok(()) }); - Ok((dns_addr, pkarr_url, state, join_handle)) + Ok((nameserver, pkarr_url, state, join_handle)) } mod state { @@ -792,7 +792,6 @@ mod test_dns_pkarr { const TTL: u32 = 30; let this = self.clone(); async move { - println!("QUERY {:?}", query.queries()); for query in query.queries() { let Some(node_id) = parse_hickory_node_info_name(query.name()) else { continue; @@ -828,7 +827,7 @@ mod test_dns_pkarr { use super::State as AppState; - pub async fn spawn( + pub async fn run_pkarr_relay( state: AppState, cancel: CancellationToken, ) -> Result<(Url, JoinHandle>)> { diff --git a/iroh-net/src/dns.rs b/iroh-net/src/dns.rs index b1a5d649a96..7cfb3117fb2 100644 --- a/iroh-net/src/dns.rs +++ b/iroh-net/src/dns.rs @@ -176,95 +176,4 @@ pub(crate) mod tests { assert!(!res.is_empty()); dbg!(res); } - - pub(crate) mod dns_server { - use std::net::{Ipv4Addr, SocketAddr}; - - use anyhow::{ensure, Result}; - use futures::{future::BoxFuture, Future}; - use hickory_proto::{ - op::{header::MessageType, Message}, - serialize::binary::BinDecodable, - }; - use tokio::{net::UdpSocket, task::JoinHandle}; - use tokio_util::sync::CancellationToken; - use tracing::{debug, warn}; - - pub trait Resolver: Send + Sync + 'static { - fn resolve( - &self, - query: &Message, - reply: &mut Message, - ) -> impl Future> + Send; - } - - pub type ResolveCallback = Box< - dyn Fn(&Message, &mut Message) -> BoxFuture<'static, Result<()>> - + Send - + Sync - + 'static, - >; - impl Resolver for ResolveCallback { - fn resolve( - &self, - query: &Message, - reply: &mut Message, - ) -> impl Future> + Send { - (&self)(query, reply) - } - } - - pub async fn spawn( - resolver: impl Resolver, - cancel: CancellationToken, - ) -> Result<(SocketAddr, JoinHandle>)> { - let bind_addr = SocketAddr::from((Ipv4Addr::LOCALHOST, 0)); - let socket = UdpSocket::bind(bind_addr).await?; - let bound_addr = socket.local_addr()?; - let s = TestDnsServer { - socket, - cancel, - resolver, - }; - let join_handle = tokio::task::spawn(async move { s.run().await }); - Ok((bound_addr, join_handle)) - } - - struct TestDnsServer { - resolver: R, - socket: UdpSocket, - cancel: CancellationToken, - } - - impl TestDnsServer { - async fn run(self) -> Result<()> { - let mut buf = [0; 1450]; - loop { - tokio::select! { - _ = self.cancel.cancelled() => break, - res = self.socket.recv_from(&mut buf) => { - let (len, from) = res?; - if let Err(err) = self.handle_datagram(from, &buf[..len]).await { - warn!(?err, %from, "failed to handle incoming datagram"); - } - } - }; - } - Ok(()) - } - - async fn handle_datagram(&self, from: SocketAddr, buf: &[u8]) -> Result<()> { - let packet = Message::from_bytes(buf)?; - debug!(queries = ?packet.queries(), %from, "received query"); - let mut reply = packet.clone(); - reply.set_message_type(MessageType::Response); - self.resolver.resolve(&packet, &mut reply).await?; - debug!(?reply, %from, "send reply"); - let buf = reply.to_vec()?; - let len = self.socket.send_to(&buf, from).await?; - ensure!(len == buf.len(), "failed to send complete packet"); - Ok(()) - } - } - } } diff --git a/iroh-net/src/test_utils.rs b/iroh-net/src/test_utils.rs index 6292349a2b8..d7e3b5619a2 100644 --- a/iroh-net/src/test_utils.rs +++ b/iroh-net/src/test_utils.rs @@ -62,3 +62,91 @@ pub(crate) async fn run_relay_server() -> Result<(RelayMap, RelayUrl, CleanupDro Ok((m, url, CleanupDropGuard(tx))) } + +pub mod dns_server { + use std::net::{Ipv4Addr, SocketAddr}; + + use anyhow::{ensure, Result}; + use futures::{future::BoxFuture, Future}; + use hickory_proto::{ + op::{header::MessageType, Message}, + serialize::binary::BinDecodable, + }; + use tokio::{net::UdpSocket, task::JoinHandle}; + use tokio_util::sync::CancellationToken; + use tracing::{debug, warn}; + + pub trait Resolver: Send + Sync + 'static { + fn resolve( + &self, + query: &Message, + reply: &mut Message, + ) -> impl Future> + Send; + } + + pub type ResolveCallback = Box< + dyn Fn(&Message, &mut Message) -> BoxFuture<'static, Result<()>> + Send + Sync + 'static, + >; + impl Resolver for ResolveCallback { + fn resolve( + &self, + query: &Message, + reply: &mut Message, + ) -> impl Future> + Send { + (&self)(query, reply) + } + } + + pub async fn run_dns_server( + resolver: impl Resolver, + cancel: CancellationToken, + ) -> Result<(SocketAddr, JoinHandle>)> { + let bind_addr = SocketAddr::from((Ipv4Addr::LOCALHOST, 0)); + let socket = UdpSocket::bind(bind_addr).await?; + let bound_addr = socket.local_addr()?; + let s = TestDnsServer { + socket, + cancel, + resolver, + }; + let join_handle = tokio::task::spawn(async move { s.run().await }); + Ok((bound_addr, join_handle)) + } + + struct TestDnsServer { + resolver: R, + socket: UdpSocket, + cancel: CancellationToken, + } + + impl TestDnsServer { + async fn run(self) -> Result<()> { + let mut buf = [0; 1450]; + loop { + tokio::select! { + _ = self.cancel.cancelled() => break, + res = self.socket.recv_from(&mut buf) => { + let (len, from) = res?; + if let Err(err) = self.handle_datagram(from, &buf[..len]).await { + warn!(?err, %from, "failed to handle incoming datagram"); + } + } + }; + } + Ok(()) + } + + async fn handle_datagram(&self, from: SocketAddr, buf: &[u8]) -> Result<()> { + let packet = Message::from_bytes(buf)?; + debug!(queries = ?packet.queries(), %from, "received query"); + let mut reply = packet.clone(); + reply.set_message_type(MessageType::Response); + self.resolver.resolve(&packet, &mut reply).await?; + debug!(?reply, %from, "send reply"); + let buf = reply.to_vec()?; + let len = self.socket.send_to(&buf, from).await?; + ensure!(len == buf.len(), "failed to send complete packet"); + Ok(()) + } + } +} From 180a651b84492e1f13cc9413ccf81149ade8e765 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 17:01:02 +0100 Subject: [PATCH 45/66] chore: clippy --- iroh-net/src/magic_endpoint.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs index 3cacda0bc74..445f6702157 100644 --- a/iroh-net/src/magic_endpoint.rs +++ b/iroh-net/src/magic_endpoint.rs @@ -522,7 +522,7 @@ impl MagicEndpoint { /// Get a reference to the DNS resolver used in this [`MagicEndpoint`]. pub fn dns_resolver(&self) -> &DnsResolver { - &self.msock.dns_resolver() + self.msock.dns_resolver() } /// Close the QUIC endpoint and the magic socket. From 877de7e80eb6c2b3c70793ebf2e0096a0eb45610 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 17:22:39 +0100 Subject: [PATCH 46/66] chore: clippy --- iroh-net/src/test_utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/test_utils.rs b/iroh-net/src/test_utils.rs index d7e3b5619a2..207a76299b6 100644 --- a/iroh-net/src/test_utils.rs +++ b/iroh-net/src/test_utils.rs @@ -93,7 +93,7 @@ pub mod dns_server { query: &Message, reply: &mut Message, ) -> impl Future> + Send { - (&self)(query, reply) + (self)(query, reply) } } From 5d9df01d653f4affbb28f0a1f6e06e196e61e58e Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 25 Mar 2024 22:38:19 +0100 Subject: [PATCH 47/66] tests: improve dns discovery tests --- iroh-net/src/discovery.rs | 38 +++++++++++++++++++++++++++----------- 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 6f19cc014d9..c6bc804da02 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -546,8 +546,8 @@ mod tests { /// publish to. The relay and DNS servers share their state. #[cfg(test)] mod test_dns_pkarr { - use std::future::Future; use std::net::SocketAddr; + use std::{future::Future, time::Duration}; use anyhow::Result; use hickory_resolver::{config::NameServerConfig, AsyncResolver, TokioAsyncResolver}; @@ -601,7 +601,7 @@ mod test_dns_pkarr { let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); let (nameserver, pkarr_url, _state, task) = - spawn_dns_and_pkarr(origin.clone(), cancel.clone()).await?; + run_dns_and_pkarr_servers(origin.clone(), cancel.clone()).await?; let secret_key = SecretKey::generate(); let node_id = secret_key.public(); @@ -636,19 +636,20 @@ mod test_dns_pkarr { let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); + let timeout = Duration::from_secs(1); + let (nameserver, pkarr_url, state, task) = - spawn_dns_and_pkarr(origin.clone(), cancel.clone()).await?; + run_dns_and_pkarr_servers(&origin, cancel.clone()).await?; + let (relay_map, _relay_url, _relay_guard) = run_relay_server().await?; - let (relay_map, _relay_url, _relay_guard) = run_relay_server().await.unwrap(); let ep1 = ep_with_discovery(relay_map.clone(), nameserver, &origin, &pkarr_url).await?; let ep2 = ep_with_discovery(relay_map, nameserver, &origin, &pkarr_url).await?; // wait until our shared state received the update from pkarr publishing - state.on_update().await; + state.on_node(&ep1.node_id(), timeout).await?; // we connect only by node id! - let ep2_node_id = ep2.node_id(); - let res = ep1.connect(ep2_node_id.into(), TEST_ALPN).await; + let res = ep2.connect(ep1.node_id().into(), TEST_ALPN).await; assert!(res.is_ok(), "connection established"); cancel.cancel(); task.await??; @@ -703,11 +704,11 @@ mod test_dns_pkarr { (node_info, signed_packet) } - async fn spawn_dns_and_pkarr( - origin: String, + async fn run_dns_and_pkarr_servers( + origin: impl ToString, cancel: CancellationToken, ) -> Result<(SocketAddr, Url, State, JoinHandle>)> { - let state = State::new(origin); + let state = State::new(origin.to_string()); let (nameserver, dns_task) = run_dns_server(state.clone(), cancel.clone()).await?; let (pkarr_url, pkarr_task) = run_pkarr_relay(state.clone(), cancel.clone()).await?; let join_handle = tokio::task::spawn(async move { @@ -720,12 +721,14 @@ mod test_dns_pkarr { mod state { use crate::NodeId; + use anyhow::{anyhow, Result}; use parking_lot::{Mutex, MutexGuard}; use pkarr::SignedPacket; use std::{ collections::{hash_map, HashMap}, ops::Deref, sync::Arc, + time::Duration, }; #[derive(Debug, Clone)] @@ -748,6 +751,20 @@ mod test_dns_pkarr { self.notify.notified() } + pub async fn on_node(&self, node: &NodeId, timeout: Duration) -> Result<()> { + let timeout = tokio::time::sleep(timeout); + tokio::pin!(timeout); + loop { + if self.get(node).is_some() { + return Ok(()); + } + tokio::select! { + _ = &mut timeout => return Err(anyhow!("timeout")), + _ = self.on_update() => {} + } + } + } + pub fn upsert(&self, signed_packet: SignedPacket) -> anyhow::Result { let node_id = NodeId::from_bytes(&signed_packet.public_key().to_bytes())?; let mut map = self.packets.lock(); @@ -771,7 +788,6 @@ mod test_dns_pkarr { Ok(updated) } pub fn get(&self, node_id: &NodeId) -> Option + '_> { - println!("GET {node_id}"); let map = self.packets.lock(); if map.contains_key(node_id) { let guard = MutexGuard::map(map, |state| state.get_mut(node_id).unwrap()); From 428f811c8566827dadeb10d82ad3471942600e30 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 26 Mar 2024 12:23:03 +0100 Subject: [PATCH 48/66] tests: increase timeout because windows is slow --- iroh-net/src/discovery.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index c6bc804da02..b7d4c5ac1ff 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -636,7 +636,7 @@ mod test_dns_pkarr { let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); - let timeout = Duration::from_secs(1); + let timeout = Duration::from_secs(2); let (nameserver, pkarr_url, state, task) = run_dns_and_pkarr_servers(&origin, cancel.clone()).await?; From a21eadf58757af917874fb2637a5db30fa2baeb7 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 26 Mar 2024 13:02:34 +0100 Subject: [PATCH 49/66] remove debug leftovers --- iroh-net/src/discovery/pkarr_publish.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index e39e37b1054..f1d03ec9052 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -73,12 +73,6 @@ impl Publisher { } let _ = self.last_published.write().insert(info.clone()); let signed_packet = info.to_pkarr_signed_packet(&self.secret_key, self.ttl)?; - println!( - "PUBLISH!!! self {} sp pk {}, nid from sp {}", - self.secret_key.public(), - signed_packet.public_key(), - crate::NodeId::from_bytes(signed_packet.public_key().as_bytes()).unwrap() - ); self.pkarr_client.publish(&signed_packet).await?; Ok(()) } From c3252b6417600689b36417c88ca1eec3ab6ac760 Mon Sep 17 00:00:00 2001 From: Franz Heinzmann Date: Thu, 28 Mar 2024 11:22:14 +0100 Subject: [PATCH 50/66] Update iroh-net/src/discovery/dns.rs Co-authored-by: Kasey --- iroh-net/src/discovery/dns.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs index ada51944d57..4a96de97cfa 100644 --- a/iroh-net/src/discovery/dns.rs +++ b/iroh-net/src/discovery/dns.rs @@ -18,8 +18,8 @@ pub const N0_TESTDNS_NODE_ORIGIN: &str = "testdns.iroh.link"; /// Node information is resolved via an _iroh_node.z32encodednodeid TXT record. /// /// The content of this record is expected to be a DNS attribute string, with a required -/// `node=` attribute containing the base32 encoded node id and a derp_url attribute containing the -/// node's home Derp server. +/// `node=` attribute containing the base32 encoded node id and a relay_url attribute containing the +/// node's home iroh-relay server. /// /// The discovery has to be configured with a `node_origin`, which is the domain name under which /// lookups for nodes will be made. From e43d0085f556dca755cedb00d079658b4218d736 Mon Sep 17 00:00:00 2001 From: Franz Heinzmann Date: Mon, 8 Apr 2024 16:26:38 +0200 Subject: [PATCH 51/66] refactor(dns): use one DNS record per attribute (#2150) Adapts #2045 to match #2149 * change the DNS and pkarr encoding for the NodeInfo to use one DNS record for each attribue * remove the node= attribute as the node id is contained in the DNS name ## Description ## Notes & open questions ## Change checklist - [ ] Self-review. - [ ] Documentation updates if relevant. - [ ] Tests if relevant. --- iroh-net/src/discovery.rs | 5 +- iroh-net/src/discovery/dns.rs | 41 +-- iroh-net/src/discovery/pkarr_publish.rs | 11 +- iroh-net/src/dns/node_info.rs | 338 ++++++++++++++---------- 4 files changed, 229 insertions(+), 166 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index b7d4c5ac1ff..3d8a8348462 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -817,8 +817,9 @@ mod test_dns_pkarr { continue; }; let node_info = NodeInfo::from_pkarr_signed_packet(packet)?; - let record = node_info.to_hickory_record(&this.origin, TTL)?; - reply.add_answer(record); + for record in node_info.to_hickory_records(&this.origin, TTL)? { + reply.add_answer(record); + } } Ok(()) } diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs index 4a96de97cfa..31da1885fe8 100644 --- a/iroh-net/src/discovery/dns.rs +++ b/iroh-net/src/discovery/dns.rs @@ -12,31 +12,38 @@ use crate::dns; /// The n0 testing DNS node origin pub const N0_TESTDNS_NODE_ORIGIN: &str = "testdns.iroh.link"; -/// DNS node discovery. +/// DNS node discovery /// -/// The DNS discovery looks up node addressing information over the Domain Name System. -/// Node information is resolved via an _iroh_node.z32encodednodeid TXT record. +/// When asked to resolve a [`NodeId`], this service performs a lookup in the Domain Name System (DNS). /// -/// The content of this record is expected to be a DNS attribute string, with a required -/// `node=` attribute containing the base32 encoded node id and a relay_url attribute containing the -/// node's home iroh-relay server. +/// It uses the [`MagicEndpoint`]'s DNS resolver to query for `TXT` records under the domain +/// `_iroh..`: /// -/// The discovery has to be configured with a `node_origin`, which is the domain name under which -/// lookups for nodes will be made. -/// With a origin of mydns.example, a node info record would be searched at -/// _iroh_node.z32encodednodeid.mydns.example TXT +/// * `_iroh`: is the record name +/// * `` is the [`NodeId`] encoded in [`z-base-32`] format +/// * `` is the node origin domain as set in [`DnsDiscovery::new`]. +/// +/// Each TXT record returned from the query is expected to contain a string in the format `=`. +/// If a TXT record contains multiple character strings, they are concatenated first. +/// The supported attributes are: +/// * `relay=`: The URL of the home relay server of the node +/// +/// The DNS resolver defaults to using the nameservers configured on the host system, but can be changed +/// with [`crate::magic_endpoint::MagicEndpointBuilder::dns_resolver`]. +/// +/// [z-base-32]: https://philzimmermann.com/docs/human-oriented-base-32-encoding.txt #[derive(Debug)] pub struct DnsDiscovery { - node_origin: String, + origin_domain: String, } impl DnsDiscovery { - /// Create a new DNS discovery with `node_origin` appended to all lookups. - pub fn new(node_origin: String) -> Self { - Self { node_origin } + /// Create a new DNS discovery. + pub fn new(origin_domain: String) -> Self { + Self { origin_domain } } - /// Create a new DNS discovery which uses the n0 testdns origin. + /// Create a new DNS discovery which uses the [`N0_TESTDNS_NODE_ORIGIN`] origin domain. pub fn n0_testdns() -> Self { Self::new(N0_TESTDNS_NODE_ORIGIN.to_string()) } @@ -51,9 +58,9 @@ impl Discovery for DnsDiscovery { let resolver = ep.dns_resolver().clone(); let fut = async move { let node_addr = - dns::node_info::lookup_by_id(&resolver, &node_id, &self.node_origin).await?; + dns::node_info::lookup_by_id(&resolver, &node_id, &self.origin_domain).await?; Ok(DiscoveryItem { - provenance: "iroh-dns", + provenance: "dns", last_updated: None, addr_info: node_addr.info, }) diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index f1d03ec9052..d7e4344c4f7 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -2,14 +2,10 @@ //! //! This service only implements the [`Discovery::publish`] method and does not provide discovery. //! It encodes the node information into a DNS packet in the format resolvable by the -//! [`super::dns::DnsDiscovery`], which means a single _iroh_node TXT record, under the z32 encoded -//! node id as origin domain. +//! [`super::dns::DnsDiscovery`]. //! //! [pkarr]: https://pkarr.org -// TODO: Decide what to do with this module once publishing over relays land. Either remove, or -// leave in the repo but do not enable it by default in the iroh node. - use std::sync::Arc; use anyhow::Result; @@ -23,7 +19,7 @@ use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, Addr /// The n0 testing pkarr relay pub const N0_TESTDNS_PKARR_RELAY: &str = "https://testdns.iroh.link/pkarr"; -/// Default TTL for the _iroh_node TXT record in the pkarr signed packet +/// Default TTL for the _iroh TXT record in the pkarr signed packet const DEFAULT_PKARR_TTL: u32 = 30; /// Publish node info to a pkarr relay. @@ -66,8 +62,9 @@ impl Publisher { pub async fn publish_addr_info(&self, info: &AddrInfo) -> Result<()> { let info = NodeInfo::new( self.secret_key.public(), - info.relay_url.clone().map(Url::from), + info.relay_url.clone().map(Into::into), ); + // only republish if the [`NodeInfo`] changed if self.last_published.read().as_ref() == Some(&info) { return Ok(()); } diff --git a/iroh-net/src/dns/node_info.rs b/iroh-net/src/dns/node_info.rs index dbb3cf768d2..3e7d9ae93fe 100644 --- a/iroh-net/src/dns/node_info.rs +++ b/iroh-net/src/dns/node_info.rs @@ -1,25 +1,32 @@ //! This module contains functions and structs to lookup node information from DNS //! and to encode node information in Pkarr signed packets. -use std::{collections::HashMap, fmt, str::FromStr}; +use std::{collections::BTreeMap, str::FromStr}; -use anyhow::{anyhow, bail, Result}; +use anyhow::{anyhow, ensure, Result}; use hickory_proto::error::ProtoError; use hickory_resolver::{Name, TokioAsyncResolver}; use url::Url; use crate::{key::SecretKey, AddrInfo, NodeAddr, NodeId}; -const ATTR_RELAY: &str = "relay"; -const ATTR_NODE_ID: &str = "node"; +/// The DNS name for the iroh TXT record +pub const IROH_TXT_NAME: &str = "_iroh"; -/// The label for the node info TXT record -pub const IROH_NODE_TXT_LABEL: &str = "_iroh_node"; +/// The attributes supported for `_iroh` DNS records +#[derive( + Debug, strum::Display, strum::AsRefStr, strum::EnumString, Hash, Eq, PartialEq, Ord, PartialOrd, +)] +#[strum(serialize_all = "kebab-case")] +pub enum IrohAttr { + /// `relay`: URL of home relay + Relay, +} /// Lookup node info by domain name /// -/// The domain name must either contain an _iroh_node TXT record or be a CNAME record that leads to -/// an _iroh_node TXT record. +/// The domain name must either contain an _iroh TXT record or be a CNAME record that leads to +/// an _iroh TXT record. pub async fn lookup_by_domain(resolver: &TokioAsyncResolver, domain: &str) -> Result { let name = Name::from_str(domain)?; let info = lookup_node_info(resolver, name).await?; @@ -36,20 +43,17 @@ pub async fn lookup_by_id( lookup_by_domain(resolver, &domain).await } -pub(crate) async fn lookup_node_info( - resolver: &TokioAsyncResolver, - name: Name, -) -> Result { - let name = ensure_iroh_node_txt_label(name)?; +async fn lookup_node_info(resolver: &TokioAsyncResolver, name: Name) -> Result { + let name = ensure_iroh_txt_label(name)?; let lookup = resolver.txt_lookup(name).await?; - NodeInfo::from_hickory_lookup(lookup.as_lookup()) + NodeInfo::from_hickory_records(lookup.as_lookup().records()) } -fn ensure_iroh_node_txt_label(name: Name) -> Result { - if name.iter().next() == Some(IROH_NODE_TXT_LABEL.as_bytes()) { +fn ensure_iroh_txt_label(name: Name) -> Result { + if name.iter().next() == Some(IROH_TXT_NAME.as_bytes()) { Ok(name) } else { - Name::parse(IROH_NODE_TXT_LABEL, Some(&name)) + Name::parse(IROH_TXT_NAME, Some(&name)) } } @@ -70,7 +74,7 @@ pub fn from_z32(s: &str) -> Result { Ok(node_id) } -/// Node info contained in a DNS _iroh_node TXT record. +/// Node info contained in a DNS _iroh TXT record. #[derive(derive_more::Debug, Clone, Eq, PartialEq)] pub struct NodeInfo { /// The node id @@ -80,6 +84,36 @@ pub struct NodeInfo { pub relay_url: Option, } +impl From for NodeInfo { + fn from(attrs: NodeAttrs) -> Self { + (&attrs).into() + } +} + +impl From<&NodeAttrs> for NodeInfo { + fn from(attrs: &NodeAttrs) -> Self { + let node_id = attrs.node_id(); + let attrs = attrs.attrs(); + let relay_url = attrs + .get(&IrohAttr::Relay) + .into_iter() + .flatten() + .next() + .and_then(|s| Url::parse(s).ok()); + Self { node_id, relay_url } + } +} + +impl From<&NodeInfo> for NodeAttrs { + fn from(info: &NodeInfo) -> Self { + let mut attrs = vec![]; + if let Some(relay_url) = &info.relay_url { + attrs.push((IrohAttr::Relay, relay_url.to_string())); + } + Self::from_parts(info.node_id, attrs.into_iter()) + } +} + impl From for NodeAddr { fn from(value: NodeInfo) -> Self { NodeAddr { @@ -104,61 +138,20 @@ impl NodeInfo { Self { node_id, relay_url } } - /// Convert this node info into a DNS attribute string. - /// - /// It will look like this: - /// `node=b32encodednodeid relay=https://myrelay.example` - pub fn to_attribute_string(&self) -> String { - let mut attrs = vec![]; - attrs.push(fmt_attr(ATTR_NODE_ID, self.node_id)); - if let Some(relay) = &self.relay_url { - attrs.push(fmt_attr(ATTR_RELAY, relay)); - } - attrs.join(" ") - } - - /// Try to parse a [`NodeInfo`] from the lookup result of our DNS resolver. - pub fn from_hickory_lookup(lookup: &hickory_resolver::lookup::Lookup) -> Result { - Self::from_hickory_records(lookup.records()) + fn to_attrs(&self) -> NodeAttrs { + self.into() } /// Try to parse a [`NodeInfo`] from a set of DNS records. pub fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { - use hickory_proto::rr; - let (node_id, txt) = records - .iter() - .find_map(|rr| match rr.data() { - Some(rr::RData::TXT(txt)) => { - parse_hickory_node_info_name(rr.name()).map(|node_id| (node_id, txt)) - } - _ => None, - }) - .ok_or_else(|| anyhow!("no TXT record with name _iroh_node.b32encodedpubkey found"))?; - let node_info = Self::parse_from_attributes(&txt.to_string())?; - if node_info.node_id != node_id { - bail!("node id mismatch between record name and TXT value"); - } - Ok(node_info) + let attrs = NodeAttrs::from_hickory_records(records)?; + Ok(attrs.into()) } - /// Parse the [`NodeInfo`] from an attribute string. - /// - /// See [Self::to_attribute_string] for the expected format. - pub fn parse_from_attributes(attrs: &str) -> Result { - let attrs = parse_attrs(attrs); - let Some(node) = attrs.get(ATTR_NODE_ID) else { - bail!("missing required node attribute"); - }; - if node.len() != 1 { - bail!("more than one node attribute is not allowed"); - } - let node_id = NodeId::from_str(node[0])?; - let relay_url: Option = attrs - .get(ATTR_RELAY) - .into_iter() - .flatten() - .find_map(|x| Url::parse(x).ok()); - Ok(Self { node_id, relay_url }) + /// Try to parse a [`NodeInfo`] from a [`pkarr::SignedPacket`]. + pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { + let attrs = NodeAttrs::from_pkarr_signed_packet(packet)?; + Ok(attrs.into()) } /// Create a [`pkarr::SignedPacket`] by constructing a DNS packet and @@ -168,72 +161,18 @@ impl NodeInfo { secret_key: &SecretKey, ttl: u32, ) -> Result { - let packet = self.to_pkarr_dns_packet(ttl)?; - let keypair = pkarr::Keypair::from_secret_key(&secret_key.to_bytes()); - let signed_packet = pkarr::SignedPacket::from_packet(&keypair, &packet)?; - Ok(signed_packet) - } - - fn to_pkarr_dns_packet(&self, ttl: u32) -> Result> { - use pkarr::dns::{self, rdata}; - let name = dns::Name::new(IROH_NODE_TXT_LABEL)?.into_owned(); - let rdata = { - let value = self.to_attribute_string(); - let txt = rdata::TXT::new().with_string(&value)?.into_owned(); - rdata::RData::TXT(txt) - }; - - let mut packet = dns::Packet::new_reply(0); - packet - .answers - .push(dns::ResourceRecord::new(name, dns::CLASS::IN, ttl, rdata)); - Ok(packet) + self.to_attrs().to_pkarr_signed_packet(secret_key, ttl) } /// Convert into a [`hickory_proto::rr::Record`] DNS record. - pub fn to_hickory_record(&self, origin: &str, ttl: u32) -> Result { - use hickory_proto::rr; - let name = format!( - "{}.{}.{}", - IROH_NODE_TXT_LABEL, - to_z32(&self.node_id), - origin - ); - let name = rr::Name::from_utf8(name)?; - let value = self.to_attribute_string(); - let txt = rr::rdata::TXT::new(vec![value]); - let rdata = rr::RData::TXT(txt); - let record = rr::Record::from_rdata(name, ttl, rdata); - Ok(record) - } - - /// Try to parse a [`NodeInfo`] from a [`pkarr::SignedPacket`]. - pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { - use pkarr::dns::{self, rdata::RData}; - let pubkey = packet.public_key(); - let pubkey_z32 = pubkey.to_z32(); - let node_id = NodeId::from(*pubkey.verifying_key()); - let zone = dns::Name::new(&pubkey_z32)?; - let inner = packet.packet(); - let txt_record = inner - .answers - .iter() - .find_map(|rr| match &rr.rdata { - RData::TXT(txt) => match rr.name.without(&zone) { - Some(name) if name.to_string() == IROH_NODE_TXT_LABEL => Some(txt), - Some(_) | None => None, - }, - _ => None, - }) - .ok_or_else(|| anyhow!("missing _iroh_node txt record"))?; - - let txt_record = txt_record.to_owned(); - let txt = String::try_from(txt_record)?; - let info = Self::parse_from_attributes(&txt)?; - if info.node_id != node_id { - bail!("node id mismatch between record name and TXT value"); - } - Ok(info) + pub fn to_hickory_records( + &self, + origin: &str, + ttl: u32, + ) -> Result + 'static> { + let attrs = self.to_attrs(); + let records = attrs.to_hickory_records(origin, ttl)?; + Ok(records.collect::>().into_iter()) } } @@ -243,7 +182,7 @@ pub(crate) fn parse_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Op } let mut labels = name.iter(); let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; - if label != IROH_NODE_TXT_LABEL { + if label != IROH_TXT_NAME { return None; } let label = std::str::from_utf8(labels.next().expect("num_labels checked")).ok()?; @@ -251,17 +190,136 @@ pub(crate) fn parse_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Op Some(node_id) } -fn fmt_attr(label: &str, value: impl fmt::Display) -> String { - format!("{label}={value}") +struct NodeAttrs { + node_id: NodeId, + attrs: BTreeMap>, } -fn parse_attrs<'a>(s: &'a str) -> HashMap<&'a str, Vec<&'a str>> { - let mut map: HashMap<&'a str, Vec<&'a str>> = HashMap::new(); - let parts = s.split(' '); - for part in parts { - if let Some((name, value)) = part.split_once('=') { - map.entry(name).or_default().push(value); +impl NodeAttrs { + fn from_parts(node_id: NodeId, pairs: impl Iterator) -> Self { + let mut attrs: BTreeMap> = BTreeMap::new(); + for (k, v) in pairs { + attrs.entry(k).or_default().push(v); } + Self { attrs, node_id } + } + + fn from_strings(node_id: NodeId, strings: impl Iterator) -> Result { + let mut attrs: BTreeMap> = BTreeMap::new(); + for s in strings { + let mut parts = s.split('='); + let (Some(key), Some(value)) = (parts.next(), parts.next()) else { + continue; + }; + let Ok(attr) = IrohAttr::from_str(key) else { + continue; + }; + attrs.entry(attr).or_default().push(value.to_string()); + } + Ok(Self { attrs, node_id }) + } + + fn attrs(&self) -> &BTreeMap> { + &self.attrs + } + + fn node_id(&self) -> NodeId { + self.node_id + } + + /// Try to parse a [`NodeInfo`] from a [`pkarr::SignedPacket`]. + fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { + use pkarr::dns::{self, rdata::RData}; + let pubkey = packet.public_key(); + let pubkey_z32 = pubkey.to_z32(); + let node_id = NodeId::from(*pubkey.verifying_key()); + let zone = dns::Name::new(&pubkey_z32)?; + let inner = packet.packet(); + let txt_data = inner.answers.iter().filter_map(|rr| match &rr.rdata { + RData::TXT(txt) => match rr.name.without(&zone) { + Some(name) if name.to_string() == IROH_TXT_NAME => Some(txt), + Some(_) | None => None, + }, + _ => None, + }); + + let txt_strs = txt_data.filter_map(|s| String::try_from(s.clone()).ok()); + Self::from_strings(node_id, txt_strs) + } + + /// Try to parse a [`NodeInfo`] from a set of DNS records. + fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { + use hickory_proto::rr; + let mut records = records.iter().filter_map(|rr| match rr.data() { + Some(rr::RData::TXT(txt)) => { + parse_hickory_node_info_name(rr.name()).map(|node_id| (node_id, txt)) + } + _ => None, + }); + let (node_id, first) = records.next().ok_or_else(|| { + anyhow!("invalid DNS answer: no TXT record with name _iroh.z32encodedpubkey found") + })?; + ensure!( + &records.all(|(n, _)| n == node_id), + "invalid DNS answer: all _iroh txt records must belong to the same node domain" + ); + let records = records.map(|(_, txt)| txt).chain(Some(first)); + let strings = records.map(ToString::to_string); + Self::from_strings(node_id, strings) + } + + fn to_txt_strings(&self) -> impl Iterator + '_ { + self.attrs + .iter() + .flat_map(move |(k, vs)| vs.iter().map(move |v| format!("{k}={v}"))) + } + + /// Convert into list of [`hickory_proto::rr::Record`]. + fn to_hickory_records( + &self, + origin: &str, + ttl: u32, + ) -> Result + '_> { + use hickory_proto::rr; + let name = format!("{}.{}.{}", IROH_TXT_NAME, to_z32(&self.node_id), origin); + let name = rr::Name::from_utf8(name)?; + let records = self.to_txt_strings().map(move |s| { + let txt = rr::rdata::TXT::new(vec![s]); + let rdata = rr::RData::TXT(txt); + rr::Record::from_rdata(name.clone(), ttl, rdata) + }); + Ok(records) + } + + /// Create a [`pkarr::SignedPacket`] by constructing a DNS packet and + /// signing it with a [`SecretKey`]. + fn to_pkarr_signed_packet( + &self, + secret_key: &SecretKey, + ttl: u32, + ) -> Result { + let packet = self.to_pkarr_dns_packet(ttl)?; + let keypair = pkarr::Keypair::from_secret_key(&secret_key.to_bytes()); + let signed_packet = pkarr::SignedPacket::from_packet(&keypair, &packet)?; + Ok(signed_packet) + } + + fn to_pkarr_dns_packet(&self, ttl: u32) -> Result> { + use pkarr::dns::{self, rdata}; + let name = dns::Name::new(IROH_TXT_NAME)?.into_owned(); + + let mut packet = dns::Packet::new_reply(0); + for s in self.to_txt_strings() { + let mut txt = rdata::TXT::new(); + txt.add_string(&s)?; + let rdata = rdata::RData::TXT(txt.into_owned()); + packet.answers.push(dns::ResourceRecord::new( + name.clone(), + dns::CLASS::IN, + ttl, + rdata, + )); + } + Ok(packet) } - map } From ed0b6a4c0060eef7b35a2dd321711679db870cd4 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 8 Apr 2024 22:44:10 +0200 Subject: [PATCH 52/66] change: move from testdns.iroh.link to dns.iroh.link --- iroh-net/src/discovery.rs | 36 +++++++++++++++++++++++++ iroh-net/src/discovery/dns.rs | 8 +++--- iroh-net/src/discovery/pkarr_publish.rs | 8 +++--- iroh/src/node/builder.rs | 4 +-- 4 files changed, 46 insertions(+), 10 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 3d8a8348462..e6eb02931cf 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -656,6 +656,27 @@ mod test_dns_pkarr { Ok(()) } + #[tokio::test] + async fn publish_discover_online_defaults() -> Result<()> { + let _logging_guard = iroh_test::logging::setup(); + + let cancel = CancellationToken::new(); + + let ep1 = ep_with_n0_defaults().await?; + let ep2 = ep_with_n0_defaults().await?; + + // wait until ep1 announced + let _addr = ep1.my_addr().await?; + // wait more for publish to finish and DNS server to handle things + tokio::time::sleep(Duration::from_secs(2)).await; + + // we connect only by node id! + let res = ep2.connect(ep1.node_id().into(), TEST_ALPN).await; + assert!(res.is_ok(), "connection established"); + cancel.cancel(); + Ok(()) + } + async fn ep_with_discovery( relay_map: RelayMap, nameserver: SocketAddr, @@ -682,6 +703,21 @@ mod test_dns_pkarr { Ok(ep) } + async fn ep_with_n0_defaults() -> Result { + let secret_key = SecretKey::generate(); + let discovery = ConcurrentDiscovery::new(vec![ + Box::new(DnsDiscovery::n0_dns()), + Box::new(pkarr_publish::Publisher::n0_dns(secret_key.clone())), + ]); + let ep = MagicEndpoint::builder() + .secret_key(secret_key) + .alpns(vec![TEST_ALPN.to_vec()]) + .discovery(Box::new(discovery)) + .bind(0) + .await?; + Ok(ep) + } + fn dns_resolver(nameserver: SocketAddr) -> Result { let mut config = hickory_resolver::config::ResolverConfig::new(); let nameserver_config = diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs index 31da1885fe8..befb00cc48f 100644 --- a/iroh-net/src/discovery/dns.rs +++ b/iroh-net/src/discovery/dns.rs @@ -10,7 +10,7 @@ use futures::{future::FutureExt, stream::BoxStream, StreamExt}; use crate::dns; /// The n0 testing DNS node origin -pub const N0_TESTDNS_NODE_ORIGIN: &str = "testdns.iroh.link"; +pub const N0_DNS_NODE_ORIGIN: &str = "dns.iroh.link"; /// DNS node discovery /// @@ -43,9 +43,9 @@ impl DnsDiscovery { Self { origin_domain } } - /// Create a new DNS discovery which uses the [`N0_TESTDNS_NODE_ORIGIN`] origin domain. - pub fn n0_testdns() -> Self { - Self::new(N0_TESTDNS_NODE_ORIGIN.to_string()) + /// Create a new DNS discovery which uses the [`N0_DNS_NODE_ORIGIN`] origin domain. + pub fn n0_dns() -> Self { + Self::new(N0_DNS_NODE_ORIGIN.to_string()) } } diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index d7e4344c4f7..5a510966551 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -17,7 +17,7 @@ use url::Url; use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, AddrInfo}; /// The n0 testing pkarr relay -pub const N0_TESTDNS_PKARR_RELAY: &str = "https://testdns.iroh.link/pkarr"; +pub const N0_DNS_PKARR_RELAY: &str = "https://dns.iroh.link/pkarr"; /// Default TTL for the _iroh TXT record in the pkarr signed packet const DEFAULT_PKARR_TTL: u32 = 30; @@ -45,9 +45,9 @@ impl Publisher { } } - /// Create a config that publishes to the n0 testdns server. - pub fn n0_testdns(secret_key: SecretKey) -> Self { - let pkarr_relay: Url = N0_TESTDNS_PKARR_RELAY.parse().expect("url is valid"); + /// Create a config that publishes to the n0 dns server. + pub fn n0_dns(secret_key: SecretKey) -> Self { + let pkarr_relay: Url = N0_DNS_PKARR_RELAY.parse().expect("url is valid"); Self::new(secret_key, pkarr_relay) } diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs index 6dd6dcfbc14..8602803888b 100644 --- a/iroh/src/node/builder.rs +++ b/iroh/src/node/builder.rs @@ -324,10 +324,10 @@ where NodeDiscoveryConfig::Default => { let discovery = ConcurrentDiscovery::new(vec![ // Enable DNS discovery by default - Box::new(DnsDiscovery::n0_testdns()), + Box::new(DnsDiscovery::n0_dns()), // Enable pkarr publishing by default // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. - Box::new(pkarr_publish::Publisher::n0_testdns( + Box::new(pkarr_publish::Publisher::n0_dns( self.secret_key.clone(), )), ]); From 1ee96bafd5d0a646fa88c95be86c98db51281103 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 8 Apr 2024 22:49:03 +0200 Subject: [PATCH 53/66] refactor: renames for clarity --- iroh-net/src/discovery.rs | 18 ++++++++---------- iroh-net/src/discovery/pkarr_publish.rs | 6 +++--- iroh/src/node/builder.rs | 8 +++----- 3 files changed, 14 insertions(+), 18 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index e6eb02931cf..f017c389aa4 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -76,7 +76,7 @@ impl ConcurrentDiscovery { } /// Create a new [`ConcurrentDiscovery`]. - pub fn new(services: Vec>) -> Self { + pub fn from_services(services: Vec>) -> Self { Self { services } } @@ -481,7 +481,7 @@ mod tests { let ep2 = { let secret = SecretKey::generate(); let disco1 = disco_shared.create_lying_discovery(secret.public()); - let disco = ConcurrentDiscovery::new(vec![Box::new(disco1)]); + let disco = ConcurrentDiscovery::from_services(vec![Box::new(disco1)]); new_endpoint(secret, disco).await }; let ep1_addr = NodeAddr::new(ep1.node_id()); @@ -558,7 +558,7 @@ mod test_dns_pkarr { use url::Url; use crate::{ - discovery::pkarr_publish, + discovery::{dns::DnsDiscovery, ConcurrentDiscovery, pkarr_publish::PkarrPublisher}, dns::node_info::{lookup_by_id, parse_hickory_node_info_name, NodeInfo}, relay::{RelayMap, RelayMode}, test_utils::{ @@ -570,8 +570,6 @@ mod test_dns_pkarr { use self::{pkarr_relay::run_pkarr_relay, state::State}; - use super::{dns::DnsDiscovery, ConcurrentDiscovery}; - #[tokio::test] async fn dns_resolve() -> Result<()> { let _logging_guard = iroh_test::logging::setup(); @@ -605,7 +603,7 @@ mod test_dns_pkarr { let secret_key = SecretKey::generate(); let node_id = secret_key.public(); - let publisher = pkarr_publish::Publisher::new(secret_key, pkarr_url); + let publisher = PkarrPublisher::new(secret_key, pkarr_url); let addr_info = AddrInfo { relay_url: Some("https://relay.example".parse().unwrap()), @@ -685,9 +683,9 @@ mod test_dns_pkarr { ) -> Result { let secret_key = SecretKey::generate(); let resolver = dns_resolver(nameserver)?; - let discovery = ConcurrentDiscovery::new(vec![ + let discovery = ConcurrentDiscovery::from_services(vec![ Box::new(DnsDiscovery::new(node_origin.to_string())), - Box::new(pkarr_publish::Publisher::new( + Box::new(PkarrPublisher::new( secret_key.clone(), pkarr_relay.clone(), )), @@ -705,9 +703,9 @@ mod test_dns_pkarr { async fn ep_with_n0_defaults() -> Result { let secret_key = SecretKey::generate(); - let discovery = ConcurrentDiscovery::new(vec![ + let discovery = ConcurrentDiscovery::from_services(vec![ Box::new(DnsDiscovery::n0_dns()), - Box::new(pkarr_publish::Publisher::n0_dns(secret_key.clone())), + Box::new(PkarrPublisher::n0_dns(secret_key.clone())), ]); let ep = MagicEndpoint::builder() .secret_key(secret_key) diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index 5a510966551..f6966332183 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -24,7 +24,7 @@ const DEFAULT_PKARR_TTL: u32 = 30; /// Publish node info to a pkarr relay. #[derive(derive_more::Debug, Clone)] -pub struct Publisher { +pub struct PkarrPublisher { #[debug("SecretKey")] secret_key: SecretKey, #[debug("PkarrClient")] @@ -33,7 +33,7 @@ pub struct Publisher { ttl: u32, } -impl Publisher { +impl PkarrPublisher { /// Create a new config with a secret key and a pkarr relay URL. pub fn new(secret_key: SecretKey, pkarr_relay: Url) -> Self { let pkarr_client = PkarrRelayClient::new(pkarr_relay); @@ -75,7 +75,7 @@ impl Publisher { } } -impl Discovery for Publisher { +impl Discovery for PkarrPublisher { fn publish(&self, info: &AddrInfo) { let this = self.clone(); let info = info.clone(); diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs index 8602803888b..1cd3f74c275 100644 --- a/iroh/src/node/builder.rs +++ b/iroh/src/node/builder.rs @@ -16,7 +16,7 @@ use iroh_bytes::{ }; use iroh_gossip::net::{Gossip, GOSSIP_ALPN}; use iroh_net::{ - discovery::{dns::DnsDiscovery, pkarr_publish, ConcurrentDiscovery, Discovery}, + discovery::{dns::DnsDiscovery, pkarr_publish::PkarrPublisher, ConcurrentDiscovery, Discovery}, magic_endpoint::get_alpn, relay::RelayMode, util::AbortingJoinHandle, @@ -322,14 +322,12 @@ where NodeDiscoveryConfig::None => None, NodeDiscoveryConfig::Custom(discovery) => Some(discovery), NodeDiscoveryConfig::Default => { - let discovery = ConcurrentDiscovery::new(vec![ + let discovery = ConcurrentDiscovery::from_services(vec![ // Enable DNS discovery by default Box::new(DnsDiscovery::n0_dns()), // Enable pkarr publishing by default // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. - Box::new(pkarr_publish::Publisher::n0_dns( - self.secret_key.clone(), - )), + Box::new(PkarrPublisher::n0_dns(self.secret_key.clone())), ]); Some(Box::new(discovery)) } From 6ac89545cf26bf938492fbe9ff131f6c7e32503b Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 8 Apr 2024 22:57:22 +0200 Subject: [PATCH 54/66] chore: fmt --- iroh-net/src/discovery.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 3eb7af446b0..658b01f0c72 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -558,7 +558,7 @@ mod test_dns_pkarr { use url::Url; use crate::{ - discovery::{dns::DnsDiscovery, ConcurrentDiscovery, pkarr_publish::PkarrPublisher}, + discovery::{dns::DnsDiscovery, pkarr_publish::PkarrPublisher, ConcurrentDiscovery}, dns::node_info::{lookup_by_id, parse_hickory_node_info_name, NodeInfo}, relay::{RelayMap, RelayMode}, test_utils::{ @@ -685,10 +685,7 @@ mod test_dns_pkarr { let resolver = dns_resolver(nameserver)?; let discovery = ConcurrentDiscovery::from_services(vec![ Box::new(DnsDiscovery::new(node_origin.to_string())), - Box::new(PkarrPublisher::new( - secret_key.clone(), - pkarr_relay.clone(), - )), + Box::new(PkarrPublisher::new(secret_key.clone(), pkarr_relay.clone())), ]); let ep = MagicEndpoint::builder() .relay_mode(RelayMode::Custom(relay_map)) From 4524018b0d7820d4acb75934f62bb65bb956e595 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 4 Apr 2024 00:35:04 +0200 Subject: [PATCH 55/66] tests: cleanups and improvements --- iroh-net/src/discovery.rs | 114 ++++++++++++++++------------------ iroh-net/src/dns/node_info.rs | 8 ++- iroh-net/src/test_utils.rs | 28 +++++++-- 3 files changed, 83 insertions(+), 67 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 658b01f0c72..444c933a22d 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -543,26 +543,24 @@ mod tests { /// This module contains end-to-end tests for DNS node discovery. /// /// The tests run a minimal test DNS server to resolve against, and a minimal pkarr relay to -/// publish to. The relay and DNS servers share their state. +/// publish to. The DNS and pkarr servers share their state. #[cfg(test)] mod test_dns_pkarr { use std::net::SocketAddr; - use std::{future::Future, time::Duration}; + use std::time::Duration; use anyhow::Result; - use hickory_resolver::{config::NameServerConfig, AsyncResolver, TokioAsyncResolver}; use iroh_base::key::SecretKey; - use pkarr::SignedPacket; use tokio::task::JoinHandle; use tokio_util::sync::CancellationToken; use url::Url; use crate::{ discovery::{dns::DnsDiscovery, pkarr_publish::PkarrPublisher, ConcurrentDiscovery}, - dns::node_info::{lookup_by_id, parse_hickory_node_info_name, NodeInfo}, + dns::node_info::{lookup_by_id, NodeInfo}, relay::{RelayMap, RelayMode}, test_utils::{ - dns_server::{run_dns_server, Resolver}, + dns_server::{create_dns_resolver, run_dns_server}, run_relay_server, }, AddrInfo, MagicEndpoint, NodeAddr, @@ -573,16 +571,21 @@ mod test_dns_pkarr { #[tokio::test] async fn dns_resolve() -> Result<()> { let _logging_guard = iroh_test::logging::setup(); + let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); let state = State::new(origin.clone()); let (nameserver, dns_task) = run_dns_server(state.clone(), cancel.clone()).await?; - let node_secret = SecretKey::generate(); - let (node_info, signed_packet) = generate_node_info(&node_secret); + let secret_key = SecretKey::generate(); + let node_info = NodeInfo::new( + secret_key.public(), + Some("https://relay.example".parse().unwrap()), + ); + let signed_packet = node_info.to_pkarr_signed_packet(&secret_key, 30)?; state.upsert(signed_packet)?; - let resolver = dns_resolver(nameserver)?; + let resolver = create_dns_resolver(nameserver)?; let resolved = lookup_by_id(&resolver, &node_info.node_id, &origin).await?; assert_eq!(resolved, node_info.into()); @@ -596,22 +599,26 @@ mod test_dns_pkarr { async fn pkarr_publish_dns_resolve() -> Result<()> { let _logging_guard = iroh_test::logging::setup(); - let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); - let (nameserver, pkarr_url, _state, task) = + let cancel = CancellationToken::new(); + let timeout = Duration::from_secs(2); + + let (nameserver, pkarr_url, state, task) = run_dns_and_pkarr_servers(origin.clone(), cancel.clone()).await?; let secret_key = SecretKey::generate(); let node_id = secret_key.public(); - let publisher = PkarrPublisher::new(secret_key, pkarr_url); let addr_info = AddrInfo { relay_url: Some("https://relay.example".parse().unwrap()), ..Default::default() }; + let publisher = PkarrPublisher::new(secret_key, pkarr_url); publisher.publish_addr_info(&addr_info).await?; + // wait until our shared state received the update from pkarr publishing + state.on_node(&node_id, timeout).await?; - let resolver = dns_resolver(nameserver)?; + let resolver = create_dns_resolver(nameserver)?; let resolved = lookup_by_id(&resolver, &node_id, &origin).await?; let expected = NodeAddr { @@ -632,8 +639,8 @@ mod test_dns_pkarr { async fn pkarr_publish_dns_discover() -> Result<()> { let _logging_guard = iroh_test::logging::setup(); - let cancel = CancellationToken::new(); let origin = "testdns.example".to_string(); + let cancel = CancellationToken::new(); let timeout = Duration::from_secs(2); let (nameserver, pkarr_url, state, task) = @@ -682,7 +689,7 @@ mod test_dns_pkarr { pkarr_relay: &Url, ) -> Result { let secret_key = SecretKey::generate(); - let resolver = dns_resolver(nameserver)?; + let resolver = create_dns_resolver(nameserver)?; let discovery = ConcurrentDiscovery::from_services(vec![ Box::new(DnsDiscovery::new(node_origin.to_string())), Box::new(PkarrPublisher::new(secret_key.clone(), pkarr_relay.clone())), @@ -713,28 +720,6 @@ mod test_dns_pkarr { Ok(ep) } - fn dns_resolver(nameserver: SocketAddr) -> Result { - let mut config = hickory_resolver::config::ResolverConfig::new(); - let nameserver_config = - NameServerConfig::new(nameserver, hickory_resolver::config::Protocol::Udp); - config.add_name_server(nameserver_config); - let resolver = AsyncResolver::tokio(config, Default::default()); - Ok(resolver) - } - - fn generate_node_info(secret: &SecretKey) -> (NodeInfo, SignedPacket) { - let node_id = secret.public(); - let relay_url: Url = "https://relay.example".parse().expect("valid url"); - let node_info = NodeInfo { - node_id, - relay_url: Some(relay_url.clone()), - }; - let signed_packet = node_info - .to_pkarr_signed_packet(secret, 30) - .expect("valid packet"); - (node_info, signed_packet) - } - async fn run_dns_and_pkarr_servers( origin: impl ToString, cancel: CancellationToken, @@ -751,21 +736,25 @@ mod test_dns_pkarr { } mod state { - use crate::NodeId; - use anyhow::{anyhow, Result}; + use anyhow::{bail, Result}; use parking_lot::{Mutex, MutexGuard}; use pkarr::SignedPacket; use std::{ collections::{hash_map, HashMap}, + future::Future, ops::Deref, sync::Arc, time::Duration, }; + use crate::dns::node_info::{node_id_from_hickory_name, NodeInfo}; + use crate::test_utils::dns_server::QueryHandler; + use crate::NodeId; + #[derive(Debug, Clone)] pub struct State { packets: Arc>>, - pub origin: String, + origin: String, notify: Arc, } @@ -785,15 +774,13 @@ mod test_dns_pkarr { pub async fn on_node(&self, node: &NodeId, timeout: Duration) -> Result<()> { let timeout = tokio::time::sleep(timeout); tokio::pin!(timeout); - loop { - if self.get(node).is_some() { - return Ok(()); - } + while self.get(node).is_none() { tokio::select! { - _ = &mut timeout => return Err(anyhow!("timeout")), + _ = &mut timeout => bail!("timeout"), _ = self.on_update() => {} } } + Ok(()) } pub fn upsert(&self, signed_packet: SignedPacket) -> anyhow::Result { @@ -818,6 +805,8 @@ mod test_dns_pkarr { } Ok(updated) } + + /// Returns a mutex guard, do not hold over await points pub fn get(&self, node_id: &NodeId) -> Option + '_> { let map = self.packets.lock(); if map.contains_key(node_id) { @@ -827,34 +816,41 @@ mod test_dns_pkarr { None } } - } - } - impl Resolver for State { - fn resolve( - &self, - query: &hickory_proto::op::Message, - reply: &mut hickory_proto::op::Message, - ) -> impl Future> + Send { - const TTL: u32 = 30; - let this = self.clone(); - async move { + pub fn resolve_dns( + &self, + query: &hickory_proto::op::Message, + reply: &mut hickory_proto::op::Message, + ttl: u32, + ) -> Result<()> { for query in query.queries() { - let Some(node_id) = parse_hickory_node_info_name(query.name()) else { + let Some(node_id) = node_id_from_hickory_name(query.name()) else { continue; }; - let packet = this.get(&node_id); + let packet = self.get(&node_id); let Some(packet) = packet.as_ref() else { continue; }; let node_info = NodeInfo::from_pkarr_signed_packet(packet)?; - for record in node_info.to_hickory_records(&this.origin, TTL)? { + for record in node_info.to_hickory_records(&self.origin, ttl)? { reply.add_answer(record); } } Ok(()) } } + + impl QueryHandler for State { + fn resolve( + &self, + query: &hickory_proto::op::Message, + reply: &mut hickory_proto::op::Message, + ) -> impl Future> + Send { + const TTL: u32 = 30; + let res = self.resolve_dns(query, reply, TTL); + futures::future::ready(res) + } + } } mod pkarr_relay { diff --git a/iroh-net/src/dns/node_info.rs b/iroh-net/src/dns/node_info.rs index 3e7d9ae93fe..db5b85c46f8 100644 --- a/iroh-net/src/dns/node_info.rs +++ b/iroh-net/src/dns/node_info.rs @@ -176,7 +176,11 @@ impl NodeInfo { } } -pub(crate) fn parse_hickory_node_info_name(name: &hickory_proto::rr::Name) -> Option { +/// Parse a [`NodeId`] from iroh DNS name. +/// +/// Takes a [`hickory_proto::rr::Name`] DNS name and expects the first label to be `_iroh` +/// and the second label to be a z32 encoded [`NodeId`]. Does not care about subsequent labels. +pub(crate) fn node_id_from_hickory_name(name: &hickory_proto::rr::Name) -> Option { if name.num_labels() < 2 { return None; } @@ -252,7 +256,7 @@ impl NodeAttrs { use hickory_proto::rr; let mut records = records.iter().filter_map(|rr| match rr.data() { Some(rr::RData::TXT(txt)) => { - parse_hickory_node_info_name(rr.name()).map(|node_id| (node_id, txt)) + node_id_from_hickory_name(rr.name()).map(|node_id| (node_id, txt)) } _ => None, }); diff --git a/iroh-net/src/test_utils.rs b/iroh-net/src/test_utils.rs index ca13f47db6a..652d81a09d6 100644 --- a/iroh-net/src/test_utils.rs +++ b/iroh-net/src/test_utils.rs @@ -63,7 +63,8 @@ pub async fn run_relay_server() -> Result<(RelayMap, RelayUrl, CleanupDropGuard) Ok((m, url, CleanupDropGuard(tx))) } -pub mod dns_server { +#[cfg(test)] +pub(crate) mod dns_server { use std::net::{Ipv4Addr, SocketAddr}; use anyhow::{ensure, Result}; @@ -72,11 +73,13 @@ pub mod dns_server { op::{header::MessageType, Message}, serialize::binary::BinDecodable, }; + use hickory_resolver::{config::NameServerConfig, TokioAsyncResolver}; use tokio::{net::UdpSocket, task::JoinHandle}; use tokio_util::sync::CancellationToken; use tracing::{debug, warn}; - pub trait Resolver: Send + Sync + 'static { + /// Trait used by [`run_dns_server`] for answering DNS queries. + pub trait QueryHandler: Send + Sync + 'static { fn resolve( &self, query: &Message, @@ -84,10 +87,10 @@ pub mod dns_server { ) -> impl Future> + Send; } - pub type ResolveCallback = Box< + pub type QueryHandlerFunction = Box< dyn Fn(&Message, &mut Message) -> BoxFuture<'static, Result<()>> + Send + Sync + 'static, >; - impl Resolver for ResolveCallback { + impl QueryHandler for QueryHandlerFunction { fn resolve( &self, query: &Message, @@ -97,8 +100,11 @@ pub mod dns_server { } } + /// Run a DNS server. + /// + /// Must pass a [`QueryHandler`] that answers queries. Can be a [`ResolveCallback`] or a struct. pub async fn run_dns_server( - resolver: impl Resolver, + resolver: impl QueryHandler, cancel: CancellationToken, ) -> Result<(SocketAddr, JoinHandle>)> { let bind_addr = SocketAddr::from((Ipv4Addr::LOCALHOST, 0)); @@ -113,13 +119,23 @@ pub mod dns_server { Ok((bound_addr, join_handle)) } + /// Create a DNS resolver with a single nameserver. + pub fn create_dns_resolver(nameserver: SocketAddr) -> Result { + let mut config = hickory_resolver::config::ResolverConfig::new(); + let nameserver_config = + NameServerConfig::new(nameserver, hickory_resolver::config::Protocol::Udp); + config.add_name_server(nameserver_config); + let resolver = hickory_resolver::AsyncResolver::tokio(config, Default::default()); + Ok(resolver) + } + struct TestDnsServer { resolver: R, socket: UdpSocket, cancel: CancellationToken, } - impl TestDnsServer { + impl TestDnsServer { async fn run(self) -> Result<()> { let mut buf = [0; 1450]; loop { From 2e136a5801ab3565fac8f94077da5f3bfbe8762a Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Thu, 4 Apr 2024 00:34:53 +0200 Subject: [PATCH 56/66] feat: republishing in pkarr publisher --- iroh-net/src/discovery.rs | 8 +- iroh-net/src/discovery/pkarr_publish.rs | 128 ++++++++++++++++-------- iroh/src/node/builder.rs | 1 - 3 files changed, 94 insertions(+), 43 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 444c933a22d..40bd7cdc215 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -613,12 +613,14 @@ mod test_dns_pkarr { relay_url: Some("https://relay.example".parse().unwrap()), ..Default::default() }; + + + let resolver = create_dns_resolver(nameserver)?; let publisher = PkarrPublisher::new(secret_key, pkarr_url); - publisher.publish_addr_info(&addr_info).await?; + // does not block, update happens in background task + publisher.update_addr_info(&addr_info); // wait until our shared state received the update from pkarr publishing state.on_node(&node_id, timeout).await?; - - let resolver = create_dns_resolver(nameserver)?; let resolved = lookup_by_id(&resolver, &node_id, &origin).await?; let expected = NodeAddr { diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index f6966332183..6a671abee47 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -6,43 +6,70 @@ //! //! [pkarr]: https://pkarr.org -use std::sync::Arc; - use anyhow::Result; -use parking_lot::RwLock; use pkarr::SignedPacket; +use tokio::time::{Duration, Instant}; use tracing::warn; use url::Url; +use watchable::Watchable; -use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, AddrInfo}; +use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, AddrInfo, NodeId}; /// The n0 testing pkarr relay pub const N0_DNS_PKARR_RELAY: &str = "https://dns.iroh.link/pkarr"; -/// Default TTL for the _iroh TXT record in the pkarr signed packet -const DEFAULT_PKARR_TTL: u32 = 30; +/// Default TTL for the _iroh_node TXT record in the pkarr signed packet +pub const DEFAULT_PKARR_TTL: u32 = 30; + +/// Interval in which we will republish our node info even if unchanged. +pub const DEFAULT_REPUBLISH_INTERVAL: Duration = Duration::from_secs(60 * 5); /// Publish node info to a pkarr relay. #[derive(derive_more::Debug, Clone)] pub struct PkarrPublisher { - #[debug("SecretKey")] - secret_key: SecretKey, - #[debug("PkarrClient")] - pkarr_client: PkarrRelayClient, - last_published: Arc>>, - ttl: u32, + node_id: NodeId, + watch: Watchable>, } impl PkarrPublisher { /// Create a new config with a secret key and a pkarr relay URL. + /// + /// Will use [`DEFAULT_PKARR_TTL`] as the time-to-live value for the published packets. + /// Will republish info, even if unchanged, every [`DEFAULT_REPUBLISH_INTERVAL`] (5 minutes). pub fn new(secret_key: SecretKey, pkarr_relay: Url) -> Self { + Self::with_options( + secret_key, + pkarr_relay, + DEFAULT_PKARR_TTL, + DEFAULT_REPUBLISH_INTERVAL, + ) + } + + /// Create a new [`PkarrPublisher`] with a custom time-to-live (ttl) value for the published + /// [`pkarr::SignedPacket`]s. + pub fn with_options( + secret_key: SecretKey, + pkarr_relay: Url, + ttl: u32, + republish_interval: std::time::Duration, + ) -> Self { + let node_id = secret_key.public(); let pkarr_client = PkarrRelayClient::new(pkarr_relay); - Self { + let watch = Watchable::default(); + let service = PublisherService { + ttl, + watch: watch.clone(), secret_key, pkarr_client, - ttl: DEFAULT_PKARR_TTL, - last_published: Default::default(), - } + republish_interval: republish_interval.into(), + }; + // TODO: Make this task cancelablle and and/or store the task handle. + tokio::task::spawn(async move { + if let Err(err) = service.run().await { + warn!(?err, "PkarrPublisher service failed") + } + }); + Self { watch, node_id } } /// Create a config that publishes to the n0 dns server. @@ -51,39 +78,62 @@ impl PkarrPublisher { Self::new(secret_key, pkarr_relay) } - /// Set the TTL for pkarr packets, in seconds. + /// Publish [`AddrInfo`] about this node to a pkarr relay. /// - /// Default value is 30 seconds. - pub fn set_ttl(&mut self, ttl: u32) { - self.ttl = ttl; + /// This is a nonblocking function, the actual update is performed in the background. + pub fn update_addr_info(&self, info: &AddrInfo) { + let info = NodeInfo::new(self.node_id, info.relay_url.clone().map(Into::into)); + self.watch.update(Some(info)).ok(); + } +} + +/// Publish node info to a pkarr relay. +#[derive(derive_more::Debug, Clone)] +struct PublisherService { + #[debug("SecretKey")] + secret_key: SecretKey, + #[debug("PkarrClient")] + pkarr_client: PkarrRelayClient, + watch: Watchable>, + ttl: u32, + republish_interval: Duration, +} + +impl PublisherService { + async fn run(&self) -> Result<()> { + let watcher = self.watch.watch(); + let republish = tokio::time::sleep(Duration::MAX); + tokio::pin!(republish); + loop { + if watcher.peek().is_some() { + self.publish_current().await?; + } + tokio::select! { + res = watcher.watch_async() => match res { + Ok(()) => {}, + Err(_disconnected) => break, + }, + _ = &mut republish => {} + } + republish + .as_mut() + .reset(Instant::now() + self.republish_interval); + } + Ok(()) } - /// Publish [`AddrInfo`] about this node to a pkarr relay. - pub async fn publish_addr_info(&self, info: &AddrInfo) -> Result<()> { - let info = NodeInfo::new( - self.secret_key.public(), - info.relay_url.clone().map(Into::into), - ); - // only republish if the [`NodeInfo`] changed - if self.last_published.read().as_ref() == Some(&info) { - return Ok(()); + async fn publish_current(&self) -> Result<()> { + if let Some(info) = self.watch.get() { + let signed_packet = info.to_pkarr_signed_packet(&self.secret_key, self.ttl)?; + self.pkarr_client.publish(&signed_packet).await?; } - let _ = self.last_published.write().insert(info.clone()); - let signed_packet = info.to_pkarr_signed_packet(&self.secret_key, self.ttl)?; - self.pkarr_client.publish(&signed_packet).await?; Ok(()) } } impl Discovery for PkarrPublisher { fn publish(&self, info: &AddrInfo) { - let this = self.clone(); - let info = info.clone(); - tokio::task::spawn(async move { - if let Err(err) = this.publish_addr_info(&info).await { - warn!("failed to publish address update: {err:?}"); - } - }); + self.update_addr_info(info); } } diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs index d12135df6f7..029036c22b8 100644 --- a/iroh/src/node/builder.rs +++ b/iroh/src/node/builder.rs @@ -347,7 +347,6 @@ where // Enable DNS discovery by default Box::new(DnsDiscovery::n0_dns()), // Enable pkarr publishing by default - // TODO: We don't want nodes to self-publish. Remove once publishing over derpers lands. Box::new(PkarrPublisher::n0_dns(self.secret_key.clone())), ]); Some(Box::new(discovery)) From c47d0dc37300efb5890522cd8aa2541445e8ee9c Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 8 Apr 2024 23:27:37 +0200 Subject: [PATCH 57/66] fix: improve pkarr publisher --- iroh-net/src/discovery/pkarr_publish.rs | 61 ++++++++++++++++--------- 1 file changed, 39 insertions(+), 22 deletions(-) diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index 6a671abee47..00181bddc00 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -6,22 +6,27 @@ //! //! [pkarr]: https://pkarr.org +use std::sync::Arc; + use anyhow::Result; use pkarr::SignedPacket; -use tokio::time::{Duration, Instant}; +use tokio::{ + task::JoinHandle, + time::{Duration, Instant}, +}; use tracing::warn; use url::Url; use watchable::Watchable; use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, AddrInfo, NodeId}; -/// The n0 testing pkarr relay +/// The pkarr relay run by n0. pub const N0_DNS_PKARR_RELAY: &str = "https://dns.iroh.link/pkarr"; -/// Default TTL for the _iroh_node TXT record in the pkarr signed packet +/// Default TTL for the records in the pkarr signed packet pub const DEFAULT_PKARR_TTL: u32 = 30; -/// Interval in which we will republish our node info even if unchanged. +/// Interval in which we will republish our node info even if unchanged: 5 minutes. pub const DEFAULT_REPUBLISH_INTERVAL: Duration = Duration::from_secs(60 * 5); /// Publish node info to a pkarr relay. @@ -29,13 +34,14 @@ pub const DEFAULT_REPUBLISH_INTERVAL: Duration = Duration::from_secs(60 * 5); pub struct PkarrPublisher { node_id: NodeId, watch: Watchable>, + join_handle: Arc>, } impl PkarrPublisher { /// Create a new config with a secret key and a pkarr relay URL. /// /// Will use [`DEFAULT_PKARR_TTL`] as the time-to-live value for the published packets. - /// Will republish info, even if unchanged, every [`DEFAULT_REPUBLISH_INTERVAL`] (5 minutes). + /// Will republish info, even if unchanged, every [`DEFAULT_REPUBLISH_INTERVAL`]. pub fn new(secret_key: SecretKey, pkarr_relay: Url) -> Self { Self::with_options( secret_key, @@ -63,16 +69,17 @@ impl PkarrPublisher { pkarr_client, republish_interval: republish_interval.into(), }; - // TODO: Make this task cancelablle and and/or store the task handle. - tokio::task::spawn(async move { - if let Err(err) = service.run().await { - warn!(?err, "PkarrPublisher service failed") - } - }); - Self { watch, node_id } + // TODO: Make this task cancelable and and/or store the task handle. + let join_handle = tokio::task::spawn(service.run()); + let join_handle = Arc::new(join_handle); + Self { + watch, + node_id, + join_handle, + } } - /// Create a config that publishes to the n0 dns server. + /// Create a config that publishes to the n0 dns server through [`N0_DNS_PKARR_RELAY`]. pub fn n0_dns(secret_key: SecretKey) -> Self { let pkarr_relay: Url = N0_DNS_PKARR_RELAY.parse().expect("url is valid"); Self::new(secret_key, pkarr_relay) @@ -87,6 +94,21 @@ impl PkarrPublisher { } } +impl Discovery for PkarrPublisher { + fn publish(&self, info: &AddrInfo) { + self.update_addr_info(info); + } +} + +impl Drop for PkarrPublisher { + fn drop(&mut self) { + // this means we're dropping the last reference + if let Some(handle) = Arc::get_mut(&mut self.join_handle) { + handle.abort(); + } + } +} + /// Publish node info to a pkarr relay. #[derive(derive_more::Debug, Clone)] struct PublisherService { @@ -100,13 +122,15 @@ struct PublisherService { } impl PublisherService { - async fn run(&self) -> Result<()> { + async fn run(self) { let watcher = self.watch.watch(); let republish = tokio::time::sleep(Duration::MAX); tokio::pin!(republish); loop { if watcher.peek().is_some() { - self.publish_current().await?; + if let Err(err) = self.publish_current().await { + warn!(?err, url = %self.pkarr_client.pkarr_relay , "Failed to publish to pkarr"); + } } tokio::select! { res = watcher.watch_async() => match res { @@ -119,7 +143,6 @@ impl PublisherService { .as_mut() .reset(Instant::now() + self.republish_interval); } - Ok(()) } async fn publish_current(&self) -> Result<()> { @@ -131,12 +154,6 @@ impl PublisherService { } } -impl Discovery for PkarrPublisher { - fn publish(&self, info: &AddrInfo) { - self.update_addr_info(info); - } -} - /// A pkarr client to publish [`pkarr::SignedPacket`]s to a pkarr relay. #[derive(Debug, Clone)] pub(crate) struct PkarrRelayClient { From a172e6c986b36acd82d093c49a1378e5a41faf27 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 8 Apr 2024 23:29:05 +0200 Subject: [PATCH 58/66] fix: magic endpoint constructor in tests --- iroh-net/src/discovery.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 40bd7cdc215..4fc97452716 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -614,7 +614,6 @@ mod test_dns_pkarr { ..Default::default() }; - let resolver = create_dns_resolver(nameserver)?; let publisher = PkarrPublisher::new(secret_key, pkarr_url); // does not block, update happens in background task @@ -698,6 +697,7 @@ mod test_dns_pkarr { ]); let ep = MagicEndpoint::builder() .relay_mode(RelayMode::Custom(relay_map)) + .insecure_skip_relay_cert_verify(true) .secret_key(secret_key) .dns_resolver(resolver) .alpns(vec![TEST_ALPN.to_vec()]) From bc5c05b6d4e788326ab8e0f44821e7f1aae47cd3 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 9 Apr 2024 00:12:10 +0200 Subject: [PATCH 59/66] fix: pkarr republishing --- iroh-net/src/discovery/pkarr_publish.rs | 44 +++++++++++++++---------- 1 file changed, 26 insertions(+), 18 deletions(-) diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index 00181bddc00..566547a429c 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -14,9 +14,9 @@ use tokio::{ task::JoinHandle, time::{Duration, Instant}, }; -use tracing::warn; +use tracing::{debug, info, warn}; use url::Url; -use watchable::Watchable; +use watchable::{Watchable, Watcher}; use crate::{discovery::Discovery, dns::node_info::NodeInfo, key::SecretKey, AddrInfo, NodeId}; @@ -64,7 +64,7 @@ impl PkarrPublisher { let watch = Watchable::default(); let service = PublisherService { ttl, - watch: watch.clone(), + watcher: watch.watch(), secret_key, pkarr_client, republish_interval: republish_interval.into(), @@ -116,40 +116,48 @@ struct PublisherService { secret_key: SecretKey, #[debug("PkarrClient")] pkarr_client: PkarrRelayClient, - watch: Watchable>, + watcher: Watcher>, ttl: u32, republish_interval: Duration, } impl PublisherService { async fn run(self) { - let watcher = self.watch.watch(); + let mut failed_attemps = 0; let republish = tokio::time::sleep(Duration::MAX); tokio::pin!(republish); loop { - if watcher.peek().is_some() { - if let Err(err) = self.publish_current().await { + if let Some(info) = self.watcher.get() { + if let Err(err) = self.publish_current(info).await { warn!(?err, url = %self.pkarr_client.pkarr_relay , "Failed to publish to pkarr"); + failed_attemps += 1; + // Retry after increasing timeout + republish + .as_mut() + .reset(Instant::now() + Duration::from_secs(failed_attemps)); + } else { + failed_attemps = 0; + // Republish after fixed interval + republish + .as_mut() + .reset(Instant::now() + self.republish_interval); } } + // Wait until either the retry/republish timeout is reached, or the node info changed. tokio::select! { - res = watcher.watch_async() => match res { - Ok(()) => {}, + res = self.watcher.watch_async() => match res { + Ok(()) => debug!("Publish node info to pkarr (info changed)"), Err(_disconnected) => break, }, - _ = &mut republish => {} + _ = &mut republish => debug!("Publish node info to pkarr (interval elapsed)"), } - republish - .as_mut() - .reset(Instant::now() + self.republish_interval); } } - async fn publish_current(&self) -> Result<()> { - if let Some(info) = self.watch.get() { - let signed_packet = info.to_pkarr_signed_packet(&self.secret_key, self.ttl)?; - self.pkarr_client.publish(&signed_packet).await?; - } + async fn publish_current(&self, info: NodeInfo) -> Result<()> { + info!("Publish node info to pkarr"); + let signed_packet = info.to_pkarr_signed_packet(&self.secret_key, self.ttl)?; + self.pkarr_client.publish(&signed_packet).await?; Ok(()) } } From 15740c964430154241ab85d0d1f0caf46f29a565 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 9 Apr 2024 00:15:19 +0200 Subject: [PATCH 60/66] deps: axum --- Cargo.lock | 281 +++--------------------------------------------- iroh/Cargo.toml | 1 - 2 files changed, 15 insertions(+), 267 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 95725a70e51..a6d11817f98 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -187,28 +187,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "async-stream" -version = "0.3.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd56dd203fef61ac097dd65721a419ddccb106b2d2b70ba60a6b529f03961a51" -dependencies = [ - "async-stream-impl", - "futures-core", - "pin-project-lite", -] - -[[package]] -name = "async-stream-impl" -version = "0.3.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.53", -] - [[package]] name = "async-trait" version = "0.1.78" @@ -248,40 +226,12 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "axum" -version = "0.6.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" -dependencies = [ - "async-trait", - "axum-core 0.3.4", - "bitflags 1.3.2", - "bytes", - "futures-util", - "http 0.2.12", - "http-body 0.4.6", - "hyper 0.14.28", - "itoa", - "matchit", - "memchr", - "mime", - "percent-encoding", - "pin-project-lite", - "rustversion", - "serde", - "sync_wrapper", - "tower", - "tower-layer", - "tower-service", -] - -[[package]] -name = "axum" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1236b4b292f6c4d6dc34604bb5120d85c3fe1d1aa596bd5cc52ca054d13e7b9e" +checksum = "3a6c9af12842a67734c9a2e355436e5d03b22383ed60cf13cd0c18fbfe3dcbcf" dependencies = [ "async-trait", - "axum-core 0.4.3", + "axum-core", "bytes", "futures-util", "http 1.1.0", @@ -300,7 +250,7 @@ dependencies = [ "serde_json", "serde_path_to_error", "serde_urlencoded", - "sync_wrapper", + "sync_wrapper 1.0.1", "tokio", "tower", "tower-layer", @@ -308,23 +258,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "axum-core" -version = "0.3.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "759fa577a247914fd3f7f76d62972792636412fbfd634cd452f6a385a74d2d2c" -dependencies = [ - "async-trait", - "bytes", - "futures-util", - "http 0.2.12", - "http-body 0.4.6", - "mime", - "rustversion", - "tower-layer", - "tower-service", -] - [[package]] name = "axum-core" version = "0.4.3" @@ -340,7 +273,7 @@ dependencies = [ "mime", "pin-project-lite", "rustversion", - "sync_wrapper", + "sync_wrapper 0.1.2", "tower-layer", "tower-service", "tracing", @@ -741,43 +674,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "console-api" -version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd326812b3fd01da5bb1af7d340d0d555fd3d4b641e7f1dfcf5962a902952787" -dependencies = [ - "futures-core", - "prost", - "prost-types", - "tonic", - "tracing-core", -] - -[[package]] -name = "console-subscriber" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7481d4c57092cd1c19dd541b92bdce883de840df30aa5d03fd48a3935c01842e" -dependencies = [ - "console-api", - "crossbeam-channel", - "crossbeam-utils", - "futures-task", - "hdrhistogram", - "humantime", - "prost-types", - "serde", - "serde_json", - "thread_local", - "tokio", - "tokio-stream", - "tonic", - "tracing", - "tracing-core", - "tracing-subscriber", -] - [[package]] name = "const-oid" version = "0.9.6" @@ -850,15 +746,6 @@ version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "19d374276b40fb8bbdee95aef7c7fa6b5316ec764510eb64b8dd0e2ed0d7e7f5" -[[package]] -name = "crc32fast" -version = "1.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3855a8a784b474f333699ef2bbca9db2c4a1f6d9088a90a2d25b1eb53111eaa" -dependencies = [ - "cfg-if", -] - [[package]] name = "criterion" version = "0.5.1" @@ -901,15 +788,6 @@ version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7059fff8937831a9ae6f0fe4d658ffabf58f2ca96aa9dec1c889f936f705f216" -[[package]] -name = "crossbeam-channel" -version = "0.5.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab3db02a9c5b5121e1e42fbdb1aeb65f5e02624cc58c43f2884c6ccac0b82f95" -dependencies = [ - "crossbeam-utils", -] - [[package]] name = "crossbeam-deque" version = "0.8.5" @@ -1555,16 +1433,6 @@ version = "0.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c007b1ae3abe1cb6f85a16305acd418b7ca6343b953633fee2b76d8f108b830f" -[[package]] -name = "flate2" -version = "1.0.28" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" -dependencies = [ - "crc32fast", - "miniz_oxide", -] - [[package]] name = "flume" version = "0.11.0" @@ -1828,25 +1696,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "h2" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51ee2dd2e4f378392eeff5d51618cd9a63166a2513846bbc55f21cfacd9199d4" -dependencies = [ - "bytes", - "fnv", - "futures-core", - "futures-sink", - "futures-util", - "http 1.1.0", - "indexmap 2.2.5", - "slab", - "tokio", - "tokio-util", - "tracing", -] - [[package]] name = "half" version = "2.4.0" @@ -1897,10 +1746,7 @@ version = "7.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "765c9198f173dd59ce26ff9f95ef0aafd0a0fe01fb9d72841bc5066a4c06511d" dependencies = [ - "base64 0.21.7", "byteorder", - "flate2", - "nom", "num-traits", ] @@ -2125,12 +1971,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "humantime" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" - [[package]] name = "hyper" version = "0.14.28" @@ -2141,7 +1981,7 @@ dependencies = [ "futures-channel", "futures-core", "futures-util", - "h2 0.3.26", + "h2", "http 0.2.12", "http-body 0.4.6", "httparse", @@ -2164,7 +2004,6 @@ dependencies = [ "bytes", "futures-channel", "futures-util", - "h2 0.4.3", "http 1.1.0", "http-body 1.0.0", "httparse", @@ -2190,18 +2029,6 @@ dependencies = [ "tokio-rustls", ] -[[package]] -name = "hyper-timeout" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbb958482e8c7be4bc3cf272a766a2b0bf1a6755e7a6ae777f017a31d11b13b1" -dependencies = [ - "hyper 0.14.28", - "pin-project-lite", - "tokio", - "tokio-io-timeout", -] - [[package]] name = "hyper-util" version = "0.1.3" @@ -2375,7 +2202,6 @@ dependencies = [ "bao-tree", "bytes", "clap", - "console-subscriber", "data-encoding", "derive_more", "flume", @@ -2632,7 +2458,7 @@ version = "0.13.0" dependencies = [ "aead", "anyhow", - "axum 0.7.4", + "axum", "backoff", "bytes", "clap", @@ -2802,15 +2628,6 @@ dependencies = [ "either", ] -[[package]] -name = "itertools" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" -dependencies = [ - "either", -] - [[package]] name = "itertools" version = "0.12.1" @@ -3907,38 +3724,6 @@ dependencies = [ "unarray", ] -[[package]] -name = "prost" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "146c289cda302b98a28d40c8b3b90498d6e526dd24ac2ecea73e4e491685b94a" -dependencies = [ - "bytes", - "prost-derive", -] - -[[package]] -name = "prost-derive" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efb6c9a1dd1def8e2124d17e83a20af56f1570d6c2d2bd9e266ccb768df3840e" -dependencies = [ - "anyhow", - "itertools 0.11.0", - "proc-macro2", - "quote", - "syn 2.0.53", -] - -[[package]] -name = "prost-types" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "193898f59edcf43c26227dcd4c8427f00d99d61e95dcde58dabd49fa291d470e" -dependencies = [ - "prost", -] - [[package]] name = "quanta" version = "0.12.2" @@ -4281,7 +4066,7 @@ dependencies = [ "encoding_rs", "futures-core", "futures-util", - "h2 0.3.26", + "h2", "http 0.2.12", "http-body 0.4.6", "hyper 0.14.28", @@ -4298,7 +4083,7 @@ dependencies = [ "serde", "serde_json", "serde_urlencoded", - "sync_wrapper", + "sync_wrapper 0.1.2", "system-configuration", "tokio", "tokio-rustls", @@ -5169,6 +4954,12 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" +[[package]] +name = "sync_wrapper" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" + [[package]] name = "synstructure" version = "0.12.6" @@ -5358,20 +5149,9 @@ dependencies = [ "signal-hook-registry", "socket2", "tokio-macros", - "tracing", "windows-sys 0.48.0", ] -[[package]] -name = "tokio-io-timeout" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30b74022ada614a1b4834de765f9bb43877f910cc8ce4be40e89042c9223a8bf" -dependencies = [ - "pin-project-lite", - "tokio", -] - [[package]] name = "tokio-macros" version = "2.2.0" @@ -5518,33 +5298,6 @@ dependencies = [ "winnow 0.6.5", ] -[[package]] -name = "tonic" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d560933a0de61cf715926b9cac824d4c883c2c43142f787595e48280c40a1d0e" -dependencies = [ - "async-stream", - "async-trait", - "axum 0.6.20", - "base64 0.21.7", - "bytes", - "h2 0.3.26", - "http 0.2.12", - "http-body 0.4.6", - "hyper 0.14.28", - "hyper-timeout", - "percent-encoding", - "pin-project", - "prost", - "tokio", - "tokio-stream", - "tower", - "tower-layer", - "tower-service", - "tracing", -] - [[package]] name = "tower" version = "0.4.13" @@ -5553,13 +5306,9 @@ checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" dependencies = [ "futures-core", "futures-util", - "indexmap 1.9.3", "pin-project", "pin-project-lite", - "rand", - "slab", "tokio", - "tokio-util", "tower-layer", "tower-service", "tracing", diff --git a/iroh/Cargo.toml b/iroh/Cargo.toml index af65ecd65cb..fd28310c9df 100644 --- a/iroh/Cargo.toml +++ b/iroh/Cargo.toml @@ -67,7 +67,6 @@ test-utils = ["iroh-net/test-utils"] [dev-dependencies] anyhow = { version = "1" } bytes = "1" -console-subscriber = "0.2" genawaiter = { version = "0.99", features = ["futures03"] } iroh = { path = ".", features = ["test-utils"] } iroh-test = { path = "../iroh-test" } From 134aeb4697fce7d5fe6686d43f1b09861d853431 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 9 Apr 2024 00:17:06 +0200 Subject: [PATCH 61/66] fix: remove online test, will hit rate limits --- iroh-net/src/discovery.rs | 36 ------------------------------------ 1 file changed, 36 deletions(-) diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs index 4fc97452716..117ab689b30 100644 --- a/iroh-net/src/discovery.rs +++ b/iroh-net/src/discovery.rs @@ -662,27 +662,6 @@ mod test_dns_pkarr { Ok(()) } - #[tokio::test] - async fn publish_discover_online_defaults() -> Result<()> { - let _logging_guard = iroh_test::logging::setup(); - - let cancel = CancellationToken::new(); - - let ep1 = ep_with_n0_defaults().await?; - let ep2 = ep_with_n0_defaults().await?; - - // wait until ep1 announced - let _addr = ep1.my_addr().await?; - // wait more for publish to finish and DNS server to handle things - tokio::time::sleep(Duration::from_secs(2)).await; - - // we connect only by node id! - let res = ep2.connect(ep1.node_id().into(), TEST_ALPN).await; - assert!(res.is_ok(), "connection established"); - cancel.cancel(); - Ok(()) - } - async fn ep_with_discovery( relay_map: RelayMap, nameserver: SocketAddr, @@ -707,21 +686,6 @@ mod test_dns_pkarr { Ok(ep) } - async fn ep_with_n0_defaults() -> Result { - let secret_key = SecretKey::generate(); - let discovery = ConcurrentDiscovery::from_services(vec![ - Box::new(DnsDiscovery::n0_dns()), - Box::new(PkarrPublisher::n0_dns(secret_key.clone())), - ]); - let ep = MagicEndpoint::builder() - .secret_key(secret_key) - .alpns(vec![TEST_ALPN.to_vec()]) - .discovery(Box::new(discovery)) - .bind(0) - .await?; - Ok(ep) - } - async fn run_dns_and_pkarr_servers( origin: impl ToString, cancel: CancellationToken, From 3f2f543ebb168366e792dd632a475a08fa2418bf Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Tue, 9 Apr 2024 00:23:18 +0200 Subject: [PATCH 62/66] cleanups --- iroh-net/src/discovery/pkarr_publish.rs | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index 566547a429c..6607c01c952 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -33,7 +33,7 @@ pub const DEFAULT_REPUBLISH_INTERVAL: Duration = Duration::from_secs(60 * 5); #[derive(derive_more::Debug, Clone)] pub struct PkarrPublisher { node_id: NodeId, - watch: Watchable>, + watchable: Watchable>, join_handle: Arc>, } @@ -61,21 +61,19 @@ impl PkarrPublisher { ) -> Self { let node_id = secret_key.public(); let pkarr_client = PkarrRelayClient::new(pkarr_relay); - let watch = Watchable::default(); + let watchable = Watchable::default(); let service = PublisherService { ttl, - watcher: watch.watch(), + watcher: watchable.watch(), secret_key, pkarr_client, - republish_interval: republish_interval.into(), + republish_interval, }; - // TODO: Make this task cancelable and and/or store the task handle. let join_handle = tokio::task::spawn(service.run()); - let join_handle = Arc::new(join_handle); Self { - watch, + watchable, node_id, - join_handle, + join_handle: Arc::new(join_handle), } } @@ -90,7 +88,7 @@ impl PkarrPublisher { /// This is a nonblocking function, the actual update is performed in the background. pub fn update_addr_info(&self, info: &AddrInfo) { let info = NodeInfo::new(self.node_id, info.relay_url.clone().map(Into::into)); - self.watch.update(Some(info)).ok(); + self.watchable.update(Some(info)).ok(); } } From bf92de56c177f60c1f042821fb26b54f19832fd3 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 10 Apr 2024 13:18:13 +0200 Subject: [PATCH 63/66] refactor: make PkarrRelayClient public --- iroh-net/src/discovery/pkarr_publish.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iroh-net/src/discovery/pkarr_publish.rs b/iroh-net/src/discovery/pkarr_publish.rs index 6607c01c952..a39cac4b5fa 100644 --- a/iroh-net/src/discovery/pkarr_publish.rs +++ b/iroh-net/src/discovery/pkarr_publish.rs @@ -162,7 +162,7 @@ impl PublisherService { /// A pkarr client to publish [`pkarr::SignedPacket`]s to a pkarr relay. #[derive(Debug, Clone)] -pub(crate) struct PkarrRelayClient { +pub struct PkarrRelayClient { inner: pkarr::PkarrClient, pkarr_relay: Url, } From cb2454e9c7b44ae94763b9f432b05803210f96f5 Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Wed, 10 Apr 2024 15:04:49 +0200 Subject: [PATCH 64/66] feat: make attribute parsing generic and pub --- iroh-net/src/dns/node_info.rs | 121 +++++++++++++++++++++------------- 1 file changed, 77 insertions(+), 44 deletions(-) diff --git a/iroh-net/src/dns/node_info.rs b/iroh-net/src/dns/node_info.rs index db5b85c46f8..6964f90758f 100644 --- a/iroh-net/src/dns/node_info.rs +++ b/iroh-net/src/dns/node_info.rs @@ -1,7 +1,7 @@ //! This module contains functions and structs to lookup node information from DNS //! and to encode node information in Pkarr signed packets. -use std::{collections::BTreeMap, str::FromStr}; +use std::{collections::BTreeMap, fmt::Display, hash::Hash, str::FromStr}; use anyhow::{anyhow, ensure, Result}; use hickory_proto::error::ProtoError; @@ -13,7 +13,7 @@ use crate::{key::SecretKey, AddrInfo, NodeAddr, NodeId}; /// The DNS name for the iroh TXT record pub const IROH_TXT_NAME: &str = "_iroh"; -/// The attributes supported for `_iroh` DNS records +/// The attributes supported by iroh for `_iroh` DNS records #[derive( Debug, strum::Display, strum::AsRefStr, strum::EnumString, Hash, Eq, PartialEq, Ord, PartialOrd, )] @@ -28,8 +28,8 @@ pub enum IrohAttr { /// The domain name must either contain an _iroh TXT record or be a CNAME record that leads to /// an _iroh TXT record. pub async fn lookup_by_domain(resolver: &TokioAsyncResolver, domain: &str) -> Result { - let name = Name::from_str(domain)?; - let info = lookup_node_info(resolver, name).await?; + let attrs = TxtAttrs::::lookup_by_domain(resolver, domain).await?; + let info: NodeInfo = attrs.into(); Ok(info.into()) } @@ -39,22 +39,9 @@ pub async fn lookup_by_id( node_id: &NodeId, origin: &str, ) -> Result { - let domain = format!("{}.{}", to_z32(node_id), origin); - lookup_by_domain(resolver, &domain).await -} - -async fn lookup_node_info(resolver: &TokioAsyncResolver, name: Name) -> Result { - let name = ensure_iroh_txt_label(name)?; - let lookup = resolver.txt_lookup(name).await?; - NodeInfo::from_hickory_records(lookup.as_lookup().records()) -} - -fn ensure_iroh_txt_label(name: Name) -> Result { - if name.iter().next() == Some(IROH_TXT_NAME.as_bytes()) { - Ok(name) - } else { - Name::parse(IROH_TXT_NAME, Some(&name)) - } + let attrs = TxtAttrs::::lookup_by_id(resolver, node_id, origin).await?; + let info: NodeInfo = attrs.into(); + Ok(info.into()) } /// Encode a [`NodeId`] in [`z-base-32`] encoding. @@ -84,14 +71,14 @@ pub struct NodeInfo { pub relay_url: Option, } -impl From for NodeInfo { - fn from(attrs: NodeAttrs) -> Self { +impl From> for NodeInfo { + fn from(attrs: TxtAttrs) -> Self { (&attrs).into() } } -impl From<&NodeAttrs> for NodeInfo { - fn from(attrs: &NodeAttrs) -> Self { +impl From<&TxtAttrs> for NodeInfo { + fn from(attrs: &TxtAttrs) -> Self { let node_id = attrs.node_id(); let attrs = attrs.attrs(); let relay_url = attrs @@ -104,7 +91,7 @@ impl From<&NodeAttrs> for NodeInfo { } } -impl From<&NodeInfo> for NodeAttrs { +impl From<&NodeInfo> for TxtAttrs { fn from(info: &NodeInfo) -> Self { let mut attrs = vec![]; if let Some(relay_url) = &info.relay_url { @@ -138,19 +125,19 @@ impl NodeInfo { Self { node_id, relay_url } } - fn to_attrs(&self) -> NodeAttrs { + fn to_attrs(&self) -> TxtAttrs { self.into() } /// Try to parse a [`NodeInfo`] from a set of DNS records. pub fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { - let attrs = NodeAttrs::from_hickory_records(records)?; + let attrs = TxtAttrs::from_hickory_records(records)?; Ok(attrs.into()) } /// Try to parse a [`NodeInfo`] from a [`pkarr::SignedPacket`]. pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { - let attrs = NodeAttrs::from_pkarr_signed_packet(packet)?; + let attrs = TxtAttrs::from_pkarr_signed_packet(packet)?; Ok(attrs.into()) } @@ -194,28 +181,35 @@ pub(crate) fn node_id_from_hickory_name(name: &hickory_proto::rr::Name) -> Optio Some(node_id) } -struct NodeAttrs { +/// Attributes parsed from `_iroh` TXT records. +/// +/// This struct is generic over the key type. When using with String, this will parse all +/// attributes. Can also be used with an enum, if it implements [`FromStr`] and [`Display`]. +#[derive(Debug)] +pub struct TxtAttrs { node_id: NodeId, - attrs: BTreeMap>, + attrs: BTreeMap>, } -impl NodeAttrs { - fn from_parts(node_id: NodeId, pairs: impl Iterator) -> Self { - let mut attrs: BTreeMap> = BTreeMap::new(); +impl TxtAttrs { + /// Create from a node id and an iterator of key-value pairs. + pub fn from_parts(node_id: NodeId, pairs: impl Iterator) -> Self { + let mut attrs: BTreeMap> = BTreeMap::new(); for (k, v) in pairs { attrs.entry(k).or_default().push(v); } Self { attrs, node_id } } - fn from_strings(node_id: NodeId, strings: impl Iterator) -> Result { - let mut attrs: BTreeMap> = BTreeMap::new(); + /// Create from a node id and an iterator of "{key}={value}" strings. + pub fn from_strings(node_id: NodeId, strings: impl Iterator) -> Result { + let mut attrs: BTreeMap> = BTreeMap::new(); for s in strings { let mut parts = s.split('='); let (Some(key), Some(value)) = (parts.next(), parts.next()) else { continue; }; - let Ok(attr) = IrohAttr::from_str(key) else { + let Ok(attr) = T::from_str(key) else { continue; }; attrs.entry(attr).or_default().push(value.to_string()); @@ -223,16 +217,41 @@ impl NodeAttrs { Ok(Self { attrs, node_id }) } - fn attrs(&self) -> &BTreeMap> { + async fn lookup(resolver: &TokioAsyncResolver, name: Name) -> Result { + let name = ensure_iroh_txt_label(name)?; + let lookup = resolver.txt_lookup(name).await?; + let attrs = Self::from_hickory_records(lookup.as_lookup().records())?; + Ok(attrs) + } + + /// Lookup attributes for a node id and origin domain. + pub async fn lookup_by_id( + resolver: &TokioAsyncResolver, + node_id: &NodeId, + origin: &str, + ) -> Result { + let name = node_domain(node_id, origin)?; + TxtAttrs::lookup(resolver, name).await + } + + /// Lookup attributes for a domain. + pub async fn lookup_by_domain(resolver: &TokioAsyncResolver, domain: &str) -> Result { + let name = Name::from_str(domain)?; + TxtAttrs::lookup(resolver, name).await + } + + /// Get a reference to the parsed attributes. + pub fn attrs(&self) -> &BTreeMap> { &self.attrs } - fn node_id(&self) -> NodeId { + /// Get the node id. + pub fn node_id(&self) -> NodeId { self.node_id } - /// Try to parse a [`NodeInfo`] from a [`pkarr::SignedPacket`]. - fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { + /// Try to parse a from a [`pkarr::SignedPacket`]. + pub fn from_pkarr_signed_packet(packet: &pkarr::SignedPacket) -> Result { use pkarr::dns::{self, rdata::RData}; let pubkey = packet.public_key(); let pubkey_z32 = pubkey.to_z32(); @@ -251,8 +270,8 @@ impl NodeAttrs { Self::from_strings(node_id, txt_strs) } - /// Try to parse a [`NodeInfo`] from a set of DNS records. - fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { + /// Try to parse a from a set of DNS records. + pub fn from_hickory_records(records: &[hickory_proto::rr::Record]) -> Result { use hickory_proto::rr; let mut records = records.iter().filter_map(|rr| match rr.data() { Some(rr::RData::TXT(txt)) => { @@ -279,7 +298,7 @@ impl NodeAttrs { } /// Convert into list of [`hickory_proto::rr::Record`]. - fn to_hickory_records( + pub fn to_hickory_records( &self, origin: &str, ttl: u32, @@ -297,7 +316,7 @@ impl NodeAttrs { /// Create a [`pkarr::SignedPacket`] by constructing a DNS packet and /// signing it with a [`SecretKey`]. - fn to_pkarr_signed_packet( + pub fn to_pkarr_signed_packet( &self, secret_key: &SecretKey, ttl: u32, @@ -327,3 +346,17 @@ impl NodeAttrs { Ok(packet) } } + +fn ensure_iroh_txt_label(name: Name) -> Result { + if name.iter().next() == Some(IROH_TXT_NAME.as_bytes()) { + Ok(name) + } else { + Name::parse(IROH_TXT_NAME, Some(&name)) + } +} + +fn node_domain(node_id: &NodeId, origin: &str) -> Result { + let domain = format!("{}.{}", to_z32(node_id), origin); + let domain = Name::from_str(&domain)?; + Ok(domain) +} From e6f56d33f5dc95c2b2285cab223cd5b128b9aad3 Mon Sep 17 00:00:00 2001 From: Franz Heinzmann Date: Thu, 11 Apr 2024 08:17:54 +0200 Subject: [PATCH 65/66] feat: pkarr relay with DNS server (#2167) ## Description Imports https://github.com/n0-computer/iroh-dns-server into this repo. See n0-computer/iroh-dns-server#5 for previous review/discussion. Now includes an integration smoke test in `iroh-dns-server/src/lib.rs`. ## Notes & open questions I *think* I addressed most review points that came up in the initial review. Prominently still open is: * The `redb` store is used from async context but only exposes a sync interface (redb default). I think this is fine for medium load. However for better performance we should reuse transactions, which likely means we need an actor on a separate thread, as we do in iroh-bytes and iroh-sync. ## Change checklist - [ ] Self-review. - [ ] Documentation updates if relevant. - [ ] Tests if relevant. --- Cargo.lock | 360 ++++++++++++++++++-- Cargo.toml | 1 + iroh-dns-server/Cargo.toml | 54 +++ iroh-dns-server/README.md | 38 +++ iroh-dns-server/config.dev.toml | 18 + iroh-dns-server/config.prod.toml | 13 + iroh-dns-server/examples/convert.rs | 33 ++ iroh-dns-server/examples/publish.rs | 106 ++++++ iroh-dns-server/examples/resolve.rs | 77 +++++ iroh-dns-server/src/config.rs | 133 ++++++++ iroh-dns-server/src/dns.rs | 277 +++++++++++++++ iroh-dns-server/src/dns/node_authority.rs | 190 +++++++++++ iroh-dns-server/src/http.rs | 258 ++++++++++++++ iroh-dns-server/src/http/doh.rs | 77 +++++ iroh-dns-server/src/http/doh/extract.rs | 250 ++++++++++++++ iroh-dns-server/src/http/doh/response.rs | 144 ++++++++ iroh-dns-server/src/http/error.rs | 101 ++++++ iroh-dns-server/src/http/extract.rs | 1 + iroh-dns-server/src/http/pkarr.rs | 52 +++ iroh-dns-server/src/http/rate_limiting.rs | 40 +++ iroh-dns-server/src/http/tls.rs | 184 ++++++++++ iroh-dns-server/src/lib.rs | 73 ++++ iroh-dns-server/src/main.rs | 36 ++ iroh-dns-server/src/metrics.rs | 64 ++++ iroh-dns-server/src/server.rs | 107 ++++++ iroh-dns-server/src/state.rs | 12 + iroh-dns-server/src/store.rs | 190 +++++++++++ iroh-dns-server/src/store/signed_packets.rs | 98 ++++++ iroh-dns-server/src/util.rs | 151 ++++++++ iroh-net/Cargo.toml | 2 +- 30 files changed, 3107 insertions(+), 33 deletions(-) create mode 100644 iroh-dns-server/Cargo.toml create mode 100644 iroh-dns-server/README.md create mode 100644 iroh-dns-server/config.dev.toml create mode 100644 iroh-dns-server/config.prod.toml create mode 100644 iroh-dns-server/examples/convert.rs create mode 100644 iroh-dns-server/examples/publish.rs create mode 100644 iroh-dns-server/examples/resolve.rs create mode 100644 iroh-dns-server/src/config.rs create mode 100644 iroh-dns-server/src/dns.rs create mode 100644 iroh-dns-server/src/dns/node_authority.rs create mode 100644 iroh-dns-server/src/http.rs create mode 100644 iroh-dns-server/src/http/doh.rs create mode 100644 iroh-dns-server/src/http/doh/extract.rs create mode 100644 iroh-dns-server/src/http/doh/response.rs create mode 100644 iroh-dns-server/src/http/error.rs create mode 100644 iroh-dns-server/src/http/extract.rs create mode 100644 iroh-dns-server/src/http/pkarr.rs create mode 100644 iroh-dns-server/src/http/rate_limiting.rs create mode 100644 iroh-dns-server/src/http/tls.rs create mode 100644 iroh-dns-server/src/lib.rs create mode 100644 iroh-dns-server/src/main.rs create mode 100644 iroh-dns-server/src/metrics.rs create mode 100644 iroh-dns-server/src/server.rs create mode 100644 iroh-dns-server/src/state.rs create mode 100644 iroh-dns-server/src/store.rs create mode 100644 iroh-dns-server/src/store/signed_packets.rs create mode 100644 iroh-dns-server/src/util.rs diff --git a/Cargo.lock b/Cargo.lock index c635ee8a5c3..efcaa392fc4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -130,6 +130,12 @@ version = "1.0.81" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0952808a6c2afd1aa8947271f3a60f1a6763c7b912d210184c5149b5cf147247" +[[package]] +name = "arc-swap" +version = "1.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" + [[package]] name = "arrayref" version = "0.3.7" @@ -148,8 +154,24 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7f6fd5ddaf0351dff5b8da21b2fb4ff8e08ddd02857f0bf69c47639106c0fff0" dependencies = [ - "asn1-rs-derive", - "asn1-rs-impl", + "asn1-rs-derive 0.4.0", + "asn1-rs-impl 0.1.0", + "displaydoc", + "nom", + "num-traits", + "rusticata-macros", + "thiserror", + "time", +] + +[[package]] +name = "asn1-rs" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22ad1373757efa0f70ec53939aabc7152e1591cb485208052993070ac8d2429d" +dependencies = [ + "asn1-rs-derive 0.5.0", + "asn1-rs-impl 0.2.0", "displaydoc", "nom", "num-traits", @@ -167,7 +189,19 @@ dependencies = [ "proc-macro2", "quote", "syn 1.0.109", - "synstructure", + "synstructure 0.12.6", +] + +[[package]] +name = "asn1-rs-derive" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7378575ff571966e99a744addeff0bff98b8ada0dedf1956d59e634db95eaac1" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.53", + "synstructure 0.13.1", ] [[package]] @@ -181,6 +215,17 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "asn1-rs-impl" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b18050c2cd6fe86c3a76584ef5e0baf286d038cda203eb6223df2cc413565f7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.53", +] + [[package]] name = "async-trait" version = "0.1.78" @@ -226,6 +271,7 @@ checksum = "3a6c9af12842a67734c9a2e355436e5d03b22383ed60cf13cd0c18fbfe3dcbcf" dependencies = [ "async-trait", "axum-core", + "axum-macros", "bytes", "futures-util", "http 1.1.0", @@ -273,6 +319,41 @@ dependencies = [ "tracing", ] +[[package]] +name = "axum-macros" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00c055ee2d014ae5981ce1016374e8213682aa14d9bf40e48ab48b5f3ef20eaa" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "syn 2.0.53", +] + +[[package]] +name = "axum-server" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1ad46c3ec4e12f4a4b6835e173ba21c25e484c9d02b49770bf006ce5367c036" +dependencies = [ + "arc-swap", + "bytes", + "futures-util", + "http 1.1.0", + "http-body 1.0.0", + "http-body-util", + "hyper 1.2.0", + "hyper-util", + "pin-project-lite", + "rustls", + "rustls-pemfile 2.1.2", + "tokio", + "tokio-rustls", + "tower", + "tower-service", +] + [[package]] name = "backoff" version = "0.4.0" @@ -346,6 +427,15 @@ version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9475866fec1451be56a3c2400fd081ff546538961565ccb5b7142cbd22bc7a51" +[[package]] +name = "base64-url" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb9fb9fb058cc3063b5fc88d9a21eefa2735871498a04e1650da76ed511c8569" +dependencies = [ + "base64 0.21.7", +] + [[package]] name = "base64ct" version = "1.6.0" @@ -1025,7 +1115,21 @@ version = "8.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dbd676fbbab537128ef0278adb5576cf363cff6aa22a7b24effe97347cfab61e" dependencies = [ - "asn1-rs", + "asn1-rs 0.5.2", + "displaydoc", + "nom", + "num-bigint", + "num-traits", + "rusticata-macros", +] + +[[package]] +name = "der-parser" +version = "9.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cd0a5c643689626bec213c4d8bd4d96acc8ffdb4ad4bb6bc16abf27d5f4b553" +dependencies = [ + "asn1-rs 0.6.1", "displaydoc", "nom", "num-bigint", @@ -1453,6 +1557,16 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "forwarded-header-value" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8835f84f38484cc86f110a805655697908257fb9a7af005234060891557198e9" +dependencies = [ + "nonempty", + "thiserror", +] + [[package]] name = "futures" version = "0.3.30" @@ -1689,6 +1803,25 @@ dependencies = [ "tracing", ] +[[package]] +name = "h2" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "816ec7294445779408f36fe57bc5b7fc1cf59664059096c65f905c1c61f58069" +dependencies = [ + "bytes", + "fnv", + "futures-core", + "futures-sink", + "futures-util", + "http 1.1.0", + "indexmap 2.2.5", + "slab", + "tokio", + "tokio-util", + "tracing", +] + [[package]] name = "half" version = "2.4.0" @@ -1798,9 +1931,14 @@ dependencies = [ "ipnet", "once_cell", "rand", + "ring 0.16.20", + "rustls", + "rustls-pemfile 1.0.4", + "serde", "thiserror", "tinyvec", "tokio", + "tokio-rustls", "tracing", "url", ] @@ -1820,9 +1958,35 @@ dependencies = [ "parking_lot", "rand", "resolv-conf", + "rustls", + "serde", "smallvec", "thiserror", "tokio", + "tokio-rustls", + "tracing", +] + +[[package]] +name = "hickory-server" +version = "0.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fbbb45bc4dcb456445732c705e3cfdc7393b8bcae5c36ecec36b9d76bd67cb5" +dependencies = [ + "async-trait", + "bytes", + "cfg-if", + "enum-as-inner", + "futures-util", + "hickory-proto", + "hickory-resolver", + "rustls", + "serde", + "thiserror", + "time", + "tokio", + "tokio-rustls", + "tokio-util", "tracing", ] @@ -1974,7 +2138,7 @@ dependencies = [ "futures-channel", "futures-core", "futures-util", - "h2", + "h2 0.3.26", "http 0.2.12", "http-body 0.4.6", "httparse", @@ -1997,6 +2161,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", + "h2 0.4.4", "http 1.1.0", "http-body 1.0.0", "httparse", @@ -2382,6 +2547,52 @@ dependencies = [ "walkdir", ] +[[package]] +name = "iroh-dns-server" +version = "0.1.0" +dependencies = [ + "anyhow", + "async-trait", + "axum", + "axum-server", + "base64-url", + "bytes", + "clap", + "derive_more", + "dirs-next", + "futures", + "governor", + "hickory-proto", + "hickory-resolver", + "hickory-server", + "http 1.1.0", + "iroh-metrics", + "iroh-net", + "lru", + "parking_lot", + "pkarr", + "rcgen 0.12.1", + "redb 2.0.0", + "regex", + "rustls", + "rustls-pemfile 1.0.4", + "serde", + "struct_iterable", + "strum 0.26.2", + "tokio", + "tokio-rustls", + "tokio-rustls-acme", + "tokio-stream", + "tokio-util", + "toml 0.8.12", + "tower-http", + "tower_governor", + "tracing", + "tracing-subscriber", + "url", + "z32", +] + [[package]] name = "iroh-gossip" version = "0.13.0" @@ -2503,7 +2714,7 @@ dependencies = [ "ring 0.17.8", "rtnetlink", "rustls", - "rustls-pemfile", + "rustls-pemfile 1.0.4", "rustls-webpki", "serde", "serde_bytes", @@ -2530,7 +2741,7 @@ dependencies = [ "webpki-roots", "windows 0.51.1", "wmi", - "x509-parser", + "x509-parser 0.15.1", "z32", ] @@ -2924,6 +3135,12 @@ dependencies = [ "minimal-lexical", ] +[[package]] +name = "nonempty" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e9e591e719385e6ebaeb5ce5d3887f7d5676fceca6411d1925ccc95745f3d6f7" + [[package]] name = "nonzero_ext" version = "0.3.0" @@ -3107,7 +3324,16 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9bedf36ffb6ba96c2eb7144ef6270557b52e54b20c0a8e1eb2ff99a6c6959bff" dependencies = [ - "asn1-rs", + "asn1-rs 0.5.2", +] + +[[package]] +name = "oid-registry" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c958dd45046245b9c3c2547369bb634eb461670b2e7e0de552905801a648d1d" +dependencies = [ + "asn1-rs 0.6.1", ] [[package]] @@ -3235,16 +3461,6 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8835116a5c179084a830efb3adc117ab007512b535bc1a21c991d3b32a6b44dd" -[[package]] -name = "pem" -version = "2.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b13fe415cdf3c8e44518e18a7c95a13431d9bdf6d15367d82b23c377fdd441a" -dependencies = [ - "base64 0.21.7", - "serde", -] - [[package]] name = "pem" version = "3.0.3" @@ -3874,7 +4090,7 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "52c4f3084aa3bc7dfbba4eff4fab2a54db4324965d8872ab933565e6fbd83bc6" dependencies = [ - "pem 3.0.3", + "pem", "ring 0.16.20", "time", "yasna", @@ -3886,7 +4102,7 @@ version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "48406db8ac1f3cbc7dcdb56ec355343817958a356ff430259bb07baf7607e1e1" dependencies = [ - "pem 3.0.3", + "pem", "ring 0.17.8", "time", "yasna", @@ -4022,7 +4238,7 @@ dependencies = [ "encoding_rs", "futures-core", "futures-util", - "h2", + "h2 0.3.26", "http 0.2.12", "http-body 0.4.6", "hyper 0.14.28", @@ -4035,7 +4251,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "rustls", - "rustls-pemfile", + "rustls-pemfile 1.0.4", "serde", "serde_json", "serde_urlencoded", @@ -4215,7 +4431,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a9aace74cb666635c918e9c12bc0d348266037aa8eb599b5cba565709a8dff00" dependencies = [ "openssl-probe", - "rustls-pemfile", + "rustls-pemfile 1.0.4", "schannel", "security-framework", ] @@ -4229,6 +4445,22 @@ dependencies = [ "base64 0.21.7", ] +[[package]] +name = "rustls-pemfile" +version = "2.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29993a25686778eb88d4189742cd713c9bce943bc54251a33509dc63cbacf73d" +dependencies = [ + "base64 0.22.0", + "rustls-pki-types", +] + +[[package]] +name = "rustls-pki-types" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ecd36cc4259e3e4514335c4a138c6b43171a8d61d8f5c9348f9fc7529416f247" + [[package]] name = "rustls-webpki" version = "0.101.7" @@ -4922,6 +5154,17 @@ dependencies = [ "unicode-xid", ] +[[package]] +name = "synstructure" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.53", +] + [[package]] name = "sysinfo" version = "0.26.9" @@ -5125,19 +5368,22 @@ dependencies = [ [[package]] name = "tokio-rustls-acme" -version = "0.2.0" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfb6f50b5523d014ba161512c37457acb16fd8218c883c7152e0a67ab763f2d4" +checksum = "2ebc06d846f8367f24c3a8882328707d1a5e507ef4f40943723ddbe2c17b9f24" dependencies = [ "async-trait", + "axum-server", "base64 0.21.7", "chrono", "futures", "log", - "pem 2.0.1", - "rcgen 0.11.3", + "num-bigint", + "pem", + "proc-macro2", + "rcgen 0.12.1", "reqwest", - "ring 0.16.20", + "ring 0.17.8", "rustls", "serde", "serde_json", @@ -5146,7 +5392,7 @@ dependencies = [ "tokio-rustls", "url", "webpki-roots", - "x509-parser", + "x509-parser 0.16.0", ] [[package]] @@ -5264,6 +5510,23 @@ dependencies = [ "tracing", ] +[[package]] +name = "tower-http" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e9cd434a998747dd2c4276bc96ee2e0c7a2eadf3cae88e52be55a05fa9053f5" +dependencies = [ + "bitflags 2.5.0", + "bytes", + "http 1.1.0", + "http-body 1.0.0", + "http-body-util", + "pin-project-lite", + "tower-layer", + "tower-service", + "tracing", +] + [[package]] name = "tower-layer" version = "0.3.2" @@ -5276,6 +5539,22 @@ version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" +[[package]] +name = "tower_governor" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3790eac6ad3fb8d9d96c2b040ae06e2517aa24b067545d1078b96ae72f7bb9a7" +dependencies = [ + "axum", + "forwarded-header-value", + "governor", + "http 1.1.0", + "pin-project", + "thiserror", + "tower", + "tracing", +] + [[package]] name = "tracing" version = "0.1.40" @@ -5944,12 +6223,29 @@ version = "0.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7069fba5b66b9193bd2c5d3d4ff12b839118f6bcbef5328efafafb5395cf63da" dependencies = [ - "asn1-rs", + "asn1-rs 0.5.2", + "data-encoding", + "der-parser 8.2.0", + "lazy_static", + "nom", + "oid-registry 0.6.1", + "rusticata-macros", + "thiserror", + "time", +] + +[[package]] +name = "x509-parser" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcbc162f30700d6f3f82a24bf7cc62ffe7caea42c0b2cba8bf7f3ae50cf51f69" +dependencies = [ + "asn1-rs 0.6.1", "data-encoding", - "der-parser", + "der-parser 9.0.0", "lazy_static", "nom", - "oid-registry", + "oid-registry 0.7.0", "rusticata-macros", "thiserror", "time", diff --git a/Cargo.toml b/Cargo.toml index d945bda86c5..24dc3874d30 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -3,6 +3,7 @@ members = [ "iroh", "iroh-bytes", "iroh-base", + "iroh-dns-server", "iroh-gossip", "iroh-metrics", "iroh-net", diff --git a/iroh-dns-server/Cargo.toml b/iroh-dns-server/Cargo.toml new file mode 100644 index 00000000000..a1fd9997122 --- /dev/null +++ b/iroh-dns-server/Cargo.toml @@ -0,0 +1,54 @@ +[package] +name = "iroh-dns-server" +version = "0.1.0" +edition = "2021" +description = "A pkarr relay and DNS server" +license = "MIT OR Apache-2.0" +authors = ["Frando ", "n0 team"] +repository = "https://github.com/n0-computer/iroh-dns-server" +keywords = ["networking", "pkarr", "dns", "dns-server", "iroh"] +readme = "README.md" + +[dependencies] +anyhow = "1.0.80" +async-trait = "0.1.77" +axum = { version = "0.7.4", features = ["macros"] } +axum-server = { version = "0.6.0", features = ["tls-rustls"] } +base64-url = "2.0.2" +bytes = "1.5.0" +clap = { version = "4.5.1", features = ["derive"] } +derive_more = { version = "1.0.0-beta.1", features = ["debug", "display", "into", "from"] } +dirs-next = "2.0.0" +futures = "0.3.30" +governor = "0.6.3" +hickory-proto = "0.24.0" +hickory-server = { version = "0.24.0", features = ["dns-over-rustls"] } +http = "1.0.0" +iroh-metrics = { version = "0.13.0", path = "../iroh-metrics" } +lru = "0.12.3" +parking_lot = "0.12.1" +pkarr = { version = "1.1.2", features = [ "async", "relay"], default_features = false } +rcgen = "0.12.1" +redb = "2.0.0" +regex = "1.10.3" +rustls = "0.21" +rustls-pemfile = "1" +serde = { version = "1.0.197", features = ["derive"] } +struct_iterable = "0.1.1" +strum = { version = "0.26.1", features = ["derive"] } +tokio = { version = "1.36.0", features = ["full"] } +tokio-rustls = "0.24" +tokio-rustls-acme = { version = "0.3", features = ["axum"] } +tokio-stream = "0.1.14" +tokio-util = "0.7.10" +toml = "0.8.10" +tower-http = { version = "0.5.2", features = ["cors", "trace"] } +tower_governor = "0.3.2" +tracing = "0.1.40" +tracing-subscriber = "0.3.18" +url = "2.5.0" +z32 = "1.1.1" + +[dev-dependencies] +hickory-resolver = "0.24.0" +iroh-net = { version = "0.13.0", path = "../iroh-net" } diff --git a/iroh-dns-server/README.md b/iroh-dns-server/README.md new file mode 100644 index 00000000000..e6f68c7784f --- /dev/null +++ b/iroh-dns-server/README.md @@ -0,0 +1,38 @@ +# iroh-dns-server + +A server that functions as a [pkarr](https://github.com/Nuhvi/pkarr/) relay and +[DNS](https://de.wikipedia.org/wiki/Domain_Name_System) server. + +This server compiles to a binary `iroh-dns-server`. It needs a config file, of +which there are two examples included: + +- [`config.dev.toml`](./config.dev.toml) - suitable for local development +- [`config.prod.toml`](./config.dev.toml) - suitable for production, after + adjusting the domain names and IP addresses + +The server will expose the following services: + +- A DNS server listening on UDP and TCP for DNS queries +- A HTTP and/or HTTPS server which provides the following routes: + - `/pkarr`: `GET` and `PUT` for pkarr signed packets + - `/dns-query`: Answer DNS queries over + [DNS-over-HTTPS](https://datatracker.ietf.org/doc/html/rfc8484) + +All received and valid pkarr signed packets will be served over DNS. The pkarr +packet origin will be appended with the origin as configured by this server. + +# License + +This project is licensed under either of + +- Apache License, Version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or + http://www.apache.org/licenses/LICENSE-2.0) +- MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT) + +at your option. + +### Contribution + +Unless you explicitly state otherwise, any contribution intentionally submitted +for inclusion in this project by you, as defined in the Apache-2.0 license, +shall be dual licensed as above, without any additional terms or conditions. diff --git a/iroh-dns-server/config.dev.toml b/iroh-dns-server/config.dev.toml new file mode 100644 index 00000000000..80db5955734 --- /dev/null +++ b/iroh-dns-server/config.dev.toml @@ -0,0 +1,18 @@ +[http] +port = 8080 +bind_addr = "127.0.0.1" + +[https] +port = 8443 +bind_addr = "127.0.0.1" +domains = ["localhost"] +cert_mode = "self_signed" + +[dns] +port = 5300 +bind_addr = "127.0.0.1" +default_soa = "dns1.irohdns.example hostmaster.irohdns.example 0 10800 3600 604800 3600" +default_ttl = 900 +origins = ["irohdns.example.", "."] +rr_a = "127.0.0.1" +rr_ns = "ns1.irohdns.example." diff --git a/iroh-dns-server/config.prod.toml b/iroh-dns-server/config.prod.toml new file mode 100644 index 00000000000..8dde5fb6ba3 --- /dev/null +++ b/iroh-dns-server/config.prod.toml @@ -0,0 +1,13 @@ +[https] +port = 443 +domains = ["irohdns.example.org"] +cert_mode = "lets_encrypt" +letsencrypt_prod = true + +[dns] +port = 53 +default_soa = "dns1.irohdns.example.org hostmaster.irohdns.example.org 0 10800 3600 604800 3600" +default_ttl = 30 +origins = ["irohdns.example.org", "."] +rr_a = "203.0.10.10" +rr_ns = "ns1.irohdns.example.org." diff --git a/iroh-dns-server/examples/convert.rs b/iroh-dns-server/examples/convert.rs new file mode 100644 index 00000000000..401b28f5852 --- /dev/null +++ b/iroh-dns-server/examples/convert.rs @@ -0,0 +1,33 @@ +use std::str::FromStr; + +use clap::Parser; +use iroh_net::NodeId; + +#[derive(Debug, Parser)] +struct Cli { + #[clap(subcommand)] + command: Command, +} + +#[derive(Debug, Parser)] +enum Command { + NodeToPkarr { node_id: String }, + PkarrToNode { z32_pubkey: String }, +} + +fn main() -> anyhow::Result<()> { + let args = Cli::parse(); + match args.command { + Command::NodeToPkarr { node_id } => { + let node_id = NodeId::from_str(&node_id)?; + let public_key = pkarr::PublicKey::try_from(*node_id.as_bytes())?; + println!("{}", public_key.to_z32()) + } + Command::PkarrToNode { z32_pubkey } => { + let public_key = pkarr::PublicKey::try_from(z32_pubkey.as_str())?; + let node_id = NodeId::from_bytes(public_key.as_bytes())?; + println!("{}", node_id) + } + } + Ok(()) +} diff --git a/iroh-dns-server/examples/publish.rs b/iroh-dns-server/examples/publish.rs new file mode 100644 index 00000000000..3bad9bb9f5c --- /dev/null +++ b/iroh-dns-server/examples/publish.rs @@ -0,0 +1,106 @@ +use std::str::FromStr; + +use anyhow::{bail, Result}; +use clap::{Parser, ValueEnum}; +use iroh_net::{ + discovery::{ + dns::N0_DNS_NODE_ORIGIN, + pkarr_publish::{PkarrRelayClient, N0_DNS_PKARR_RELAY}, + }, + dns::node_info::{to_z32, NodeInfo, IROH_TXT_NAME}, + key::SecretKey, + NodeId, +}; +use url::Url; + +const LOCALHOST_PKARR: &str = "http://localhost:8080/pkarr"; +const EXAMPLE_ORIGIN: &str = "irohdns.example"; + +#[derive(ValueEnum, Clone, Debug, Default, Copy, strum::Display)] +#[strum(serialize_all = "kebab-case")] +pub enum Env { + /// Use the pkarr relay run by number0. + #[default] + Default, + /// Use a relay listening at http://localhost:8080 + Dev, +} + +/// Publish a record to an irohdns server. +/// +/// You have to set the IROH_SECRET environment variable to the node secret for which to publish. +#[derive(Parser, Debug)] +struct Cli { + /// Environment to publish to. + #[clap(value_enum, short, long, default_value_t = Env::Default)] + env: Env, + /// Pkarr Relay URL. If set, the --env option will be ignored. + #[clap(long, conflicts_with = "env")] + pkarr_relay: Option, + /// Home relay server to publish for this node + relay_url: Url, + /// Create a new node secret if IROH_SECRET is unset. Only for development / debugging. + #[clap(short, long)] + create: bool, +} + +#[tokio::main] +async fn main() -> Result<()> { + tracing_subscriber::fmt::init(); + let args = Cli::parse(); + + let secret_key = match std::env::var("IROH_SECRET") { + Ok(s) => SecretKey::from_str(&s)?, + Err(_) if args.create => { + let s = SecretKey::generate(); + println!("Generated a new node secret. To reuse, set"); + println!("IROH_SECRET={s}"); + s + } + Err(_) => { + bail!("Environtment variable IROH_SECRET is not set. To create a new secret, use the --create option.") + } + }; + + let node_id = secret_key.public(); + let pkarr_relay = match (args.pkarr_relay, args.env) { + (Some(pkarr_relay), _) => pkarr_relay, + (None, Env::Default) => N0_DNS_PKARR_RELAY.parse().expect("valid url"), + (None, Env::Dev) => LOCALHOST_PKARR.parse().expect("valid url"), + }; + + println!("announce {node_id}:"); + println!(" relay={}", args.relay_url); + println!(); + println!("publish to {pkarr_relay} ..."); + + let pkarr = PkarrRelayClient::new(pkarr_relay); + let node_info = NodeInfo::new(node_id, Some(args.relay_url)); + let signed_packet = node_info.to_pkarr_signed_packet(&secret_key, 30)?; + pkarr.publish(&signed_packet).await?; + + println!("signed packet published."); + println!("resolve with:"); + + match args.env { + Env::Default => { + println!(" cargo run --example resolve -- node {}", node_id); + println!(" dig {} TXT", fmt_domain(&node_id, N0_DNS_NODE_ORIGIN)) + } + Env::Dev => { + println!( + " cargo run --example resolve -- --env dev node {}", + node_id + ); + println!( + " dig @localhost -p 5300 {} TXT", + fmt_domain(&node_id, EXAMPLE_ORIGIN) + ) + } + } + Ok(()) +} + +fn fmt_domain(node_id: &NodeId, origin: &str) -> String { + format!("{}.{}.{}", IROH_TXT_NAME, to_z32(node_id), origin) +} diff --git a/iroh-dns-server/examples/resolve.rs b/iroh-dns-server/examples/resolve.rs new file mode 100644 index 00000000000..b9464ab38c9 --- /dev/null +++ b/iroh-dns-server/examples/resolve.rs @@ -0,0 +1,77 @@ +use std::net::SocketAddr; + +use clap::{Parser, ValueEnum}; +use hickory_resolver::{ + config::{NameServerConfig, Protocol, ResolverConfig}, + AsyncResolver, +}; +use iroh_net::{ + discovery::dns::N0_DNS_NODE_ORIGIN, + dns::{node_info::TxtAttrs, DnsResolver}, + NodeId, +}; + +const LOCALHOST_DNS: &str = "127.0.0.1:5300"; +const EXAMPLE_ORIGIN: &str = "irohdns.example"; + +#[derive(ValueEnum, Clone, Debug, Default)] +pub enum Env { + /// Use the system's nameservers with origin domain dns.iroh.link + #[default] + Default, + /// Use a localhost DNS server listening on port 5300 + Dev, +} + +#[derive(Debug, Parser)] +struct Cli { + #[clap(value_enum, short, long, default_value_t = Env::Default)] + env: Env, + #[clap(subcommand)] + command: Command, +} + +#[derive(Debug, Parser)] +enum Command { + /// Resolve node info by node id. + Node { node_id: NodeId }, + /// Resolve node info by domain. + Domain { domain: String }, +} + +#[tokio::main] +async fn main() -> anyhow::Result<()> { + let args = Cli::parse(); + let (resolver, origin) = match args.env { + Env::Default => ( + iroh_net::dns::default_resolver().clone(), + N0_DNS_NODE_ORIGIN, + ), + Env::Dev => ( + resolver_with_nameserver(LOCALHOST_DNS.parse()?), + EXAMPLE_ORIGIN, + ), + }; + let resolved = match args.command { + Command::Node { node_id } => { + TxtAttrs::::lookup_by_id(&resolver, &node_id, origin).await? + } + Command::Domain { domain } => { + TxtAttrs::::lookup_by_domain(&resolver, &domain).await? + } + }; + println!("resolved node {}", resolved.node_id()); + for (key, values) in resolved.attrs() { + for value in values { + println!(" {key}={value}"); + } + } + Ok(()) +} + +fn resolver_with_nameserver(nameserver: SocketAddr) -> DnsResolver { + let mut config = ResolverConfig::new(); + let nameserver_config = NameServerConfig::new(nameserver, Protocol::Udp); + config.add_name_server(nameserver_config); + AsyncResolver::tokio(config, Default::default()) +} diff --git a/iroh-dns-server/src/config.rs b/iroh-dns-server/src/config.rs new file mode 100644 index 00000000000..4f50fbc46b1 --- /dev/null +++ b/iroh-dns-server/src/config.rs @@ -0,0 +1,133 @@ +//! Configuration for the server + +use anyhow::{anyhow, Context, Result}; +use serde::{Deserialize, Serialize}; +use std::{ + env, + net::{IpAddr, Ipv4Addr, SocketAddr}, + path::{Path, PathBuf}, +}; + +use crate::{ + dns::DnsConfig, + http::{CertMode, HttpConfig, HttpsConfig}, +}; + +const DEFAULT_METRICS_ADDR: SocketAddr = SocketAddr::new(IpAddr::V4(Ipv4Addr::LOCALHOST), 9117); + +/// Server configuration +/// +/// The config is usually loaded from a file with [`Self::load`]. +/// +/// The struct also implements [`Default`] which creates a config suitable for local development +/// and testing. +#[derive(Debug, Serialize, Deserialize)] +pub struct Config { + /// Config for the HTTP server + /// + /// If set to `None` no HTTP server will be started. + pub http: Option, + /// Config for the HTTPS server + /// + /// If set to `None` no HTTPS server will be started. + pub https: Option, + /// Config for the DNS server. + pub dns: DnsConfig, + /// Config for the metrics server. + /// + /// The metrics server is started by default. To disable the metrics server, set to + /// `Some(MetricsConfig::disabled())`. + pub metrics: Option, +} + +/// The config for the metrics server. +#[derive(Debug, Serialize, Deserialize)] +pub struct MetricsConfig { + /// Set to true to disable the metrics server. + pub disabled: bool, + /// Optionally set a custom address to bind to. + pub bind_addr: Option, +} + +impl MetricsConfig { + /// Disable the metrics server. + pub fn disabled() -> Self { + Self { + disabled: true, + bind_addr: None, + } + } +} + +impl Config { + /// Load the config from a file. + pub async fn load(path: impl AsRef) -> Result { + let s = tokio::fs::read_to_string(path.as_ref()) + .await + .with_context(|| format!("failed to read {}", path.as_ref().to_string_lossy()))?; + let config: Config = toml::from_str(&s)?; + Ok(config) + } + + /// Get the data directory. + pub fn data_dir() -> Result { + let dir = if let Some(val) = env::var_os("IROH_DNS_DATA_DIR") { + PathBuf::from(val) + } else { + let path = dirs_next::data_dir().ok_or_else(|| { + anyhow!("operating environment provides no directory for application data") + })?; + path.join("iroh-dns") + }; + Ok(dir) + } + + /// Get the path to the store database file. + pub fn signed_packet_store_path() -> Result { + Ok(Self::data_dir()?.join("signed-packets-1.db")) + } + + /// Get the address where the metrics server should be bound, if set. + pub(crate) fn metrics_addr(&self) -> Option { + match &self.metrics { + None => Some(DEFAULT_METRICS_ADDR), + Some(conf) => match conf.disabled { + true => None, + false => Some(conf.bind_addr.unwrap_or(DEFAULT_METRICS_ADDR)), + }, + } + } +} + +impl Default for Config { + fn default() -> Self { + Self { + http: Some(HttpConfig { + port: 8080, + bind_addr: None, + }), + https: Some(HttpsConfig { + port: 8443, + bind_addr: None, + domains: vec!["localhost".to_string()], + cert_mode: CertMode::SelfSigned, + letsencrypt_contact: None, + letsencrypt_prod: None, + }), + dns: DnsConfig { + port: 5300, + bind_addr: None, + origins: vec!["irohdns.example.".to_string(), ".".to_string()], + + default_soa: "irohdns.example hostmaster.irohdns.example 0 10800 3600 604800 3600" + .to_string(), + default_ttl: 900, + + rr_a: Some(Ipv4Addr::LOCALHOST), + rr_aaaa: None, + rr_ns: Some("ns1.irohdns.example.".to_string()), + }, + metrics: None, + } + } +} diff --git a/iroh-dns-server/src/dns.rs b/iroh-dns-server/src/dns.rs new file mode 100644 index 00000000000..2faca9df6f2 --- /dev/null +++ b/iroh-dns-server/src/dns.rs @@ -0,0 +1,277 @@ +//! Implementation of a DNS name server for iroh node announces + +use std::{ + collections::BTreeMap, + io, + net::{IpAddr, Ipv4Addr, Ipv6Addr, SocketAddr}, + sync::Arc, + time::Duration, +}; + +use anyhow::{anyhow, Result}; +use async_trait::async_trait; +use bytes::Bytes; +use hickory_server::{ + authority::{Catalog, MessageResponse, ZoneType}, + proto::{ + self, + rr::{ + rdata::{self}, + RData, Record, RecordSet, RecordType, RrKey, + }, + serialize::{binary::BinEncoder, txt::RDataParser}, + }, + resolver::Name, + server::{Request, RequestHandler, ResponseHandler, ResponseInfo}, + store::in_memory::InMemoryAuthority, +}; + +use iroh_metrics::inc; +use proto::{op::ResponseCode, rr::LowerName}; +use serde::{Deserialize, Serialize}; +use tokio::{ + net::{TcpListener, UdpSocket}, + sync::broadcast, +}; + +use crate::{metrics::Metrics, store::ZoneStore}; + +use self::node_authority::NodeAuthority; + +mod node_authority; + +const DEFAULT_NS_TTL: u32 = 60 * 60 * 12; // 12h +const DEFAULT_SOA_TTL: u32 = 60 * 60 * 24 * 14; // 14d +const DEFAULT_A_TTL: u32 = 60 * 60; // 1h + +/// DNS server settings +#[derive(Clone, Debug, Serialize, Deserialize)] +pub struct DnsConfig { + /// The port to serve a local UDP DNS server at + pub port: u16, + /// The IPv4 or IPv6 address to bind the UDP DNS server. + /// Uses `0.0.0.0` if unspecified. + pub bind_addr: Option, + /// SOA record data for any authoritative DNS records + pub default_soa: String, + /// Default time to live for returned DNS records (TXT & SOA) + pub default_ttl: u32, + /// Domain used for serving the `_iroh_node..` DNS TXT entry + pub origins: Vec, + + /// `A` record to set for all origins + pub rr_a: Option, + /// `AAAA` record to set for all origins + pub rr_aaaa: Option, + /// `NS` record to set for all origins + pub rr_ns: Option, +} + +/// A DNS server that serves pkarr signed packets. +pub struct DnsServer { + local_addr: SocketAddr, + server: hickory_server::ServerFuture, +} + +impl DnsServer { + /// Spawn the server. + pub async fn spawn(config: DnsConfig, dns_handler: DnsHandler) -> Result { + const TCP_TIMEOUT: Duration = Duration::from_millis(1000); + let mut server = hickory_server::ServerFuture::new(dns_handler); + + let bind_addr = SocketAddr::new( + config.bind_addr.unwrap_or(Ipv4Addr::UNSPECIFIED.into()), + config.port, + ); + + let socket = UdpSocket::bind(bind_addr).await?; + + let socket_addr = socket.local_addr()?; + + server.register_socket(socket); + server.register_listener(TcpListener::bind(bind_addr).await?, TCP_TIMEOUT); + tracing::info!("DNS server listening on {}", bind_addr); + + Ok(Self { + server, + local_addr: socket_addr, + }) + } + + /// Get the local address of the UDP/TCP socket. + pub fn local_addr(&self) -> SocketAddr { + self.local_addr + } + + /// Shutdown the server an wait for all tasks to complete. + pub async fn shutdown(mut self) -> Result<()> { + self.server.shutdown_gracefully().await?; + Ok(()) + } + + /// Wait for all tasks to complete. + /// + /// Runs forever unless tasks fail. + pub async fn run_until_done(mut self) -> Result<()> { + self.server.block_until_done().await?; + Ok(()) + } +} + +/// State for serving DNS +#[derive(Clone, derive_more::Debug)] +pub struct DnsHandler { + #[debug("Catalog")] + catalog: Arc, +} + +impl DnsHandler { + /// Create a DNS server given some settings, a connection to the DB for DID-by-username lookups + /// and the server DID to serve under `_did.`. + pub fn new(zone_store: ZoneStore, config: &DnsConfig) -> Result { + let origins = config + .origins + .iter() + .map(Name::from_utf8) + .collect::, _>>()?; + + let (static_authority, serial) = create_static_authority(&origins, config)?; + let authority = NodeAuthority::new(zone_store, static_authority, origins, serial)?; + let authority = Arc::new(authority); + + let mut catalog = Catalog::new(); + for origin in authority.origins() { + catalog.upsert(LowerName::from(origin), Box::new(Arc::clone(&authority))); + } + + Ok(Self { + catalog: Arc::new(catalog), + }) + } + + /// Handle a DNS request + pub async fn answer_request(&self, request: Request) -> Result { + tracing::info!(?request, "Got DNS request"); + + let (tx, mut rx) = broadcast::channel(1); + let response_handle = Handle(tx); + + self.handle_request(&request, response_handle).await; + + tracing::debug!("Done handling request, trying to resolve response"); + Ok(rx.recv().await?) + } +} + +#[async_trait::async_trait] +impl RequestHandler for DnsHandler { + async fn handle_request( + &self, + request: &Request, + response_handle: R, + ) -> ResponseInfo { + inc!(Metrics, dns_requests); + match request.protocol() { + hickory_server::server::Protocol::Udp => inc!(Metrics, dns_requests_udp), + hickory_server::server::Protocol::Https => inc!(Metrics, dns_requests_https), + _ => {} + } + + let res = self.catalog.handle_request(request, response_handle).await; + match &res.response_code() { + ResponseCode::NoError => match res.answer_count() { + 0 => inc!(Metrics, dns_lookup_notfound), + _ => inc!(Metrics, dns_lookup_success), + }, + ResponseCode::NXDomain => inc!(Metrics, dns_lookup_notfound), + _ => inc!(Metrics, dns_lookup_error), + } + res + } +} + +/// A handle to the channel over which the response to a DNS request will be sent +#[derive(Debug, Clone)] +pub struct Handle(pub broadcast::Sender); + +#[async_trait] +impl ResponseHandler for Handle { + async fn send_response<'a>( + &mut self, + response: MessageResponse< + '_, + 'a, + impl Iterator + Send + 'a, + impl Iterator + Send + 'a, + impl Iterator + Send + 'a, + impl Iterator + Send + 'a, + >, + ) -> io::Result { + let mut bytes = Vec::with_capacity(512); + let info = { + let mut encoder = BinEncoder::new(&mut bytes); + response.destructive_emit(&mut encoder)? + }; + + let bytes = Bytes::from(bytes); + self.0.send(bytes).unwrap(); + + Ok(info) + } +} + +fn create_static_authority( + origins: &[Name], + config: &DnsConfig, +) -> Result<(InMemoryAuthority, u32)> { + let soa = RData::parse( + RecordType::SOA, + config.default_soa.split_ascii_whitespace(), + None, + )? + .into_soa() + .map_err(|_| anyhow!("Couldn't parse SOA: {}", config.default_soa))?; + let serial = soa.serial(); + let mut records = BTreeMap::new(); + for name in origins { + push_record( + &mut records, + serial, + Record::from_rdata(name.clone(), DEFAULT_SOA_TTL, RData::SOA(soa.clone())), + ); + if let Some(addr) = config.rr_a { + push_record( + &mut records, + serial, + Record::from_rdata(name.clone(), DEFAULT_A_TTL, RData::A(addr.into())), + ); + } + if let Some(addr) = config.rr_aaaa { + push_record( + &mut records, + serial, + Record::from_rdata(name.clone(), DEFAULT_A_TTL, RData::AAAA(addr.into())), + ); + } + if let Some(ns) = &config.rr_ns { + let ns = Name::parse(ns, Some(&Name::root()))?; + push_record( + &mut records, + serial, + Record::from_rdata(name.clone(), DEFAULT_NS_TTL, RData::NS(rdata::NS(ns))), + ); + } + } + + let static_authority = InMemoryAuthority::new(Name::root(), records, ZoneType::Primary, false) + .map_err(|e| anyhow!(e))?; + + Ok((static_authority, serial)) +} + +fn push_record(records: &mut BTreeMap, serial: u32, record: Record) { + let key = RrKey::new(record.name().clone().into(), record.record_type()); + let mut record_set = RecordSet::new(record.name(), record.record_type(), serial); + record_set.insert(record, serial); + records.insert(key, record_set); +} diff --git a/iroh-dns-server/src/dns/node_authority.rs b/iroh-dns-server/src/dns/node_authority.rs new file mode 100644 index 00000000000..67b498fc552 --- /dev/null +++ b/iroh-dns-server/src/dns/node_authority.rs @@ -0,0 +1,190 @@ +use std::{fmt, sync::Arc}; + +use anyhow::{bail, ensure, Result}; +use async_trait::async_trait; +use hickory_proto::{ + op::ResponseCode, + rr::{LowerName, Name, RecordType}, +}; +use hickory_server::{ + authority::{ + AuthLookup, Authority, LookupError, LookupOptions, LookupRecords, MessageRequest, + UpdateResult, ZoneType, + }, + server::RequestInfo, + store::in_memory::InMemoryAuthority, +}; + +use tracing::{debug, trace}; + +use crate::{ + store::ZoneStore, + util::{record_set_append_origin, PublicKeyBytes}, +}; + +#[derive(derive_more::Debug)] +pub struct NodeAuthority { + serial: u32, + origins: Vec, + #[debug("InMemoryAuthority")] + static_authority: InMemoryAuthority, + zones: ZoneStore, + // TODO: This is used by Authority::origin + // Find out what exactly this is used for - we don't have a primary origin. + first_origin: LowerName, +} + +impl NodeAuthority { + pub fn new( + zones: ZoneStore, + static_authority: InMemoryAuthority, + origins: Vec, + serial: u32, + ) -> Result { + ensure!(!origins.is_empty(), "at least one origin is required"); + let first_origin = LowerName::from(&origins[0]); + Ok(Self { + static_authority, + origins, + serial, + zones, + first_origin, + }) + } + + pub fn origins(&self) -> impl Iterator { + self.origins.iter() + } + + pub fn serial(&self) -> u32 { + self.serial + } +} + +#[async_trait] +impl Authority for NodeAuthority { + type Lookup = AuthLookup; + + fn zone_type(&self) -> ZoneType { + ZoneType::Primary + } + + fn is_axfr_allowed(&self) -> bool { + false + } + + async fn update(&self, _update: &MessageRequest) -> UpdateResult { + Err(ResponseCode::NotImp) + } + + fn origin(&self) -> &LowerName { + &self.first_origin + } + + async fn lookup( + &self, + name: &LowerName, + record_type: RecordType, + lookup_options: LookupOptions, + ) -> Result { + match record_type { + RecordType::SOA | RecordType::NS => { + self.static_authority + .lookup(name, record_type, lookup_options) + .await + } + _ => match split_and_parse_pkarr(name, &self.origins) { + Err(err) => { + trace!(%name, ?err, "name is not a pkarr zone"); + debug!("resolve static: name {name}"); + self.static_authority + .lookup(name, record_type, lookup_options) + .await + } + Ok((name, pubkey, origin)) => { + debug!(%origin, "resolve pkarr: {name} {pubkey}"); + match self + .zones + .resolve(&pubkey, &name, record_type) + .await + .map_err(err_refused)? + { + Some(pkarr_set) => { + let new_origin = Name::parse(&pubkey.to_z32(), Some(&origin)) + .map_err(err_refused)?; + let record_set = + record_set_append_origin(&pkarr_set, &new_origin, self.serial()) + .map_err(err_refused)?; + let records = LookupRecords::new(lookup_options, Arc::new(record_set)); + let answers = AuthLookup::answers(records, None); + Ok(answers) + } + None => Err(err_nx_domain("not found")), + } + } + }, + } + } + + async fn search( + &self, + request_info: RequestInfo<'_>, + lookup_options: LookupOptions, + ) -> Result { + debug!("searching NodeAuthority for: {}", request_info.query); + let lookup_name = request_info.query.name(); + let record_type: RecordType = request_info.query.query_type(); + match record_type { + RecordType::SOA => { + self.static_authority + .lookup(self.origin(), record_type, lookup_options) + .await + } + RecordType::AXFR => Err(LookupError::from(ResponseCode::Refused)), + _ => self.lookup(lookup_name, record_type, lookup_options).await, + } + } + + async fn get_nsec_records( + &self, + _name: &LowerName, + _lookup_options: LookupOptions, + ) -> Result { + Ok(AuthLookup::default()) + } +} + +fn split_and_parse_pkarr( + name: impl Into, + allowed_origins: &[Name], +) -> Result<(Name, PublicKeyBytes, Name)> { + let name = name.into(); + trace!("resolve {name}"); + for origin in allowed_origins.iter() { + trace!("try {origin}"); + if !origin.zone_of(&name) { + continue; + } + if name.num_labels() < origin.num_labels() + 1 { + bail!("invalid name"); + } + trace!("parse {origin}"); + let labels = name.iter().rev(); + let mut labels_without_origin = labels.skip(origin.num_labels() as usize); + let pkey_label = labels_without_origin.next().expect("length checked above"); + let pkey_str = std::str::from_utf8(pkey_label)?; + let pkey = PublicKeyBytes::from_z32(pkey_str)?; + let remaining_name = Name::from_labels(labels_without_origin)?; + return Ok((remaining_name, pkey, origin.clone())); + } + bail!("name does not match any origin"); +} + +fn err_refused(e: impl fmt::Debug) -> LookupError { + trace!("lookup failed (refused): {e:?}"); + LookupError::from(ResponseCode::Refused) +} +fn err_nx_domain(e: impl fmt::Debug) -> LookupError { + trace!("lookup failed (nxdomain): {e:?}"); + LookupError::from(ResponseCode::NXDomain) +} diff --git a/iroh-dns-server/src/http.rs b/iroh-dns-server/src/http.rs new file mode 100644 index 00000000000..34a1af8161d --- /dev/null +++ b/iroh-dns-server/src/http.rs @@ -0,0 +1,258 @@ +//! HTTP server part of iroh-dns-server + +use std::{ + net::{IpAddr, Ipv4Addr, SocketAddr}, + time::Instant, +}; + +use anyhow::{bail, Context, Result}; +use axum::{ + extract::{ConnectInfo, Request}, + handler::Handler, + http::Method, + middleware::{self, Next}, + response::IntoResponse, + routing::get, + Router, +}; +use iroh_metrics::{inc, inc_by}; +use serde::{Deserialize, Serialize}; +use tokio::{net::TcpListener, task::JoinSet}; +use tower_http::{ + cors::{self, CorsLayer}, + trace::TraceLayer, +}; +use tracing::{info, span, warn, Level}; + +mod doh; +mod error; +mod pkarr; +mod rate_limiting; +mod tls; + +use crate::state::AppState; +use crate::{config::Config, metrics::Metrics}; + +pub use self::tls::CertMode; + +/// Config for the HTTP server +#[derive(Debug, Serialize, Deserialize, Clone)] +pub struct HttpConfig { + /// Port to bind to + pub port: u16, + /// Optionally set a custom bind address (will use 0.0.0.0 if unset) + pub bind_addr: Option, +} + +/// Config for the HTTPS server +#[derive(Debug, Serialize, Deserialize, Clone)] +pub struct HttpsConfig { + /// Port to bind to + pub port: u16, + /// Optionally set a custom bind address (will use 0.0.0.0 if unset) + pub bind_addr: Option, + /// The list of domains for which SSL certificates should be created. + pub domains: Vec, + /// The mode of SSL certificate creation + pub cert_mode: CertMode, + /// Letsencrypt contact email address (required if using [`CertMode::LetsEncrypt`]) + pub letsencrypt_contact: Option, + /// Whether to use the letsenrypt production servers (only applies to [`CertMode::LetsEncrypt`]) + pub letsencrypt_prod: Option, +} + +/// The HTTP(S) server part of iroh-dns-server +pub struct HttpServer { + tasks: JoinSet>, + http_addr: Option, + https_addr: Option, +} + +impl HttpServer { + /// Spawn the server + pub async fn spawn( + http_config: Option, + https_config: Option, + state: AppState, + ) -> Result { + if http_config.is_none() && https_config.is_none() { + bail!("Either http or https config is required"); + } + + let app = create_app(state); + + let mut tasks = JoinSet::new(); + + // launch http + let http_addr = if let Some(config) = http_config { + let bind_addr = SocketAddr::new( + config.bind_addr.unwrap_or(Ipv4Addr::UNSPECIFIED.into()), + config.port, + ); + let app = app.clone(); + let listener = TcpListener::bind(bind_addr).await?.into_std()?; + let bound_addr = listener.local_addr()?; + let fut = axum_server::from_tcp(listener) + .serve(app.into_make_service_with_connect_info::()); + info!("HTTP server listening on {bind_addr}"); + tasks.spawn(fut); + Some(bound_addr) + } else { + None + }; + + // launch https + let https_addr = if let Some(config) = https_config { + let bind_addr = SocketAddr::new( + config.bind_addr.unwrap_or(Ipv4Addr::UNSPECIFIED.into()), + config.port, + ); + let acceptor = { + let cache_path = Config::data_dir()? + .join("cert_cache") + .join(config.cert_mode.to_string()); + tokio::fs::create_dir_all(&cache_path) + .await + .with_context(|| { + format!("failed to create cert cache dir at {cache_path:?}") + })?; + config + .cert_mode + .build( + config.domains, + cache_path, + config.letsencrypt_contact, + config.letsencrypt_prod.unwrap_or(false), + ) + .await? + }; + let listener = TcpListener::bind(bind_addr).await?.into_std()?; + let bound_addr = listener.local_addr()?; + let fut = axum_server::from_tcp(listener) + .acceptor(acceptor) + .serve(app.into_make_service_with_connect_info::()); + info!("HTTPS server listening on {bind_addr}"); + tasks.spawn(fut); + Some(bound_addr) + } else { + None + }; + + Ok(HttpServer { + tasks, + http_addr, + https_addr, + }) + } + + /// Get the bound address of the HTTP socket. + pub fn http_addr(&self) -> Option { + self.http_addr + } + + /// Get the bound address of the HTTPS socket. + pub fn https_addr(&self) -> Option { + self.https_addr + } + + /// Shutdown the server and wait for all tasks to complete. + pub async fn shutdown(mut self) -> Result<()> { + // TODO: Graceful cancellation. + self.tasks.abort_all(); + self.run_until_done().await?; + Ok(()) + } + + /// Wait for all tasks to complete. + /// + /// Runs forever unless tasks fail. + pub async fn run_until_done(mut self) -> Result<()> { + let mut final_res: anyhow::Result<()> = Ok(()); + while let Some(res) = self.tasks.join_next().await { + match res { + Ok(Ok(())) => {} + Err(err) if err.is_cancelled() => {} + Ok(Err(err)) => { + warn!(?err, "task failed"); + final_res = Err(anyhow::Error::from(err)); + } + Err(err) => { + warn!(?err, "task panicked"); + final_res = Err(err.into()); + } + } + } + final_res + } +} + +pub(crate) fn create_app(state: AppState) -> Router { + // configure cors middleware + let cors = CorsLayer::new() + // allow `GET` and `POST` when accessing the resource + .allow_methods([Method::GET, Method::POST, Method::PUT]) + // allow requests from any origin + .allow_origin(cors::Any); + + // configure tracing middleware + let trace = TraceLayer::new_for_http().make_span_with(|request: &http::Request<_>| { + let conn_info = request + .extensions() + .get::>() + .expect("connectinfo extension to be present"); + let span = span!( + Level::DEBUG, + "http_request", + method = ?request.method(), + uri = ?request.uri(), + src = %conn_info.0, + ); + span + }); + + // configure rate limiting middleware + let rate_limit = rate_limiting::create(); + + // configure routes + // + // only the pkarr::put route gets a rate limit + let router = Router::new() + .route("/dns-query", get(doh::get).post(doh::post)) + .route( + "/pkarr/:key", + get(pkarr::get).put(pkarr::put.layer(rate_limit)), + ) + .route("/healthcheck", get(|| async { "OK" })) + .route("/", get(|| async { "Hi!" })) + .with_state(state); + + // configure app + router + .layer(cors) + .layer(trace) + .route_layer(middleware::from_fn(metrics_middleware)) +} + +/// Record request metrics. +/// +// TODO: +// * Request duration would be much better tracked as a histogram. +// * It would be great to attach labels to the metrics, so that the recorded metrics +// can filter by method etc. +// +// See also +// https://github.com/tokio-rs/axum/blob/main/examples/prometheus-metrics/src/main.rs#L114 +async fn metrics_middleware(req: Request, next: Next) -> impl IntoResponse { + let start = Instant::now(); + let response = next.run(req).await; + let latency = start.elapsed().as_millis(); + let status = response.status(); + inc_by!(Metrics, http_requests_duration_ms, latency as u64); + inc!(Metrics, http_requests); + if status.is_success() { + inc!(Metrics, http_requests_success); + } else { + inc!(Metrics, http_requests_error); + } + response +} diff --git a/iroh-dns-server/src/http/doh.rs b/iroh-dns-server/src/http/doh.rs new file mode 100644 index 00000000000..caa785ff14b --- /dev/null +++ b/iroh-dns-server/src/http/doh.rs @@ -0,0 +1,77 @@ +//! DNS over HTTPS + +// This module is mostly copied from +// https://github.com/fission-codes/fission-server/blob/main/fission-server/src/routes/doh.rs + +use anyhow::anyhow; +use axum::{ + extract::State, + response::{IntoResponse, Response}, + Json, +}; +use hickory_server::proto::{self, serialize::binary::BinDecodable}; +use http::{ + header::{CACHE_CONTROL, CONTENT_TYPE}, + HeaderValue, StatusCode, +}; + +use crate::state::AppState; + +use super::error::AppResult; + +mod extract; +mod response; + +use self::extract::{DnsMimeType, DnsRequestBody, DnsRequestQuery}; + +/// GET handler for resolving DoH queries +pub async fn get( + State(state): State, + DnsRequestQuery(request, accept_type): DnsRequestQuery, +) -> AppResult { + let message_bytes = state.dns_handler.answer_request(request).await?; + let message = proto::op::Message::from_bytes(&message_bytes).map_err(|e| anyhow!(e))?; + + let min_ttl = message.answers().iter().map(|rec| rec.ttl()).min(); + + let mut response = match accept_type { + DnsMimeType::Message => (StatusCode::OK, message_bytes).into_response(), + DnsMimeType::Json => { + let response = self::response::DnsResponse::from_message(message)?; + (StatusCode::OK, Json(response)).into_response() + } + }; + + response + .headers_mut() + .insert(CONTENT_TYPE, accept_type.to_header_value()); + + if let Some(min_ttl) = min_ttl { + let maxage = + HeaderValue::from_str(&format!("s-maxage={min_ttl}")).map_err(|e| anyhow!(e))?; + response.headers_mut().insert(CACHE_CONTROL, maxage); + } + + Ok(response) +} + +/// POST handler for resolvng DoH queries +pub async fn post( + State(state): State, + DnsRequestBody(request): DnsRequestBody, +) -> Response { + let response = match state.dns_handler.answer_request(request).await { + Ok(response) => response, + Err(err) => return (StatusCode::INTERNAL_SERVER_ERROR, err.to_string()).into_response(), + }; + + ( + StatusCode::OK, + [(CONTENT_TYPE, DnsMimeType::Message.to_string())], + response, + ) + .into_response() +} + +// TODO: Port tests from +// https://github.com/fission-codes/fission-server/blob/main/fission-server/src/routes/doh.rs diff --git a/iroh-dns-server/src/http/doh/extract.rs b/iroh-dns-server/src/http/doh/extract.rs new file mode 100644 index 00000000000..819791c5638 --- /dev/null +++ b/iroh-dns-server/src/http/doh/extract.rs @@ -0,0 +1,250 @@ +//! Extractors for DNS-over-HTTPS requests + +// This module is mostly copied from +// https://github.com/fission-codes/fission-server/blob/394de877fad021260c69fdb1edd7bb4b2f98108c/fission-server/src/extract/doh.rs + +use async_trait::async_trait; +use axum::{ + extract::{ConnectInfo, FromRequest, FromRequestParts, Query}, + http::Request, +}; +use bytes::Bytes; +use hickory_server::{ + authority::MessageRequest, + proto::{ + self, + serialize::binary::{BinDecodable, BinDecoder, BinEncodable, BinEncoder}, + }, + server::{Protocol, Request as DNSRequest}, +}; +use http::{header, request::Parts, HeaderValue, StatusCode}; +use serde::Deserialize; +use std::{ + fmt::{self, Display, Formatter}, + net::SocketAddr, + str::FromStr, +}; +use tracing::info; + +use crate::http::error::AppError; + +/// A DNS packet encoding type +#[derive(Debug)] +pub enum DnsMimeType { + /// application/dns-message + Message, + /// application/dns-json + Json, +} + +impl Display for DnsMimeType { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match self { + DnsMimeType::Message => write!(f, "application/dns-message"), + DnsMimeType::Json => write!(f, "application/dns-json"), + } + } +} + +impl DnsMimeType { + /// Turn this mime type to an `Accept` HTTP header value + pub fn to_header_value(&self) -> HeaderValue { + HeaderValue::from_static(match self { + Self::Message => "application/dns-message", + Self::Json => "application/dns-json", + }) + } +} + +#[derive(Debug, Deserialize)] +struct DnsMessageQuery { + dns: String, +} + +// See: https://developers.google.com/speed/public-dns/docs/doh/json#supported_parameters +#[derive(Debug, Deserialize)] +pub struct DnsQuery { + /// Record name to look up, e.g. example.com + pub name: String, + /// Record type, e.g. A/AAAA/TXT, etc. + #[serde(rename = "type")] + pub record_type: Option, + /// Used to disable DNSSEC validation + pub cd: Option, + /// Desired content type. E.g. "application/dns-message" or "application/dns-json" + #[allow(dead_code)] + pub ct: Option, + /// Whether to return DNSSEC entries such as RRSIG, NSEC or NSEC3 + #[serde(rename = "do")] + pub dnssec_ok: Option, + /// Privacy setting for how your IP address is forwarded to authoritative nameservers + #[allow(dead_code)] + pub edns_client_subnet: Option, + /// Some url-safe random characters to pad your messages for privacy (to avoid being fingerprinted by encrytped message length) + #[allow(dead_code)] + pub random_padding: Option, + /// Whether to provide answers for all records up to the root + #[serde(rename = "rd")] + pub recursion_desired: Option, +} + +/// A DNS request encoded in the query string +#[derive(Debug)] +pub struct DnsRequestQuery(pub(crate) DNSRequest, pub(crate) DnsMimeType); + +/// A DNS request encoded in the body +#[derive(Debug)] +pub struct DnsRequestBody(pub(crate) DNSRequest); + +#[async_trait] +impl FromRequestParts for DnsRequestQuery +where + S: Send + Sync, +{ + type Rejection = AppError; + + async fn from_request_parts(parts: &mut Parts, state: &S) -> Result { + let ConnectInfo(src_addr) = ConnectInfo::from_request_parts(parts, state).await?; + + match parts.headers.get(header::ACCEPT) { + Some(content_type) if content_type == "application/dns-message" => { + handle_dns_message_query(parts, state, src_addr).await + } + Some(content_type) if content_type == "application/dns-json" => { + handle_dns_json_query(parts, state, src_addr).await + } + Some(content_type) if content_type == "application/x-javascript" => { + handle_dns_json_query(parts, state, src_addr).await + } + None => handle_dns_message_query(parts, state, src_addr).await, + _ => Err(AppError::with_status(StatusCode::NOT_ACCEPTABLE)), + } + } +} + +#[async_trait] +impl FromRequest for DnsRequestBody +where + S: Send + Sync, +{ + type Rejection = AppError; + + async fn from_request(req: axum::extract::Request, state: &S) -> Result { + let (mut parts, body) = req.into_parts(); + + let ConnectInfo(src_addr) = ConnectInfo::from_request_parts(&mut parts, state).await?; + + let req = Request::from_parts(parts, body); + + let body = Bytes::from_request(req, state) + .await + .map_err(|_| AppError::with_status(StatusCode::INTERNAL_SERVER_ERROR))?; + + let request = decode_request(&body, src_addr)?; + + Ok(DnsRequestBody(request)) + } +} + +async fn handle_dns_message_query( + parts: &mut Parts, + state: &S, + src_addr: SocketAddr, +) -> Result +where + S: Send + Sync, +{ + let Query(params) = Query::::from_request_parts(parts, state).await?; + + let buf = base64_url::decode(params.dns.as_bytes()) + .map_err(|err| AppError::new(StatusCode::BAD_REQUEST, Some(err)))?; + + let request = decode_request(&buf, src_addr)?; + + Ok(DnsRequestQuery(request, DnsMimeType::Message)) +} + +async fn handle_dns_json_query( + parts: &mut Parts, + state: &S, + src_addr: SocketAddr, +) -> Result +where + S: Send + Sync, +{ + let Query(dns_query) = Query::::from_request_parts(parts, state).await?; + + let request = encode_query_as_request(dns_query, src_addr)?; + + Ok(DnsRequestQuery(request, DnsMimeType::Json)) +} + +/// Exposed to make it usable internally... +pub(crate) fn encode_query_as_request( + question: DnsQuery, + src_addr: SocketAddr, +) -> Result { + let query_type = if let Some(record_type) = question.record_type { + record_type + .parse::() + .map(proto::rr::RecordType::from) + .or_else(|_| FromStr::from_str(&record_type.to_uppercase())) + .map_err(|err| AppError::new(StatusCode::BAD_REQUEST, Some(err)))? + } else { + proto::rr::RecordType::A + }; + + let name = proto::rr::Name::from_utf8(question.name) + .map_err(|err| AppError::new(StatusCode::BAD_REQUEST, Some(err)))?; + + let query = proto::op::Query::query(name, query_type); + + let mut message = proto::op::Message::new(); + + message + .add_query(query) + .set_message_type(proto::op::MessageType::Query) + .set_op_code(proto::op::OpCode::Query) + .set_checking_disabled(question.cd.unwrap_or(false)) + .set_recursion_desired(question.recursion_desired.unwrap_or(true)) + .set_recursion_available(true) + .set_authentic_data(question.dnssec_ok.unwrap_or(false)); + + // This is kind of a hack, but the only way I can find to + // create a MessageRequest is by decoding a buffer of bytes, + // so we encode the message into a buffer and then decode it + let mut buf = Vec::with_capacity(4096); + let mut encoder = BinEncoder::new(&mut buf); + + message + .emit(&mut encoder) + .map_err(|err| AppError::new(StatusCode::BAD_REQUEST, Some(err)))?; + + let request = decode_request(&buf, src_addr)?; + + Ok(request) +} + +fn decode_request(bytes: &[u8], src_addr: SocketAddr) -> Result { + let mut decoder = BinDecoder::new(bytes); + + match MessageRequest::read(&mut decoder) { + Ok(message) => { + info!("received message {message:?}"); + if message.message_type() != proto::op::MessageType::Query { + return Err(AppError::new( + StatusCode::BAD_REQUEST, + Some("Invalid message type: expected query"), + )); + } + + let request = DNSRequest::new(message, src_addr, Protocol::Https); + + Ok(request) + } + Err(err) => Err(AppError::new( + StatusCode::BAD_REQUEST, + Some(format!("Invalid DNS message: {}", err)), + )), + } +} diff --git a/iroh-dns-server/src/http/doh/response.rs b/iroh-dns-server/src/http/doh/response.rs new file mode 100644 index 00000000000..f4cee805c06 --- /dev/null +++ b/iroh-dns-server/src/http/doh/response.rs @@ -0,0 +1,144 @@ +//! DNS Response + +// This module is mostly copied from +// https://github.com/fission-codes/fission-server/blob/394de877fad021260c69fdb1edd7bb4b2f98108c/fission-core/src/dns.rs + +use anyhow::{anyhow, ensure, Result}; +use hickory_proto as proto; +use serde::{Deserialize, Serialize}; + +#[derive(Debug, Serialize, Deserialize)] +/// JSON representation of a DNS response +/// See: +pub struct DnsResponse { + /// Standard DNS response code + #[serde(rename = "Status")] + pub status: u32, + /// Whether the response was truncated + #[serde(rename = "TC")] + pub tc: bool, + /// Whether recursion was desired + #[serde(rename = "RD")] + pub rd: bool, + /// Whether recursion was available + #[serde(rename = "RA")] + pub ra: bool, + /// Whether the response was validated with DNSSEC + #[serde(rename = "AD")] + pub ad: bool, + /// Whether the client asked to disable DNSSEC validation + #[serde(rename = "CD")] + pub cd: bool, + /// The questions that this request answers + #[serde(rename = "Question")] + pub question: Vec, + /// The answers to the request + #[serde(rename = "Answer")] + #[serde(skip_serializing_if = "Vec::is_empty")] + pub answer: Vec, + /// An optional comment + #[serde(rename = "Comment")] + pub comment: Option, + /// IP Address / scope prefix-length of the client + /// See: + pub edns_client_subnet: Option, +} + +impl DnsResponse { + /// Create a new JSON response from a DNS message + pub fn from_message(message: proto::op::Message) -> Result { + ensure!( + message.message_type() == proto::op::MessageType::Response, + "Expected message type to be response" + ); + + ensure!( + message.query_count() == message.queries().len() as u16, + "Query count mismatch" + ); + + ensure!( + message.answer_count() == message.answers().len() as u16, + "Answer count mismatch" + ); + + let status: u32 = + >::from(message.response_code()) as u32; + + let question: Vec<_> = message + .queries() + .iter() + .map(DohQuestionJson::from_query) + .collect(); + + let answer: Vec<_> = message + .answers() + .iter() + .map(DohRecordJson::from_record) + .collect::>()?; + + Ok(DnsResponse { + status, + tc: message.truncated(), + rd: message.recursion_desired(), + ra: message.recursion_available(), + ad: message.authentic_data(), + cd: message.checking_disabled(), + question, + answer, + comment: None, + edns_client_subnet: None, + }) + } +} + +#[derive(Debug, Serialize, Deserialize)] +/// JSON representation of a DNS question +pub struct DohQuestionJson { + /// FQDN with trailing dot + pub name: String, + /// Standard DNS RR type + #[serde(rename = "type")] + pub question_type: u16, +} + +impl DohQuestionJson { + /// Create a new JSON question from a DNS query + pub fn from_query(query: &proto::op::Query) -> Self { + Self { + name: query.name().to_string(), + question_type: query.query_type().into(), + } + } +} + +#[derive(Debug, Serialize, Deserialize)] +/// JSON representation of a DNS record +pub struct DohRecordJson { + /// FQDN with trailing dot + pub name: String, + /// Standard DNS RR type + #[serde(rename = "type")] + pub record_type: u16, + /// Time-to-live, in seconds + #[serde(rename = "TTL")] + pub ttl: u32, + /// Record data + pub data: String, +} + +impl DohRecordJson { + /// Create a new JSON record from a DNS record + pub fn from_record(record: &proto::rr::Record) -> Result { + let data = record + .data() + .ok_or_else(|| anyhow!("Missing record data"))?; + + Ok(Self { + name: record.name().to_string(), + record_type: record.record_type().into(), + ttl: record.ttl(), + data: data.to_string(), + }) + } +} diff --git a/iroh-dns-server/src/http/error.rs b/iroh-dns-server/src/http/error.rs new file mode 100644 index 00000000000..7f8ab542d85 --- /dev/null +++ b/iroh-dns-server/src/http/error.rs @@ -0,0 +1,101 @@ +use axum::{ + extract::rejection::{ExtensionRejection, QueryRejection}, + http::StatusCode, + response::IntoResponse, + Json, +}; +use serde::{Deserialize, Serialize}; + +pub type AppResult = Result; + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct AppError { + #[serde(with = "serde_status_code")] + status: StatusCode, + detail: Option, +} + +impl Default for AppError { + fn default() -> Self { + Self { + status: StatusCode::INTERNAL_SERVER_ERROR, + detail: None, + } + } +} + +impl AppError { + pub fn with_status(status: StatusCode) -> AppError { + Self { + status, + detail: None, + } + } + + /// Create a new [`AppError`]. + pub fn new(status_code: StatusCode, message: Option) -> AppError { + Self { + status: status_code, + // title: Self::canonical_reason_to_string(&status_code), + detail: message.map(|m| m.to_string()), + } + } +} + +impl IntoResponse for AppError { + fn into_response(self) -> axum::response::Response { + let json = Json(self.clone()); + (self.status, json).into_response() + } +} + +impl From for AppError { + fn from(value: anyhow::Error) -> Self { + Self { + status: StatusCode::INTERNAL_SERVER_ERROR, + detail: Some(value.to_string()), + } + } +} + +impl From for AppError { + fn from(value: QueryRejection) -> Self { + Self::new(StatusCode::BAD_REQUEST, Some(value)) + } +} + +impl From for AppError { + fn from(value: ExtensionRejection) -> Self { + Self::new(StatusCode::BAD_REQUEST, Some(value)) + } +} + +/// Serialize/Deserializer for status codes. +/// +/// This is needed because status code according to JSON API spec must +/// be the status code as a STRING. +/// +/// We could have used http_serde, but it encodes the status code as a NUMBER. +pub mod serde_status_code { + use http::StatusCode; + use serde::{de::Unexpected, Deserialize, Deserializer, Serialize, Serializer}; + + /// Serialize [StatusCode]s. + pub fn serialize(status: &StatusCode, ser: S) -> Result { + String::serialize(&status.as_u16().to_string(), ser) + } + + /// Deserialize [StatusCode]s. + pub fn deserialize<'de, D>(de: D) -> Result + where + D: Deserializer<'de>, + { + let str = String::deserialize(de)?; + StatusCode::from_bytes(str.as_bytes()).map_err(|_| { + serde::de::Error::invalid_value( + Unexpected::Str(str.as_str()), + &"A valid http status code", + ) + }) + } +} diff --git a/iroh-dns-server/src/http/extract.rs b/iroh-dns-server/src/http/extract.rs new file mode 100644 index 00000000000..cff0e9089e5 --- /dev/null +++ b/iroh-dns-server/src/http/extract.rs @@ -0,0 +1 @@ +mod json; diff --git a/iroh-dns-server/src/http/pkarr.rs b/iroh-dns-server/src/http/pkarr.rs new file mode 100644 index 00000000000..a0e81fef09d --- /dev/null +++ b/iroh-dns-server/src/http/pkarr.rs @@ -0,0 +1,52 @@ +use anyhow::Result; +use axum::extract::Path; +use axum::{extract::State, response::IntoResponse}; +use bytes::Bytes; + +use http::{header, StatusCode}; + +use tracing::info; + +use crate::util::PublicKeyBytes; +use crate::{state::AppState, store::PacketSource}; + +use super::error::AppError; + +pub async fn put( + State(state): State, + Path(key): Path, + body: Bytes, +) -> Result { + let key = pkarr::PublicKey::try_from(key.as_str()) + .map_err(|e| AppError::new(StatusCode::BAD_REQUEST, Some(format!("invalid key: {e}"))))?; + let label = &key.to_z32()[..10]; + let signed_packet = pkarr::SignedPacket::from_relay_response(key, body).map_err(|e| { + AppError::new( + StatusCode::BAD_REQUEST, + Some(format!("invalid body payload: {e}")), + ) + })?; + + let updated = state + .store + .insert(signed_packet, PacketSource::PkarrPublish) + .await?; + info!(key = %label, ?updated, "pkarr upsert"); + Ok(StatusCode::NO_CONTENT) +} + +pub async fn get( + State(state): State, + Path(pubkey): Path, +) -> Result { + let pubkey = PublicKeyBytes::from_z32(&pubkey) + .map_err(|e| AppError::new(StatusCode::BAD_REQUEST, Some(format!("invalid key: {e}"))))?; + let signed_packet = state + .store + .get_signed_packet(&pubkey) + .await? + .ok_or_else(|| AppError::with_status(StatusCode::NOT_FOUND))?; + let body = signed_packet.as_relay_request(); + let headers = [(header::CONTENT_TYPE, "application/x-pkarr-signed-packet")]; + Ok((headers, body)) +} diff --git a/iroh-dns-server/src/http/rate_limiting.rs b/iroh-dns-server/src/http/rate_limiting.rs new file mode 100644 index 00000000000..991ff6e3b09 --- /dev/null +++ b/iroh-dns-server/src/http/rate_limiting.rs @@ -0,0 +1,40 @@ +use std::time::Duration; + +use governor::{clock::QuantaInstant, middleware::NoOpMiddleware}; +use tower_governor::{ + governor::GovernorConfigBuilder, key_extractor::PeerIpKeyExtractor, GovernorLayer, +}; + +/// Create the default rate-limiting layer. +/// +/// This spawns a background thread to clean up the rate limiting cache. +pub fn create() -> GovernorLayer<'static, PeerIpKeyExtractor, NoOpMiddleware> { + // Configure rate limiting: + // * allow bursts with up to five requests per IP address + // * replenish one element every two seconds + let governor_conf = GovernorConfigBuilder::default() + // .use_headers() + .per_second(4) + .burst_size(2) + .finish() + .expect("failed to build rate-limiting governor"); + + // The governor layer needs a reference that outlives the layer. + // The tower_governor crate recommends in its examples to use Box::leak here. + // In the unreleased v0.4 of tower_governor this was changed to use an Arc instead. + // https://github.com/benwis/tower-governor/pull/27 + let governor_conf = Box::leak(Box::new(governor_conf)); + + // The governor needs a background task for garbage collection (to clear expired records) + let gc_interval = Duration::from_secs(60); + let governor_limiter = governor_conf.limiter().clone(); + std::thread::spawn(move || loop { + std::thread::sleep(gc_interval); + tracing::debug!("rate limiting storage size: {}", governor_limiter.len()); + governor_limiter.retain_recent(); + }); + + GovernorLayer { + config: &*governor_conf, + } +} diff --git a/iroh-dns-server/src/http/tls.rs b/iroh-dns-server/src/http/tls.rs new file mode 100644 index 00000000000..4e079aefd49 --- /dev/null +++ b/iroh-dns-server/src/http/tls.rs @@ -0,0 +1,184 @@ +use std::{ + borrow::Cow, + io, + path::{Path, PathBuf}, + sync::{Arc, OnceLock}, +}; + +use anyhow::{bail, Context, Result}; +use axum_server::{ + accept::Accept, + tls_rustls::{RustlsAcceptor, RustlsConfig}, +}; +use futures::{future::BoxFuture, FutureExt}; +use serde::{Deserialize, Serialize}; +use tokio::io::{AsyncRead, AsyncWrite}; +use tokio_rustls_acme::{axum::AxumAcceptor, caches::DirCache, AcmeConfig}; +use tokio_stream::StreamExt; +use tracing::{debug, error, info_span, Instrument}; + +/// The mode how SSL certificates should be created. +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, strum::Display)] +#[serde(rename_all = "snake_case")] +pub enum CertMode { + /// Certs are loaded from a the `cert_cache` path + Manual, + /// ACME with LetsEncrypt servers + LetsEncrypt, + /// Create self-signed certificates and store them in the `cert_cache` path + SelfSigned, +} + +impl CertMode { + /// Build the [`TlsAcceptor`] for this mode. + pub(crate) async fn build( + &self, + domains: Vec, + cert_cache: PathBuf, + letsencrypt_contact: Option, + letsencrypt_prod: bool, + ) -> Result { + Ok(match self { + CertMode::Manual => TlsAcceptor::manual(domains, cert_cache).await?, + CertMode::SelfSigned => TlsAcceptor::self_signed(domains).await?, + CertMode::LetsEncrypt => { + let contact = + letsencrypt_contact.context("contact is required for letsencrypt cert mode")?; + TlsAcceptor::letsencrypt(domains, &contact, letsencrypt_prod, cert_cache)? + } + }) + } +} + +/// TLS Certificate Authority acceptor. +#[derive(Clone)] +pub enum TlsAcceptor { + LetsEncrypt(AxumAcceptor), + Manual(RustlsAcceptor), +} + +impl Accept + for TlsAcceptor +{ + type Stream = tokio_rustls::server::TlsStream; + type Service = S; + type Future = BoxFuture<'static, io::Result<(Self::Stream, Self::Service)>>; + + fn accept(&self, stream: I, service: S) -> Self::Future { + match self { + Self::LetsEncrypt(a) => a.accept(stream, service).boxed(), + Self::Manual(a) => a.accept(stream, service).boxed(), + } + } +} + +impl TlsAcceptor { + async fn self_signed(domains: Vec) -> Result { + let tls_cert = rcgen::generate_simple_self_signed(domains)?; + let config = RustlsConfig::from_der( + vec![tls_cert.serialize_der()?], + tls_cert.serialize_private_key_der(), + ) + .await?; + let acceptor = RustlsAcceptor::new(config); + Ok(Self::Manual(acceptor)) + } + + async fn manual(domains: Vec, dir: PathBuf) -> Result { + let config = rustls::ServerConfig::builder() + .with_safe_defaults() + .with_no_client_auth(); + if domains.len() != 1 { + bail!("Multiple domains in manual mode are not supported"); + } + let keyname = escape_hostname(&domains[0]); + let cert_path = dir.join(format!("{keyname}.crt")); + let key_path = dir.join(format!("{keyname}.key")); + + let (certs, secret_key) = tokio::task::spawn_blocking(move || { + let certs = load_certs(cert_path)?; + let key = load_secret_key(key_path)?; + anyhow::Ok((certs, key)) + }) + .await??; + + let config = config.with_single_cert(certs, secret_key)?; + let config = RustlsConfig::from_config(Arc::new(config)); + let acceptor = RustlsAcceptor::new(config); + Ok(Self::Manual(acceptor)) + } + + fn letsencrypt( + domains: Vec, + contact: &str, + is_production: bool, + dir: PathBuf, + ) -> Result { + let config = rustls::ServerConfig::builder() + .with_safe_defaults() + .with_no_client_auth(); + let mut state = AcmeConfig::new(domains) + .contact([format!("mailto:{contact}")]) + .cache_option(Some(DirCache::new(dir))) + .directory_lets_encrypt(is_production) + .state(); + + let config = config.with_cert_resolver(state.resolver()); + let acceptor = state.acceptor(); + + tokio::spawn( + async move { + loop { + match state.next().await.unwrap() { + Ok(ok) => debug!("acme event: {:?}", ok), + Err(err) => error!("error: {:?}", err), + } + } + } + .instrument(info_span!("acme")), + ); + let config = Arc::new(config); + let acceptor = AxumAcceptor::new(acceptor, config); + Ok(Self::LetsEncrypt(acceptor)) + } +} + +fn load_certs(filename: impl AsRef) -> Result> { + let certfile = std::fs::File::open(filename).context("cannot open certificate file")?; + let mut reader = std::io::BufReader::new(certfile); + + let certs = rustls_pemfile::certs(&mut reader)? + .iter() + .map(|v| rustls::Certificate(v.clone())) + .collect(); + + Ok(certs) +} + +fn load_secret_key(filename: impl AsRef) -> Result { + let keyfile = std::fs::File::open(filename.as_ref()).context("cannot open secret key file")?; + let mut reader = std::io::BufReader::new(keyfile); + + loop { + match rustls_pemfile::read_one(&mut reader).context("cannot parse secret key .pem file")? { + Some(rustls_pemfile::Item::RSAKey(key)) => return Ok(rustls::PrivateKey(key)), + Some(rustls_pemfile::Item::PKCS8Key(key)) => return Ok(rustls::PrivateKey(key)), + Some(rustls_pemfile::Item::ECKey(key)) => return Ok(rustls::PrivateKey(key)), + None => break, + _ => {} + } + } + + bail!( + "no keys found in {} (encrypted keys not supported)", + filename.as_ref().display() + ); +} + +static UNSAFE_HOSTNAME_CHARACTERS: OnceLock = OnceLock::new(); + +fn escape_hostname(hostname: &str) -> Cow<'_, str> { + let regex = UNSAFE_HOSTNAME_CHARACTERS + .get_or_init(|| regex::Regex::new(r"[^a-zA-Z0-9-\.]").expect("valid regex")); + regex.replace_all(hostname, "") +} diff --git a/iroh-dns-server/src/lib.rs b/iroh-dns-server/src/lib.rs new file mode 100644 index 00000000000..2374b86e027 --- /dev/null +++ b/iroh-dns-server/src/lib.rs @@ -0,0 +1,73 @@ +//! A DNS server and pkarr relay + +#![deny(missing_docs, rustdoc::broken_intra_doc_links)] + +pub mod config; +pub mod dns; +pub mod http; +pub mod metrics; +pub mod server; +pub mod state; +mod store; +mod util; + +#[cfg(test)] +mod tests { + use std::net::SocketAddr; + + use anyhow::Result; + use hickory_resolver::{ + config::{NameServerConfig, Protocol, ResolverConfig}, + AsyncResolver, + }; + use iroh_net::{ + discovery::pkarr_publish::PkarrRelayClient, + dns::{ + node_info::{lookup_by_id, NodeInfo}, + DnsResolver, + }, + key::SecretKey, + }; + use url::Url; + + use crate::server::Server; + + #[tokio::test] + async fn integration_smoke() -> Result<()> { + tracing_subscriber::fmt::init(); + let (server, nameserver, http_url) = Server::spawn_for_tests().await?; + + let pkarr_relay = { + let mut url = http_url.clone(); + url.set_path("/pkarr"); + url + }; + + let origin = "irohdns.example."; + + let secret_key = SecretKey::generate(); + let node_id = secret_key.public(); + let relay_url: Url = "https://relay.example.".parse()?; + let pkarr = PkarrRelayClient::new(pkarr_relay); + let node_info = NodeInfo::new(node_id, Some(relay_url.clone())); + let signed_packet = node_info.to_pkarr_signed_packet(&secret_key, 30)?; + + pkarr.publish(&signed_packet).await?; + + let resolver = test_resolver(nameserver); + let res = lookup_by_id(&resolver, &node_id, origin).await?; + + assert_eq!(res.node_id, node_id); + assert_eq!(res.info.relay_url.map(Url::from), Some(relay_url)); + + server.shutdown().await?; + Ok(()) + } + + fn test_resolver(nameserver: SocketAddr) -> DnsResolver { + let mut config = ResolverConfig::new(); + let nameserver_config = NameServerConfig::new(nameserver, Protocol::Udp); + config.add_name_server(nameserver_config); + AsyncResolver::tokio(config, Default::default()) + } +} diff --git a/iroh-dns-server/src/main.rs b/iroh-dns-server/src/main.rs new file mode 100644 index 00000000000..6a7f88d6738 --- /dev/null +++ b/iroh-dns-server/src/main.rs @@ -0,0 +1,36 @@ +#![allow(unused_imports)] + +use anyhow::Result; +use axum::{routing::get, Router}; +use clap::Parser; +use futures::{Future, FutureExt}; +use iroh_dns_server::{ + config::Config, metrics::init_metrics, server::run_with_config_until_ctrl_c, +}; +use std::net::{Ipv4Addr, SocketAddr}; +use std::path::PathBuf; +use tokio::task::JoinSet; +use tokio_util::sync::CancellationToken; +use tracing::{debug, debug_span, error, error_span, Instrument, Span}; + +#[derive(Parser, Debug)] +struct Cli { + /// Path to config file + #[clap(short, long)] + config: Option, +} + +#[tokio::main] +async fn main() -> Result<()> { + tracing_subscriber::fmt::init(); + let args = Cli::parse(); + + let config = if let Some(path) = args.config { + Config::load(path).await? + } else { + Config::default() + }; + + init_metrics(); + run_with_config_until_ctrl_c(config).await +} diff --git a/iroh-dns-server/src/metrics.rs b/iroh-dns-server/src/metrics.rs new file mode 100644 index 00000000000..7b07ac07a08 --- /dev/null +++ b/iroh-dns-server/src/metrics.rs @@ -0,0 +1,64 @@ +//! Metrics support for the server + +use iroh_metrics::core::{Core, Counter, Metric}; +use struct_iterable::Iterable; + +/// Metrics for iroh-dns-server +#[derive(Debug, Clone, Iterable)] +#[allow(missing_docs)] +pub struct Metrics { + pub pkarr_publish_update: Counter, + pub pkarr_publish_noop: Counter, + pub pkarr_publish_error: Counter, + pub dns_requests: Counter, + pub dns_requests_udp: Counter, + pub dns_requests_https: Counter, + pub dns_lookup_success: Counter, + pub dns_lookup_notfound: Counter, + pub dns_lookup_error: Counter, + pub http_requests: Counter, + pub http_requests_success: Counter, + pub http_requests_error: Counter, + pub http_requests_duration_ms: Counter, + pub store_packets_inserted: Counter, + pub store_packets_removed: Counter, + pub store_packets_updated: Counter, +} + +impl Default for Metrics { + fn default() -> Self { + Self { + pkarr_publish_update: Counter::new("Number of pkarr relay puts that updated the state"), + pkarr_publish_noop: Counter::new( + "Number of pkarr relay puts that did not update the state", + ), + pkarr_publish_error: Counter::new("Number of pkarr relay puts that failed"), + dns_requests: Counter::new("DNS requests (total)"), + dns_requests_udp: Counter::new("DNS requests via UDP"), + dns_requests_https: Counter::new("DNS requests via HTTPS (DoH)"), + dns_lookup_success: Counter::new("DNS lookup responses with at least one answer"), + dns_lookup_notfound: Counter::new("DNS lookup responses with no answers"), + dns_lookup_error: Counter::new("DNS lookup responses which failed"), + http_requests: Counter::new("Number of HTTP requests"), + http_requests_success: Counter::new("Number of HTTP requests with a 2xx status code"), + http_requests_error: Counter::new("Number of HTTP requests with a non-2xx status code"), + http_requests_duration_ms: Counter::new("Total duration of all HTTP requests"), + store_packets_inserted: Counter::new("Signed packets inserted into the store"), + store_packets_removed: Counter::new("Signed packets removed from the store"), + store_packets_updated: Counter::new("Number of updates to existing packets"), + } + } +} + +impl Metric for Metrics { + fn name() -> &'static str { + "dns_server" + } +} + +/// Init the metrics collection core. +pub fn init_metrics() { + Core::init(|reg, metrics| { + metrics.insert(Metrics::new(reg)); + }); +} diff --git a/iroh-dns-server/src/server.rs b/iroh-dns-server/src/server.rs new file mode 100644 index 00000000000..2b952a8c51e --- /dev/null +++ b/iroh-dns-server/src/server.rs @@ -0,0 +1,107 @@ +//! The main server which combines the DNS and HTTP(S) servers. + +use anyhow::Result; +use iroh_metrics::metrics::start_metrics_server; +use tracing::info; + +use crate::{ + config::Config, + dns::{DnsHandler, DnsServer}, + http::HttpServer, + state::AppState, + store::ZoneStore, +}; + +/// Spawn the server and run until the `Ctrl-C` signal is received, then shutdown. +pub async fn run_with_config_until_ctrl_c(config: Config) -> Result<()> { + let store = ZoneStore::persistent(Config::signed_packet_store_path()?)?; + let server = Server::spawn(config, store).await?; + tokio::signal::ctrl_c().await?; + info!("shutdown"); + server.shutdown().await?; + Ok(()) +} + +/// The iroh-dns server. +pub struct Server { + http_server: HttpServer, + dns_server: DnsServer, + metrics_task: tokio::task::JoinHandle>, +} + +impl Server { + /// Spawn the server. + /// + /// This will spawn several background tasks: + /// * A DNS server task + /// * A HTTP server task, if `config.http` is not empty + /// * A HTTPS server task, if `config.https` is not empty + pub async fn spawn(config: Config, store: ZoneStore) -> Result { + let dns_handler = DnsHandler::new(store.clone(), &config.dns)?; + + let state = AppState { store, dns_handler }; + + let metrics_addr = config.metrics_addr(); + let metrics_task = tokio::task::spawn(async move { + if let Some(addr) = metrics_addr { + start_metrics_server(addr).await?; + } + Ok(()) + }); + let http_server = HttpServer::spawn(config.http, config.https, state.clone()).await?; + let dns_server = DnsServer::spawn(config.dns, state.dns_handler.clone()).await?; + Ok(Self { + http_server, + dns_server, + metrics_task, + }) + } + + /// Cancel the server tasks and wait for all tasks to complete. + pub async fn shutdown(self) -> Result<()> { + self.metrics_task.abort(); + let (res1, res2) = tokio::join!(self.dns_server.shutdown(), self.http_server.shutdown(),); + res1?; + res2?; + Ok(()) + } + + /// Wait for all tasks to complete. + /// + /// This will run forever unless all tasks close with an error, or `Self::cancel` is called. + pub async fn run_until_error(self) -> Result<()> { + tokio::select! { + res = self.dns_server.run_until_done() => res?, + res = self.http_server.run_until_done() => res?, + } + self.metrics_task.abort(); + Ok(()) + } + + /// Spawn a server suitable for testing. + /// + /// This will run the DNS and HTTP servers, but not the HTTPS server. + /// + /// It returns the server handle, the [`SocketAddr`] of the DNS server and the [`Url`] of the + /// HTTP server. + #[cfg(test)] + pub async fn spawn_for_tests() -> Result<(Self, std::net::SocketAddr, url::Url)> { + use crate::config::MetricsConfig; + use std::net::{IpAddr, Ipv4Addr}; + + let mut config = Config::default(); + config.dns.port = 0; + config.dns.bind_addr = Some(IpAddr::V4(Ipv4Addr::LOCALHOST)); + config.http.as_mut().unwrap().port = 0; + config.http.as_mut().unwrap().bind_addr = Some(IpAddr::V4(Ipv4Addr::LOCALHOST)); + config.https = None; + config.metrics = Some(MetricsConfig::disabled()); + + let store = ZoneStore::in_memory()?; + let server = Self::spawn(config, store).await?; + let dns_addr = server.dns_server.local_addr(); + let http_addr = server.http_server.http_addr().expect("http is set"); + let http_url = format!("http://{http_addr}").parse()?; + Ok((server, dns_addr, http_url)) + } +} diff --git a/iroh-dns-server/src/state.rs b/iroh-dns-server/src/state.rs new file mode 100644 index 00000000000..9063d40b3d0 --- /dev/null +++ b/iroh-dns-server/src/state.rs @@ -0,0 +1,12 @@ +//! Shared state and store for the iroh-dns-server + +use crate::{dns::DnsHandler, store::ZoneStore}; + +/// The shared app state. +#[derive(Clone)] +pub struct AppState { + /// The pkarr DNS store + pub store: ZoneStore, + /// Handler for DNS requests + pub dns_handler: DnsHandler, +} diff --git a/iroh-dns-server/src/store.rs b/iroh-dns-server/src/store.rs new file mode 100644 index 00000000000..5877d009066 --- /dev/null +++ b/iroh-dns-server/src/store.rs @@ -0,0 +1,190 @@ +//! Pkarr packet store used to resolve DNS queries. + +use std::{collections::BTreeMap, num::NonZeroUsize, path::Path, sync::Arc}; + +use anyhow::Result; +use hickory_proto::rr::{Name, RecordSet, RecordType, RrKey}; +use iroh_metrics::inc; +use lru::LruCache; +use parking_lot::Mutex; +use pkarr::SignedPacket; + +use crate::{ + metrics::Metrics, + util::{signed_packet_to_hickory_records_without_origin, PublicKeyBytes}, +}; + +use self::signed_packets::SignedPacketStore; + +mod signed_packets; + +/// Cache up to 1 million pkarr zones by default +pub const DEFAULT_CACHE_CAPACITY: usize = 1024 * 1024; + +/// Where a new pkarr packet comes from +pub enum PacketSource { + /// Received via HTTPS relay PUT + PkarrPublish, +} + +/// A store for pkarr signed packets. +/// +/// Packets are stored in the persistent [`SignedPacketStore`], and cached on-demand in an in-memory LRU +/// cache used for resolving DNS queries. +#[derive(Debug, Clone)] +pub struct ZoneStore { + cache: Arc>, + store: Arc, +} + +impl ZoneStore { + /// Create a persistent store + pub fn persistent(path: impl AsRef) -> Result { + let packet_store = SignedPacketStore::persistent(path)?; + Ok(Self::new(packet_store)) + } + + /// Create an in-memory store. + pub fn in_memory() -> Result { + let packet_store = SignedPacketStore::in_memory()?; + Ok(Self::new(packet_store)) + } + + /// Create a new zone store. + pub fn new(store: SignedPacketStore) -> Self { + let zone_cache = ZoneCache::new(DEFAULT_CACHE_CAPACITY); + Self { + store: Arc::new(store), + cache: Arc::new(Mutex::new(zone_cache)), + } + } + + /// Resolve a DNS query. + // allow unused async: this will be async soon. + #[allow(clippy::unused_async)] + pub async fn resolve( + &self, + pubkey: &PublicKeyBytes, + name: &Name, + record_type: RecordType, + ) -> Result>> { + if let Some(rset) = self.cache.lock().resolve(pubkey, name, record_type) { + return Ok(Some(rset)); + } + + if let Some(packet) = self.store.get(pubkey)? { + return self + .cache + .lock() + .insert_and_resolve(&packet, name, record_type); + }; + + // This would be where mainline discovery could be added. + + Ok(None) + } + + /// Get the latest signed packet for a pubkey. + // allow unused async: this will be async soon. + #[allow(clippy::unused_async)] + pub async fn get_signed_packet(&self, pubkey: &PublicKeyBytes) -> Result> { + self.store.get(pubkey) + } + + /// Insert a signed packet into the cache and the store. + /// + /// Returns whether this produced an update, i.e. whether the packet is the newest for its + /// pubkey. + // allow unused async: this will be async soon. + #[allow(clippy::unused_async)] + pub async fn insert(&self, signed_packet: SignedPacket, _source: PacketSource) -> Result { + let pubkey = PublicKeyBytes::from_signed_packet(&signed_packet); + if self.store.upsert(signed_packet)? { + inc!(Metrics, pkarr_publish_update); + self.cache.lock().remove(&pubkey); + Ok(true) + } else { + inc!(Metrics, pkarr_publish_noop); + Ok(false) + } + } +} + +#[derive(Debug)] +struct ZoneCache { + cache: LruCache, +} + +impl ZoneCache { + fn new(cap: usize) -> Self { + let cache = LruCache::new(NonZeroUsize::new(cap).expect("capacity must be larger than 0")); + Self { cache } + } + + fn resolve( + &mut self, + pubkey: &PublicKeyBytes, + name: &Name, + record_type: RecordType, + ) -> Option> { + self.cache + .get(pubkey) + .and_then(|zone| zone.resolve(name, record_type)) + } + + fn insert_and_resolve( + &mut self, + signed_packet: &SignedPacket, + name: &Name, + record_type: RecordType, + ) -> Result>> { + let pubkey = PublicKeyBytes::from_signed_packet(signed_packet); + self.insert(signed_packet)?; + Ok(self.resolve(&pubkey, name, record_type)) + } + + fn insert(&mut self, signed_packet: &SignedPacket) -> Result<()> { + let pubkey = PublicKeyBytes::from_signed_packet(signed_packet); + if self + .cache + .peek(&pubkey) + .map(|old| old.is_newer_than(signed_packet)) + .unwrap_or(false) + { + return Ok(()); + } + self.cache + .put(pubkey, CachedZone::from_signed_packet(signed_packet)?); + Ok(()) + } + + fn remove(&mut self, pubkey: &PublicKeyBytes) { + self.cache.pop(pubkey); + } +} + +#[derive(Debug)] +struct CachedZone { + timestamp: u64, + records: BTreeMap>, +} + +impl CachedZone { + fn from_signed_packet(signed_packet: &SignedPacket) -> Result { + let (_label, records) = + signed_packet_to_hickory_records_without_origin(signed_packet, |_| true)?; + Ok(Self { + records, + timestamp: *signed_packet.timestamp(), + }) + } + + fn is_newer_than(&self, signed_packet: &SignedPacket) -> bool { + self.timestamp > *signed_packet.timestamp() + } + + fn resolve(&self, name: &Name, record_type: RecordType) -> Option> { + let key = RrKey::new(name.into(), record_type); + self.records.get(&key).cloned() + } +} diff --git a/iroh-dns-server/src/store/signed_packets.rs b/iroh-dns-server/src/store/signed_packets.rs new file mode 100644 index 00000000000..1ed03598d2b --- /dev/null +++ b/iroh-dns-server/src/store/signed_packets.rs @@ -0,0 +1,98 @@ +use std::path::Path; + +use anyhow::Result; +use iroh_metrics::inc; +use pkarr::SignedPacket; +use redb::{backends::InMemoryBackend, Database, ReadableTable, TableDefinition}; + +use crate::{metrics::Metrics, util::PublicKeyBytes}; + +pub type SignedPacketsKey = [u8; 32]; +const SIGNED_PACKETS_TABLE: TableDefinition<&SignedPacketsKey, &[u8]> = + TableDefinition::new("signed-packets-1"); + +#[derive(Debug)] +pub struct SignedPacketStore { + db: Database, +} + +impl SignedPacketStore { + pub fn persistent(path: impl AsRef) -> Result { + if let Some(parent) = path.as_ref().parent() { + std::fs::create_dir_all(parent)?; + } + let db = Database::builder().create(path)?; + Self::open(db) + } + + pub fn in_memory() -> Result { + let db = Database::builder().create_with_backend(InMemoryBackend::new())?; + Self::open(db) + } + + pub fn open(db: Database) -> Result { + let write_tx = db.begin_write()?; + { + let _table = write_tx.open_table(SIGNED_PACKETS_TABLE)?; + } + write_tx.commit()?; + Ok(Self { db }) + } + + pub fn upsert(&self, packet: SignedPacket) -> Result { + let key = PublicKeyBytes::from_signed_packet(&packet); + let tx = self.db.begin_write()?; + let mut replaced = false; + { + let mut table = tx.open_table(SIGNED_PACKETS_TABLE)?; + if let Some(existing) = get_packet(&table, &key)? { + if existing.more_recent_than(&packet) { + return Ok(false); + } else { + replaced = true; + } + } + let value = packet.as_bytes(); + table.insert(key.as_bytes(), &value[..])?; + } + tx.commit()?; + if replaced { + inc!(Metrics, store_packets_updated); + } else { + inc!(Metrics, store_packets_inserted); + } + Ok(true) + } + + pub fn get(&self, key: &PublicKeyBytes) -> Result> { + let tx = self.db.begin_read()?; + let table = tx.open_table(SIGNED_PACKETS_TABLE)?; + get_packet(&table, key) + } + + pub fn remove(&self, key: &PublicKeyBytes) -> Result { + let tx = self.db.begin_write()?; + let updated = { + let mut table = tx.open_table(SIGNED_PACKETS_TABLE)?; + let did_remove = table.remove(key.as_bytes())?.is_some(); + #[allow(clippy::let_and_return)] + did_remove + }; + tx.commit()?; + if updated { + inc!(Metrics, store_packets_removed) + } + Ok(updated) + } +} + +fn get_packet( + table: &impl ReadableTable<&'static SignedPacketsKey, &'static [u8]>, + key: &PublicKeyBytes, +) -> Result> { + let Some(row) = table.get(key.as_ref())? else { + return Ok(None); + }; + let packet = SignedPacket::from_bytes(row.value().to_vec().into(), false)?; + Ok(Some(packet)) +} diff --git a/iroh-dns-server/src/util.rs b/iroh-dns-server/src/util.rs new file mode 100644 index 00000000000..0b64f34b2ec --- /dev/null +++ b/iroh-dns-server/src/util.rs @@ -0,0 +1,151 @@ +use core::fmt; +use std::{ + collections::{btree_map, BTreeMap}, + str::FromStr, + sync::Arc, +}; + +use anyhow::{anyhow, Result}; +use hickory_proto::{ + op::Message, + rr::{ + domain::{IntoLabel, Label}, + Name, Record, RecordSet, RecordType, RrKey, + }, + serialize::binary::BinDecodable, +}; +use pkarr::SignedPacket; + +#[derive(derive_more::From, derive_more::Into, Eq, PartialEq, Ord, PartialOrd, Hash)] +pub struct PublicKeyBytes([u8; 32]); + +impl PublicKeyBytes { + pub fn from_z32(s: &str) -> Result { + let bytes = z32::decode(s.as_bytes())?; + let bytes: [u8; 32] = bytes.try_into().map_err(|_| anyhow!("invalid length"))?; + Ok(Self(bytes)) + } + + pub fn to_z32(&self) -> String { + z32::encode(&self.0) + } + + pub fn to_bytes(&self) -> [u8; 32] { + self.0 + } + + pub fn as_bytes(&self) -> &[u8; 32] { + &self.0 + } + + pub fn from_signed_packet(packet: &SignedPacket) -> Self { + Self(packet.public_key().to_bytes()) + } +} + +impl fmt::Display for PublicKeyBytes { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.to_z32()) + } +} + +impl fmt::Debug for PublicKeyBytes { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "PublicKeyBytes({})", self.to_z32()) + } +} + +impl From for PublicKeyBytes { + fn from(value: pkarr::PublicKey) -> Self { + Self(value.to_bytes()) + } +} + +impl TryFrom for pkarr::PublicKey { + type Error = anyhow::Error; + fn try_from(value: PublicKeyBytes) -> Result { + pkarr::PublicKey::try_from(value.0).map_err(anyhow::Error::from) + } +} + +impl FromStr for PublicKeyBytes { + type Err = anyhow::Error; + fn from_str(s: &str) -> Result { + Self::from_z32(s) + } +} + +impl AsRef<[u8; 32]> for PublicKeyBytes { + fn as_ref(&self) -> &[u8; 32] { + &self.0 + } +} + +pub fn signed_packet_to_hickory_message(signed_packet: &SignedPacket) -> Result { + let encoded = signed_packet.encoded_packet(); + let message = Message::from_bytes(&encoded)?; + Ok(message) +} + +pub fn signed_packet_to_hickory_records_without_origin( + signed_packet: &SignedPacket, + filter: impl Fn(&Record) -> bool, +) -> Result<(Label, BTreeMap>)> { + let common_zone = Label::from_utf8(&signed_packet.public_key().to_z32())?; + let mut message = signed_packet_to_hickory_message(signed_packet)?; + let answers = message.take_answers(); + let mut output: BTreeMap> = BTreeMap::new(); + for mut record in answers.into_iter() { + // disallow SOA and NS records + if matches!(record.record_type(), RecordType::SOA | RecordType::NS) { + continue; + } + // expect the z32 encoded pubkey as root name + let name = record.name(); + if name.num_labels() < 1 { + continue; + } + let zone = name.iter().last().unwrap().into_label()?; + if zone != common_zone { + continue; + } + if !filter(&record) { + continue; + } + + let name_without_zone = + Name::from_labels(name.iter().take(name.num_labels() as usize - 1))?; + record.set_name(name_without_zone); + + let rrkey = RrKey::new(record.name().into(), record.record_type()); + match output.entry(rrkey) { + btree_map::Entry::Vacant(e) => { + let set: RecordSet = record.into(); + e.insert(Arc::new(set)); + } + btree_map::Entry::Occupied(mut e) => { + let set = e.get_mut(); + let serial = set.serial(); + // safe because we just created the arc and are sync iterating + Arc::get_mut(set).unwrap().insert(record, serial); + } + } + } + Ok((common_zone, output)) +} + +pub fn record_set_append_origin( + input: &RecordSet, + origin: &Name, + serial: u32, +) -> Result { + let new_name = input.name().clone().append_name(origin)?; + let mut output = RecordSet::new(&new_name, input.record_type(), serial); + // TODO: less clones + for record in input.records_without_rrsigs() { + let mut record = record.clone(); + record.set_name(new_name.clone()); + output.insert(record, serial); + } + Ok(output) +} diff --git a/iroh-net/Cargo.toml b/iroh-net/Cargo.toml index 9153d0d885c..408e43f049a 100644 --- a/iroh-net/Cargo.toml +++ b/iroh-net/Cargo.toml @@ -65,7 +65,7 @@ thiserror = "1" time = "0.3.20" tokio = { version = "1", features = ["io-util", "macros", "sync", "rt", "net", "fs", "io-std", "signal", "process"] } tokio-rustls = { version = "0.24" } -tokio-rustls-acme = { version = "0.2" } +tokio-rustls-acme = { version = "0.3" } tokio-util = { version = "0.7", features = ["io-util", "io", "codec"] } tracing = "0.1" url = { version = "2.4", features = ["serde"] } From 22a01d13268ebedebdd162a37a4920d4e34b8dbc Mon Sep 17 00:00:00 2001 From: "Franz Heinzmann (Frando)" Date: Mon, 15 Apr 2024 11:25:51 +0200 Subject: [PATCH 66/66] chore: fix iroh-dns-server cargo.toml --- iroh-dns-server/Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iroh-dns-server/Cargo.toml b/iroh-dns-server/Cargo.toml index a1fd9997122..96afd9d4bf7 100644 --- a/iroh-dns-server/Cargo.toml +++ b/iroh-dns-server/Cargo.toml @@ -1,11 +1,11 @@ [package] name = "iroh-dns-server" -version = "0.1.0" +version = "0.13.0" edition = "2021" description = "A pkarr relay and DNS server" license = "MIT OR Apache-2.0" authors = ["Frando ", "n0 team"] -repository = "https://github.com/n0-computer/iroh-dns-server" +repository = "https://github.com/n0-computer/iroh" keywords = ["networking", "pkarr", "dns", "dns-server", "iroh"] readme = "README.md"