From 4af11f1e194eb894dba8a62f320553c9aef46147 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 22 Mar 2024 13:12:48 +0100
Subject: [PATCH 01/25] refactor(iroh-net): stop using the main futures crate

---
 Cargo.lock                                 | 18 +++++++++++++++++-
 iroh-net/Cargo.toml                        |  5 ++++-
 iroh-net/examples/connect-unreliable.rs    |  2 +-
 iroh-net/examples/connect.rs               |  2 +-
 iroh-net/examples/listen-unreliable.rs     |  2 +-
 iroh-net/examples/listen.rs                |  2 +-
 iroh-net/src/bin/iroh-relay.rs             |  3 ++-
 iroh-net/src/dialer.rs                     | 10 +++++-----
 iroh-net/src/discovery.rs                  | 22 +++++++++++-----------
 iroh-net/src/dns.rs                        |  2 +-
 iroh-net/src/magic_endpoint.rs             |  4 ++--
 iroh-net/src/magicsock.rs                  |  8 ++++----
 iroh-net/src/magicsock/relay_actor.rs      |  4 ++--
 iroh-net/src/magicsock/timer.rs            |  2 +-
 iroh-net/src/magicsock/udp_conn.rs         |  3 +--
 iroh-net/src/net/netmon.rs                 |  8 ++++----
 iroh-net/src/net/netmon/actor.rs           |  4 ++--
 iroh-net/src/portmapper.rs                 |  2 +-
 iroh-net/src/portmapper/current_mapping.rs |  7 ++++---
 iroh-net/src/relay/client.rs               |  5 +++--
 iroh-net/src/relay/client_conn.rs          |  6 ++++--
 iroh-net/src/relay/clients.rs              | 14 ++++++--------
 iroh-net/src/relay/codec.rs                |  4 +++-
 iroh-net/src/relay/http/client.rs          | 10 ++++------
 iroh-net/src/relay/http/server.rs          |  3 ++-
 iroh-net/src/relay/server.rs               |  2 +-
 iroh-net/src/util.rs                       |  4 +---
 27 files changed, 89 insertions(+), 69 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index b1fdb4ffe47..fd7b73980a4 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -1597,6 +1597,19 @@ version = "0.3.30"
 source = "registry+https://github.com/rust-lang/crates.io-index"
 checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1"
 
+[[package]]
+name = "futures-lite"
+version = "2.3.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "52527eb5074e35e9339c6b4e8d12600c7128b68fb25dcb9fa9dec18f7c25f3a5"
+dependencies = [
+ "fastrand",
+ "futures-core",
+ "futures-io",
+ "parking",
+ "pin-project-lite",
+]
+
 [[package]]
 name = "futures-macro"
 version = "0.3.30"
@@ -2555,7 +2568,10 @@ dependencies = [
  "derive_more",
  "duct",
  "flume",
- "futures",
+ "futures-buffered",
+ "futures-lite",
+ "futures-sink",
+ "futures-util",
  "governor",
  "hex",
  "hickory-resolver",
diff --git a/iroh-net/Cargo.toml b/iroh-net/Cargo.toml
index 80dd2f0c0f9..e90fe765aeb 100644
--- a/iroh-net/Cargo.toml
+++ b/iroh-net/Cargo.toml
@@ -26,7 +26,10 @@ default-net = "0.20"
 der = { version = "0.7", features = ["alloc", "derive"] }
 derive_more = { version = "1.0.0-beta.1", features = ["debug", "display", "from", "try_into", "deref"] }
 flume = "0.11"
-futures = "0.3.25"
+futures-buffered = "0.2.4"
+futures-lite = "2.3"
+futures-sink = "0.3.25"
+futures-util = "0.3.25"
 governor = "0.6.0"
 hex = "0.4.3"
 hostname = "0.3.1"
diff --git a/iroh-net/examples/connect-unreliable.rs b/iroh-net/examples/connect-unreliable.rs
index fc4ebd1315d..1a8ce141a7e 100644
--- a/iroh-net/examples/connect-unreliable.rs
+++ b/iroh-net/examples/connect-unreliable.rs
@@ -9,7 +9,7 @@ use std::net::SocketAddr;
 
 use anyhow::Context;
 use clap::Parser;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use iroh_base::base32;
 use iroh_net::{
     key::SecretKey,
diff --git a/iroh-net/examples/connect.rs b/iroh-net/examples/connect.rs
index 2ec735cbca7..3d9146ac866 100644
--- a/iroh-net/examples/connect.rs
+++ b/iroh-net/examples/connect.rs
@@ -9,7 +9,7 @@ use std::net::SocketAddr;
 
 use anyhow::Context;
 use clap::Parser;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use iroh_base::base32;
 use iroh_net::relay::RelayUrl;
 use iroh_net::{key::SecretKey, relay::RelayMode, MagicEndpoint, NodeAddr};
diff --git a/iroh-net/examples/listen-unreliable.rs b/iroh-net/examples/listen-unreliable.rs
index 4ed651ff1aa..ce6467cd8ae 100644
--- a/iroh-net/examples/listen-unreliable.rs
+++ b/iroh-net/examples/listen-unreliable.rs
@@ -4,7 +4,7 @@
 //! run this example from the project root:
 //!     $ cargo run --example listen-unreliable
 use anyhow::Context;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use iroh_base::base32;
 use iroh_net::{key::SecretKey, relay::RelayMode, MagicEndpoint};
 use tracing::info;
diff --git a/iroh-net/examples/listen.rs b/iroh-net/examples/listen.rs
index 5049ac43433..a99a3b1f651 100644
--- a/iroh-net/examples/listen.rs
+++ b/iroh-net/examples/listen.rs
@@ -4,7 +4,7 @@
 //! run this example from the project root:
 //!     $ cargo run --example listen
 use anyhow::Context;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use iroh_base::base32;
 use iroh_net::{key::SecretKey, relay::RelayMode, MagicEndpoint};
 use tracing::{debug, info};
diff --git a/iroh-net/src/bin/iroh-relay.rs b/iroh-net/src/bin/iroh-relay.rs
index e7b435cb8c8..5e45cc13358 100644
--- a/iroh-net/src/bin/iroh-relay.rs
+++ b/iroh-net/src/bin/iroh-relay.rs
@@ -8,11 +8,12 @@ use std::{
     path::{Path, PathBuf},
     pin::Pin,
     sync::Arc,
+    future::Future,
 };
 
 use anyhow::{anyhow, bail, Context as _, Result};
 use clap::Parser;
-use futures::{Future, StreamExt};
+use futures_lite::StreamExt;
 use http::{response::Builder as ResponseBuilder, HeaderMap};
 use hyper::body::Incoming;
 use hyper::{Method, Request, Response, StatusCode};
diff --git a/iroh-net/src/dialer.rs b/iroh-net/src/dialer.rs
index 1cd5854a384..f3b5e737b69 100644
--- a/iroh-net/src/dialer.rs
+++ b/iroh-net/src/dialer.rs
@@ -4,7 +4,7 @@ use std::{collections::HashMap, pin::Pin, task::Poll};
 
 use crate::{key::PublicKey, MagicEndpoint, NodeAddr, NodeId};
 use anyhow::anyhow;
-use futures::future::BoxFuture;
+use futures_lite::future::Boxed as BoxFuture;
 use tokio::task::JoinSet;
 use tokio_util::sync::CancellationToken;
 use tracing::error;
@@ -78,14 +78,14 @@ impl Dialer {
                         }
                         None => {
                             error!("no more pending conns available");
-                            futures::future::pending().await
+                            std::future::pending().await
                         }
                     }
                 };
 
                 (node_id, res)
             }
-            true => futures::future::pending().await,
+            true => std::future::pending().await,
         }
     }
 
@@ -95,7 +95,7 @@ impl Dialer {
     }
 }
 
-impl futures::Stream for Dialer {
+impl futures_lite::Stream for Dialer {
     type Item = (PublicKey, anyhow::Result<quinn::Connection>);
 
     fn poll_next(
@@ -117,4 +117,4 @@ impl futures::Stream for Dialer {
 }
 
 /// Future for a pending dial operation
-pub type DialFuture = BoxFuture<'static, (PublicKey, anyhow::Result<quinn::Connection>)>;
+pub type DialFuture = BoxFuture<(PublicKey, anyhow::Result<quinn::Connection>)>;
diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs
index 0bb3791fad8..a947b089fbc 100644
--- a/iroh-net/src/discovery.rs
+++ b/iroh-net/src/discovery.rs
@@ -3,7 +3,7 @@
 use std::time::Duration;
 
 use anyhow::{anyhow, ensure, Result};
-use futures::{stream::BoxStream, StreamExt};
+use futures_lite::stream::{Boxed as BoxStream, StreamExt};
 use iroh_base::node_addr::NodeAddr;
 use tokio::{sync::oneshot, task::JoinHandle};
 use tracing::{debug, error_span, warn, Instrument};
@@ -38,7 +38,7 @@ pub trait Discovery: std::fmt::Debug + Send + Sync {
         &self,
         _endpoint: MagicEndpoint,
         _node_id: NodeId,
-    ) -> Option<BoxStream<'_, Result<DiscoveryItem>>> {
+    ) -> Option<BoxStream<Result<DiscoveryItem>>> {
         None
     }
 }
@@ -99,12 +99,13 @@ impl Discovery for ConcurrentDiscovery {
         &self,
         endpoint: MagicEndpoint,
         node_id: NodeId,
-    ) -> Option<BoxStream<'_, Result<DiscoveryItem>>> {
+    ) -> Option<BoxStream<Result<DiscoveryItem>>> {
         let streams = self
             .services
             .iter()
             .filter_map(|service| service.resolve(endpoint.clone(), node_id));
-        let streams = futures::stream::select_all(streams);
+
+        let streams = futures_buffered::Merge::from_iter(streams);
         Some(Box::pin(streams))
     }
 }
@@ -189,7 +190,7 @@ impl DiscoveryTask {
     fn create_stream(
         ep: &MagicEndpoint,
         node_id: NodeId,
-    ) -> Result<BoxStream<'_, Result<DiscoveryItem>>> {
+    ) -> Result<BoxStream<Result<DiscoveryItem>>> {
         let discovery = ep
             .discovery()
             .ok_or_else(|| anyhow!("No discovery service configured"))?;
@@ -268,7 +269,6 @@ mod tests {
         time::{Duration, SystemTime},
     };
 
-    use futures::{stream, StreamExt};
     use parking_lot::Mutex;
     use rand::Rng;
 
@@ -326,7 +326,7 @@ mod tests {
             &self,
             endpoint: MagicEndpoint,
             node_id: NodeId,
-        ) -> Option<BoxStream<'_, Result<DiscoveryItem>>> {
+        ) -> Option<BoxStream<Result<DiscoveryItem>>> {
             let addr_info = match self.resolve_wrong {
                 false => self.shared.nodes.lock().get(&node_id).cloned(),
                 true => {
@@ -358,9 +358,9 @@ mod tests {
                         );
                         Ok(item)
                     };
-                    stream::once(fut).boxed()
+                    futures_lite::stream::once_future(fut).boxed()
                 }
-                None => stream::empty().boxed(),
+                None => futures_lite::stream::empty().boxed(),
             };
             Some(stream)
         }
@@ -375,8 +375,8 @@ mod tests {
             &self,
             _endpoint: MagicEndpoint,
             _node_id: NodeId,
-        ) -> Option<BoxStream<'_, Result<DiscoveryItem>>> {
-            Some(stream::empty().boxed())
+        ) -> Option<BoxStream<Result<DiscoveryItem>>> {
+            Some(futures_lite::stream::empty().boxed())
         }
     }
 
diff --git a/iroh-net/src/dns.rs b/iroh-net/src/dns.rs
index 60c1d6eabd0..431883edb09 100644
--- a/iroh-net/src/dns.rs
+++ b/iroh-net/src/dns.rs
@@ -100,7 +100,7 @@ pub async fn lookup_ipv4_ipv6<N: IntoName + TryParseIp + Clone>(
     let ipv4 = tokio::time::timeout(timeout, ipv4);
     let ipv6 = tokio::time::timeout(timeout, ipv6);
 
-    let res = futures::future::join(ipv4, ipv6).await;
+    let res = tokio::join!(ipv4, ipv6);
     match res {
         (Ok(Ok(ipv4)), Ok(Ok(ipv6))) => {
             let res = ipv4
diff --git a/iroh-net/src/magic_endpoint.rs b/iroh-net/src/magic_endpoint.rs
index 318ce5da927..df31a5d6acf 100644
--- a/iroh-net/src/magic_endpoint.rs
+++ b/iroh-net/src/magic_endpoint.rs
@@ -4,7 +4,7 @@ use std::{net::SocketAddr, path::PathBuf, sync::Arc, time::Duration};
 
 use anyhow::{anyhow, bail, ensure, Context, Result};
 use derive_more::Debug;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use quinn_proto::VarInt;
 use tokio_util::sync::{CancellationToken, WaitForCancellationFuture};
 use tracing::{debug, trace};
@@ -312,7 +312,7 @@ impl MagicEndpoint {
     ///
     /// To get the current endpoints, drop the stream after the first item was received:
     /// ```
-    /// use futures::StreamExt;
+    /// use futures_lite::StreamExt;
     /// use iroh_net::MagicEndpoint;
     ///
     /// # let rt = tokio::runtime::Builder::new_current_thread().enable_all().build().unwrap();
diff --git a/iroh-net/src/magicsock.rs b/iroh-net/src/magicsock.rs
index 05a250eac79..cae0c9ab8e5 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::{FutureExt, Stream};
+use futures_lite::{FutureExt, Stream};
 use iroh_metrics::{inc, inc_by};
 use quinn::AsyncUdpSocket;
 use rand::{seq::SliceRandom, Rng, SeedableRng};
@@ -852,7 +852,7 @@ impl Inner {
         dst_key: PublicKey,
         msg: &disco::Message,
     ) -> io::Result<bool> {
-        futures::future::poll_fn(move |cx| self.poll_send_disco_message_udp(dst, dst_key, msg, cx))
+        std::future::poll_fn(move |cx| self.poll_send_disco_message_udp(dst, dst_key, msg, cx))
             .await
     }
 
@@ -1723,7 +1723,7 @@ impl Actor {
             return;
         }
         if let Err(err) =
-            futures::future::poll_fn(|cx| self.inner.poll_handle_ping_actions(cx, &mut msgs)).await
+            std::future::poll_fn(|cx| self.inner.poll_handle_ping_actions(cx, &mut msgs)).await
         {
             debug!("failed to send pings: {err:?}");
         }
@@ -2521,7 +2521,7 @@ fn disco_message_sent(msg: &disco::Message) {
 #[cfg(test)]
 pub(crate) mod tests {
     use anyhow::Context;
-    use futures::StreamExt;
+    use futures_lite::StreamExt;
     use iroh_test::CallOnDrop;
     use rand::RngCore;
     use tokio::task::JoinSet;
diff --git a/iroh-net/src/magicsock/relay_actor.rs b/iroh-net/src/magicsock/relay_actor.rs
index eb25b666447..dc3a151e244 100644
--- a/iroh-net/src/magicsock/relay_actor.rs
+++ b/iroh-net/src/magicsock/relay_actor.rs
@@ -1,5 +1,6 @@
 use std::{
     collections::{BTreeMap, HashSet},
+    future::Future,
     net::{IpAddr, SocketAddr},
     sync::{atomic::Ordering, Arc},
     time::{Duration, Instant},
@@ -8,7 +9,6 @@ use std::{
 use anyhow::Context;
 use backoff::backoff::Backoff;
 use bytes::{Bytes, BytesMut};
-use futures::Future;
 use iroh_metrics::{inc, inc_by};
 use tokio::{
     sync::{mpsc, oneshot},
@@ -362,7 +362,7 @@ impl RelayActor {
     }
 
     async fn note_preferred(&self, my_url: &RelayUrl) {
-        futures::future::join_all(self.active_relay.iter().map(|(url, (s, _))| async move {
+        futures_buffered::join_all(self.active_relay.iter().map(|(url, (s, _))| async move {
             let is_preferred = url == my_url;
             s.send(ActiveRelayMessage::NotePreferred(is_preferred))
                 .await
diff --git a/iroh-net/src/magicsock/timer.rs b/iroh-net/src/magicsock/timer.rs
index d437aa852d2..c3f6cbcd4fe 100644
--- a/iroh-net/src/magicsock/timer.rs
+++ b/iroh-net/src/magicsock/timer.rs
@@ -1,6 +1,6 @@
+use std::future::Future;
 use std::time::Duration;
 
-use futures::Future;
 use tokio::sync::mpsc;
 use tokio::task::JoinHandle;
 use tokio::time::{self, Instant};
diff --git a/iroh-net/src/magicsock/udp_conn.rs b/iroh-net/src/magicsock/udp_conn.rs
index 9503b3a6eec..f28ccef80d5 100644
--- a/iroh-net/src/magicsock/udp_conn.rs
+++ b/iroh-net/src/magicsock/udp_conn.rs
@@ -3,11 +3,10 @@ use std::{
     io,
     net::SocketAddr,
     sync::Arc,
-    task::{Context, Poll},
+    task::{ready, Context, Poll},
 };
 
 use anyhow::{bail, Context as _};
-use futures::ready;
 use quinn::AsyncUdpSocket;
 use tokio::io::Interest;
 use tracing::{debug, trace, warn};
diff --git a/iroh-net/src/net/netmon.rs b/iroh-net/src/net/netmon.rs
index 926bae18637..f89dd0ce82e 100644
--- a/iroh-net/src/net/netmon.rs
+++ b/iroh-net/src/net/netmon.rs
@@ -1,7 +1,7 @@
 //! Monitoring of networking interfaces and route changes.
 
 use anyhow::Result;
-use futures::future::BoxFuture;
+use futures_lite::future::Boxed as BoxFuture;
 use tokio::{
     sync::{mpsc, oneshot},
     task::JoinHandle,
@@ -56,7 +56,7 @@ impl Monitor {
     /// Subscribe to network changes.
     pub async fn subscribe<F>(&self, callback: F) -> Result<CallbackToken>
     where
-        F: Fn(bool) -> BoxFuture<'static, ()> + 'static + Sync + Send,
+        F: Fn(bool) -> BoxFuture<()> + 'static + Sync + Send,
     {
         let (s, r) = oneshot::channel();
         self.actor_tx
@@ -85,10 +85,10 @@ impl Monitor {
 
 #[cfg(test)]
 mod tests {
-    use futures::FutureExt;
-
     use super::*;
 
+    use futures_util::FutureExt;
+
     #[tokio::test]
     async fn test_smoke_monitor() {
         let _guard = iroh_test::logging::setup();
diff --git a/iroh-net/src/net/netmon/actor.rs b/iroh-net/src/net/netmon/actor.rs
index f38fd2b3e6d..6838817d734 100644
--- a/iroh-net/src/net/netmon/actor.rs
+++ b/iroh-net/src/net/netmon/actor.rs
@@ -5,7 +5,7 @@ use std::{
 };
 
 use anyhow::Result;
-use futures::future::BoxFuture;
+use futures_lite::future::Boxed as BoxFuture;
 use tokio::sync::{mpsc, oneshot};
 use tracing::{debug, info, trace, warn};
 
@@ -70,7 +70,7 @@ pub(super) struct Actor {
 pub struct CallbackToken(u64);
 
 /// Callbacks that get notified about changes.
-pub(super) type Callback = Box<dyn Fn(bool) -> BoxFuture<'static, ()> + Sync + Send + 'static>;
+pub(super) type Callback = Box<dyn Fn(bool) -> BoxFuture<()> + Sync + Send + 'static>;
 
 pub(super) enum ActorMessage {
     Subscribe(Callback, oneshot::Sender<CallbackToken>),
diff --git a/iroh-net/src/portmapper.rs b/iroh-net/src/portmapper.rs
index 4e76aa8b591..4aa225073e8 100644
--- a/iroh-net/src/portmapper.rs
+++ b/iroh-net/src/portmapper.rs
@@ -7,7 +7,7 @@ use std::{
 };
 
 use anyhow::{anyhow, Result};
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use tokio::sync::{mpsc, oneshot, watch};
 use tracing::{debug, info_span, trace, Instrument};
 
diff --git a/iroh-net/src/portmapper/current_mapping.rs b/iroh-net/src/portmapper/current_mapping.rs
index 917ffe80d4f..0c2273ece31 100644
--- a/iroh-net/src/portmapper/current_mapping.rs
+++ b/iroh-net/src/portmapper/current_mapping.rs
@@ -1,13 +1,13 @@
 //! Holds the current mapping value and ensures that any change is reported accordingly.
 
 use std::{
+    future::Future,
     net::{Ipv4Addr, SocketAddrV4},
     num::NonZeroU16,
     pin::Pin,
     task::Poll,
 };
 
-use futures::Future;
 use iroh_metrics::inc;
 use std::time::Duration;
 use tokio::{sync::watch, time};
@@ -164,7 +164,7 @@ impl<M: Mapping> CurrentMapping<M> {
     }
 }
 
-impl<M: Mapping> futures::Stream for CurrentMapping<M> {
+impl<M: Mapping> futures_lite::Stream for CurrentMapping<M> {
     type Item = Event;
 
     fn poll_next(
@@ -177,8 +177,9 @@ impl<M: Mapping> futures::Stream for CurrentMapping<M> {
 
 #[cfg(test)]
 mod tests {
+    use futures_lite::StreamExt;
+
     use super::*;
-    use futures::StreamExt;
 
     // for testing a mapping is simply an ip, port pair
     type M = (Ipv4Addr, NonZeroU16);
diff --git a/iroh-net/src/relay/client.rs b/iroh-net/src/relay/client.rs
index d9f44b6832c..db739dd11b8 100644
--- a/iroh-net/src/relay/client.rs
+++ b/iroh-net/src/relay/client.rs
@@ -5,8 +5,9 @@ use std::time::Duration;
 
 use anyhow::{anyhow, bail, ensure, Context, Result};
 use bytes::Bytes;
-use futures::stream::Stream;
-use futures::{Sink, SinkExt, StreamExt};
+use futures_lite::{Stream, StreamExt};
+use futures_sink::Sink;
+use futures_util::sink::SinkExt;
 use tokio::io::{AsyncRead, AsyncWrite};
 use tokio::sync::mpsc;
 use tokio_util::codec::{FramedRead, FramedWrite};
diff --git a/iroh-net/src/relay/client_conn.rs b/iroh-net/src/relay/client_conn.rs
index 47c4e65400b..2446668438d 100644
--- a/iroh-net/src/relay/client_conn.rs
+++ b/iroh-net/src/relay/client_conn.rs
@@ -4,7 +4,9 @@ use std::time::Duration;
 
 use anyhow::{Context, Result};
 use bytes::Bytes;
-use futures::{SinkExt, StreamExt};
+use futures_lite::StreamExt;
+use futures_util::SinkExt;
+use tokio::io::{AsyncRead, AsyncWrite};
 use tokio::sync::mpsc;
 use tokio_util::codec::Framed;
 use tokio_util::sync::CancellationToken;
@@ -26,7 +28,7 @@ use super::{
 /// The [`super::server::Server`] side representation of a [`super::client::Client`]'s connection
 #[derive(Debug)]
 pub(crate) struct ClientConnManager {
-    /// Static after construction, process-wide unique counter, incremented each time we accept  
+    /// Static after construction, process-wide unique counter, incremented each time we accept
     pub(crate) conn_num: usize,
 
     // TODO: in the go impl, we have a ptr to the server & use that ptr to update stats
diff --git a/iroh-net/src/relay/clients.rs b/iroh-net/src/relay/clients.rs
index b91f4a066ea..224deceba64 100644
--- a/iroh-net/src/relay/clients.rs
+++ b/iroh-net/src/relay/clients.rs
@@ -1,15 +1,15 @@
 //! based on tailscale/derp/derp_server.go
 //!
 //! The "Server" side of the client. Uses the `ClientConnManager`.
-use crate::key::PublicKey;
 use std::collections::{HashMap, HashSet};
 
-use futures::future::join_all;
-use tokio::sync::mpsc;
+use tokio::{sync::mpsc, task::JoinSet};
 
 use iroh_metrics::inc;
 use tracing::{Instrument, Span};
 
+use crate::key::PublicKey;
+
 use super::{
     client_conn::{ClientConnBuilder, ClientConnManager},
     metrics::Metrics,
@@ -149,13 +149,11 @@ impl Clients {
 
     pub async fn shutdown(&mut self) {
         tracing::trace!("shutting down conn");
-        let mut handles = Vec::new();
+        let mut handles = JoinSet::default();
         for (_, client) in self.inner.drain() {
-            handles.push(tokio::spawn(
-                async move { client.shutdown_await().await }.instrument(Span::current()),
-            ));
+            handles.spawn(async move { client.shutdown_await().await }.instrument(Span::current()));
         }
-        join_all(handles).await;
+        while let Some(_) = handles.join_next().await {}
     }
 
     /// Record that `src` sent or forwarded a packet to `dst`
diff --git a/iroh-net/src/relay/codec.rs b/iroh-net/src/relay/codec.rs
index 670c7096515..efc8ae5b886 100644
--- a/iroh-net/src/relay/codec.rs
+++ b/iroh-net/src/relay/codec.rs
@@ -2,7 +2,9 @@ use std::time::Duration;
 
 use anyhow::{bail, ensure, Context};
 use bytes::{Buf, BufMut, Bytes, BytesMut};
-use futures::{Sink, SinkExt, Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
+use futures_sink::Sink;
+use futures_util::SinkExt;
 use iroh_base::key::PUBLIC_KEY_LENGTH;
 use tokio_util::codec::{Decoder, Encoder};
 
diff --git a/iroh-net/src/relay/http/client.rs b/iroh-net/src/relay/http/client.rs
index e7c3346b72f..5624ca26a1a 100644
--- a/iroh-net/src/relay/http/client.rs
+++ b/iroh-net/src/relay/http/client.rs
@@ -7,7 +7,7 @@ use std::time::Duration;
 
 use anyhow::bail;
 use bytes::Bytes;
-use futures::future::BoxFuture;
+use futures_lite::future::Boxed as BoxFuture;
 use hyper::body::Incoming;
 use hyper::header::UPGRADE;
 use hyper::upgrade::{Parts, Upgraded};
@@ -151,8 +151,7 @@ struct Actor {
     relay_client: Option<(RelayClient, RelayClientReceiver)>,
     is_closed: bool,
     #[debug("address family selector callback")]
-    address_family_selector:
-        Option<Box<dyn Fn() -> BoxFuture<'static, bool> + Send + Sync + 'static>>,
+    address_family_selector: Option<Box<dyn Fn() -> BoxFuture<bool> + Send + Sync + 'static>>,
     conn_gen: usize,
     is_prober: bool,
     server_public_key: Option<PublicKey>,
@@ -193,8 +192,7 @@ pub struct ClientBuilder {
     /// Default is false
     is_preferred: bool,
     /// Default is None
-    address_family_selector:
-        Option<Box<dyn Fn() -> BoxFuture<'static, bool> + Send + Sync + 'static>>,
+    address_family_selector: Option<Box<dyn Fn() -> BoxFuture<bool> + Send + Sync + 'static>>,
     /// Default is false
     is_prober: bool,
     /// Expected PublicKey of the server
@@ -240,7 +238,7 @@ impl ClientBuilder {
     /// work anyway, so we don't artificially delay the connection speed.
     pub fn address_family_selector<S>(mut self, selector: S) -> Self
     where
-        S: Fn() -> BoxFuture<'static, bool> + Send + Sync + 'static,
+        S: Fn() -> BoxFuture<bool> + Send + Sync + 'static,
     {
         self.address_family_selector = Some(Box::new(selector));
         self
diff --git a/iroh-net/src/relay/http/server.rs b/iroh-net/src/relay/http/server.rs
index b151212a1ba..c9238ba1478 100644
--- a/iroh-net/src/relay/http/server.rs
+++ b/iroh-net/src/relay/http/server.rs
@@ -1,4 +1,5 @@
 use std::collections::HashMap;
+use std::future::Future;
 use std::net::SocketAddr;
 use std::pin::Pin;
 use std::sync::Arc;
@@ -6,7 +7,7 @@ use std::sync::Arc;
 use anyhow::{bail, ensure, Context as _, Result};
 use bytes::Bytes;
 use derive_more::Debug;
-use futures::future::{Future, FutureExt};
+use futures_lite::FutureExt;
 use http::response::Builder as ResponseBuilder;
 use hyper::body::Incoming;
 use hyper::header::{HeaderValue, UPGRADE};
diff --git a/iroh-net/src/relay/server.rs b/iroh-net/src/relay/server.rs
index a354fca028d..3dd11080f22 100644
--- a/iroh-net/src/relay/server.rs
+++ b/iroh-net/src/relay/server.rs
@@ -6,7 +6,7 @@ use std::task::{Context, Poll};
 use std::time::Duration;
 
 use anyhow::{Context as _, Result};
-use futures::SinkExt;
+use futures_util::SinkExt;
 use hyper::HeaderMap;
 use iroh_metrics::core::UsageStatsReport;
 use iroh_metrics::{inc, report_usage_stats};
diff --git a/iroh-net/src/util.rs b/iroh-net/src/util.rs
index f1eee679714..0424fbf06a6 100644
--- a/iroh-net/src/util.rs
+++ b/iroh-net/src/util.rs
@@ -6,8 +6,6 @@ use std::{
     task::{Context, Poll},
 };
 
-use futures::FutureExt;
-
 /// A join handle that owns the task it is running, and aborts it when dropped.
 #[derive(Debug, derive_more::Deref)]
 pub struct AbortingJoinHandle<T>(pub tokio::task::JoinHandle<T>);
@@ -22,7 +20,7 @@ impl<T> Future for AbortingJoinHandle<T> {
     type Output = std::result::Result<T, tokio::task::JoinError>;
 
     fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
-        self.0.poll_unpin(cx)
+        Pin::new(&mut self.0).poll(cx)
     }
 }
 

From caef7baf61b317f2afd4f499d79c0498393bf121 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 22 Mar 2024 13:19:59 +0100
Subject: [PATCH 02/25] refactor(iroh-bytes): avoid using futures crate

---
 Cargo.lock                              |  2 +-
 iroh-bytes/Cargo.toml                   |  2 +-
 iroh-bytes/examples/fetch-fsm.rs        |  4 ++--
 iroh-bytes/examples/fetch-stream.rs     |  6 +++---
 iroh-bytes/examples/provide-bytes.rs    |  4 ++--
 iroh-bytes/src/get/db.rs                | 13 +++++++------
 iroh-bytes/src/provider.rs              |  2 +-
 iroh-bytes/src/store/bao_file.rs        |  8 ++++----
 iroh-bytes/src/store/fs.rs              |  6 +++---
 iroh-bytes/src/store/fs/test_support.rs |  2 +-
 iroh-bytes/src/store/fs/tests.rs        |  2 +-
 iroh-bytes/src/store/mem.rs             |  2 +-
 iroh-bytes/src/store/readonly_mem.rs    |  2 +-
 iroh-bytes/src/store/traits.rs          |  4 ++--
 iroh-bytes/src/util/progress.rs         |  3 +--
 15 files changed, 31 insertions(+), 31 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index fd7b73980a4..3d3f6cb76a9 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -2398,8 +2398,8 @@ dependencies = [
  "data-encoding",
  "derive_more",
  "flume",
- "futures",
  "futures-buffered",
+ "futures-lite",
  "genawaiter",
  "hex",
  "http-body 0.4.6",
diff --git a/iroh-bytes/Cargo.toml b/iroh-bytes/Cargo.toml
index f045ab97418..b578c95bed0 100644
--- a/iroh-bytes/Cargo.toml
+++ b/iroh-bytes/Cargo.toml
@@ -23,8 +23,8 @@ chrono = "0.4.31"
 data-encoding = "2.3.3"
 derive_more = { version = "1.0.0-beta.1", features = ["debug", "display", "deref", "deref_mut", "from", "try_into", "into"] }
 flume = "0.11"
-futures = "0.3.25"
 futures-buffered = "0.2.4"
+futures-lite = "2.3"
 genawaiter = { version = "0.99.1", features = ["futures03"] }
 hex = "0.4.3"
 iroh-base = { version = "0.12.0", features = ["redb"], path = "../iroh-base" }
diff --git a/iroh-bytes/examples/fetch-fsm.rs b/iroh-bytes/examples/fetch-fsm.rs
index ac0e7a096f9..2f606e16072 100644
--- a/iroh-bytes/examples/fetch-fsm.rs
+++ b/iroh-bytes/examples/fetch-fsm.rs
@@ -165,10 +165,10 @@ async fn write_collection(initial: AtInitial) -> Result<()> {
 #[derive(Clone)]
 struct MockEventSender;
 
-use futures::future::FutureExt;
+use futures_lite::future::FutureExt;
 
 impl iroh_bytes::provider::EventSender for MockEventSender {
-    fn send(&self, _event: iroh_bytes::provider::Event) -> futures::future::BoxFuture<()> {
+    fn send(&self, _event: iroh_bytes::provider::Event) -> futures_lite::future::Boxed<()> {
         async move {}.boxed()
     }
 }
diff --git a/iroh-bytes/examples/fetch-stream.rs b/iroh-bytes/examples/fetch-stream.rs
index cb01e9d0729..d015dc8f5b5 100644
--- a/iroh-bytes/examples/fetch-stream.rs
+++ b/iroh-bytes/examples/fetch-stream.rs
@@ -12,7 +12,7 @@ use std::io;
 
 use bao_tree::io::fsm::BaoContentItem;
 use bytes::Bytes;
-use futures::{Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use genawaiter::sync::Co;
 use genawaiter::sync::Gen;
 use tokio::io::AsyncWriteExt;
@@ -237,10 +237,10 @@ fn stream_children(initial: AtInitial) -> impl Stream<Item = io::Result<Bytes>>
 #[derive(Clone)]
 struct MockEventSender;
 
-use futures::future::FutureExt;
+use futures_lite::future::FutureExt;
 
 impl iroh_bytes::provider::EventSender for MockEventSender {
-    fn send(&self, _event: iroh_bytes::provider::Event) -> futures::future::BoxFuture<()> {
+    fn send(&self, _event: iroh_bytes::provider::Event) -> futures_lite::future::Boxed<()> {
         async move {}.boxed()
     }
 }
diff --git a/iroh-bytes/examples/provide-bytes.rs b/iroh-bytes/examples/provide-bytes.rs
index 89777d7da45..d5bdaa36187 100644
--- a/iroh-bytes/examples/provide-bytes.rs
+++ b/iroh-bytes/examples/provide-bytes.rs
@@ -110,10 +110,10 @@ async fn main() -> Result<()> {
 #[derive(Clone)]
 struct MockEventSender;
 
-use futures::future::FutureExt;
+use futures_lite::future::FutureExt;
 
 impl iroh_bytes::provider::EventSender for MockEventSender {
-    fn send(&self, _event: iroh_bytes::provider::Event) -> futures::future::BoxFuture<()> {
+    fn send(&self, _event: iroh_bytes::provider::Event) -> futures_lite::future::Boxed<()> {
         async move {}.boxed()
     }
 }
diff --git a/iroh-bytes/src/get/db.rs b/iroh-bytes/src/get/db.rs
index 49134887b47..c804bc24113 100644
--- a/iroh-bytes/src/get/db.rs
+++ b/iroh-bytes/src/get/db.rs
@@ -1,17 +1,18 @@
 //! Functions that use the iroh-bytes protocol in conjunction with a bao store.
 
-use futures::{Future, StreamExt};
+use std::future::Future;
+use std::io;
+
+use futures_lite::StreamExt;
 use iroh_base::hash::Hash;
 use iroh_base::rpc::RpcError;
 use serde::{Deserialize, Serialize};
 
+use crate::hashseq::parse_hash_seq;
 use crate::protocol::RangeSpec;
+use crate::store::BaoBatchWriter;
 use crate::store::BaoBlobSize;
 use crate::store::FallibleProgressBatchWriter;
-use std::io;
-
-use crate::hashseq::parse_hash_seq;
-use crate::store::BaoBatchWriter;
 
 use crate::{
     get::{
@@ -292,7 +293,7 @@ pub async fn blob_info<D: BaoStore>(db: &D, hash: &Hash) -> io::Result<BlobInfo<
 
 /// Like `get_blob_info`, but for multiple hashes
 async fn blob_infos<D: BaoStore>(db: &D, hash_seq: &[Hash]) -> io::Result<Vec<BlobInfo<D>>> {
-    let items = futures::stream::iter(hash_seq)
+    let items = futures_lite::stream::iter(hash_seq)
         .then(|hash| blob_info(db, hash))
         .collect::<Vec<_>>();
     items.await.into_iter().collect()
diff --git a/iroh-bytes/src/provider.rs b/iroh-bytes/src/provider.rs
index 67bfcd1cccb..75d9c81c388 100644
--- a/iroh-bytes/src/provider.rs
+++ b/iroh-bytes/src/provider.rs
@@ -4,7 +4,7 @@ use std::time::Duration;
 
 use anyhow::{Context, Result};
 use bao_tree::io::fsm::{encode_ranges_validated, Outboard};
-use futures::future::BoxFuture;
+use futures_lite::future::Boxed as BoxFuture;
 use iroh_base::rpc::RpcError;
 use iroh_io::stats::{
     SliceReaderStats, StreamWriterStats, TrackingSliceReader, TrackingStreamWriter,
diff --git a/iroh-bytes/src/store/bao_file.rs b/iroh-bytes/src/store/bao_file.rs
index 259f5d37504..3f8aed0be5b 100644
--- a/iroh-bytes/src/store/bao_file.rs
+++ b/iroh-bytes/src/store/bao_file.rs
@@ -767,7 +767,7 @@ pub(crate) fn parse_hash_pair(buf: Bytes) -> io::Result<(blake3::Hash, blake3::H
 
 #[cfg(test)]
 pub mod test_support {
-    use std::{io::Cursor, ops::Range};
+    use std::{future::Future, io::Cursor, ops::Range};
 
     use bao_tree::{
         io::{
@@ -778,7 +778,7 @@ pub mod test_support {
         },
         BlockSize, ChunkRanges,
     };
-    use futures::{Future, Stream, StreamExt};
+    use futures_lite::{Stream, StreamExt};
     use iroh_base::hash::Hash;
     use rand::RngCore;
     use range_collections::RangeSet2;
@@ -897,7 +897,7 @@ pub mod test_support {
             .chunks(mtu)
             .map(Bytes::copy_from_slice)
             .collect::<Vec<_>>();
-        futures::stream::iter(parts).then(move |part| async move {
+        futures_lite::stream::iter(parts).then(move |part| async move {
             tokio::time::sleep(delay).await;
             part
         })
@@ -916,7 +916,7 @@ mod tests {
     use std::sync::Arc;
 
     use bao_tree::{ChunkNum, ChunkRanges};
-    use futures::StreamExt;
+    use futures_lite::StreamExt;
     use tests::test_support::{
         decode_response_into_batch, local, make_wire_data, random_test_data, trickle, validate,
     };
diff --git a/iroh-bytes/src/store/fs.rs b/iroh-bytes/src/store/fs.rs
index 0b4927aba1c..68c3cf74986 100644
--- a/iroh-bytes/src/store/fs.rs
+++ b/iroh-bytes/src/store/fs.rs
@@ -78,14 +78,14 @@ use bao_tree::io::{
     sync::{ReadAt, Size},
 };
 use bytes::Bytes;
-use futures::{channel::oneshot, Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 
 use iroh_base::hash::{BlobFormat, Hash, HashAndFormat};
 use iroh_io::AsyncSliceReader;
 use redb::{AccessGuard, ReadableTable, StorageError};
 use serde::{Deserialize, Serialize};
 use smallvec::SmallVec;
-use tokio::io::AsyncWriteExt;
+use tokio::{io::AsyncWriteExt, sync::oneshot};
 use tracing::trace_span;
 
 mod import_flat_store;
@@ -1247,7 +1247,7 @@ pub(crate) enum OuterError {
     #[error("progress send error: {0}")]
     ProgressSend(#[from] ProgressSendError),
     #[error("recv error: {0}")]
-    Recv(#[from] oneshot::Canceled),
+    Recv(#[from] oneshot::error::RecvError),
     #[error("recv error: {0}")]
     FlumeRecv(#[from] flume::RecvError),
     #[error("join error: {0}")]
diff --git a/iroh-bytes/src/store/fs/test_support.rs b/iroh-bytes/src/store/fs/test_support.rs
index d6c715d145d..10dd3530b4e 100644
--- a/iroh-bytes/src/store/fs/test_support.rs
+++ b/iroh-bytes/src/store/fs/test_support.rs
@@ -7,7 +7,7 @@ use std::{
     path::{Path, PathBuf},
 };
 
-use futures::channel::oneshot;
+use tokio::sync::oneshot;
 
 use super::{
     tables::{ReadableTables, Tables},
diff --git a/iroh-bytes/src/store/fs/tests.rs b/iroh-bytes/src/store/fs/tests.rs
index 2a4d234a035..52edf6a14b1 100644
--- a/iroh-bytes/src/store/fs/tests.rs
+++ b/iroh-bytes/src/store/fs/tests.rs
@@ -37,7 +37,7 @@ pub fn to_stream(
         .chunks(mtu)
         .map(Bytes::copy_from_slice)
         .collect::<Vec<_>>();
-    futures::stream::iter(parts)
+    futures_lite::stream::iter(parts)
         .then(move |part| async move {
             tokio::time::sleep(delay).await;
             io::Result::Ok(part)
diff --git a/iroh-bytes/src/store/mem.rs b/iroh-bytes/src/store/mem.rs
index f313b93cd95..41c1097953e 100644
--- a/iroh-bytes/src/store/mem.rs
+++ b/iroh-bytes/src/store/mem.rs
@@ -6,7 +6,7 @@ use bao_tree::{
     BaoTree, ByteNum,
 };
 use bytes::{Bytes, BytesMut};
-use futures::{Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use iroh_base::hash::{BlobFormat, Hash, HashAndFormat};
 use iroh_io::AsyncSliceReader;
 use std::{
diff --git a/iroh-bytes/src/store/readonly_mem.rs b/iroh-bytes/src/store/readonly_mem.rs
index 229ec9eaaa9..5ee5f8798e3 100644
--- a/iroh-bytes/src/store/readonly_mem.rs
+++ b/iroh-bytes/src/store/readonly_mem.rs
@@ -24,7 +24,7 @@ use bao_tree::{
     io::{outboard::PreOrderMemOutboard, sync::Outboard},
 };
 use bytes::Bytes;
-use futures::Stream;
+use futures_lite::Stream;
 use iroh_io::AsyncSliceReader;
 use tokio::io::AsyncWriteExt;
 
diff --git a/iroh-bytes/src/store/traits.rs b/iroh-bytes/src/store/traits.rs
index 91ac1176911..3f4b1c24d68 100644
--- a/iroh-bytes/src/store/traits.rs
+++ b/iroh-bytes/src/store/traits.rs
@@ -1,12 +1,12 @@
 //! Traits for in-memory or persistent maps of blob with bao encoded outboards.
-use std::{collections::BTreeSet, io, path::PathBuf};
+use std::{collections::BTreeSet, future::Future, io, path::PathBuf};
 
 use bao_tree::{
     io::fsm::{BaoContentItem, Outboard},
     BaoTree, ByteNum, ChunkRanges,
 };
 use bytes::Bytes;
-use futures::{Future, Stream, StreamExt};
+use futures_lite::Stream;
 use genawaiter::rc::{Co, Gen};
 use iroh_base::rpc::RpcError;
 use iroh_io::AsyncSliceReader;
diff --git a/iroh-bytes/src/util/progress.rs b/iroh-bytes/src/util/progress.rs
index d80ec3533ed..5e0731185e1 100644
--- a/iroh-bytes/src/util/progress.rs
+++ b/iroh-bytes/src/util/progress.rs
@@ -1,10 +1,9 @@
 //! Utilities for reporting progress.
 //!
 //! The main entry point is the [ProgressSender] trait.
-use std::{io, marker::PhantomData, ops::Deref, sync::Arc};
+use std::{future::Future, io, marker::PhantomData, ops::Deref, sync::Arc};
 
 use bytes::Bytes;
-use futures::{future::BoxFuture, Future, FutureExt};
 use iroh_io::AsyncSliceWriter;
 
 /// A general purpose progress sender. This should be usable for reporting progress

From da3ba14e84c58787d22f0e913c7905d8cda5d7ae Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 22 Mar 2024 13:23:29 +0100
Subject: [PATCH 03/25] refactor(iroh-sync): avoid using futures crate directly

---
 Cargo.lock                     | 2 +-
 iroh-net/src/bin/iroh-relay.rs | 2 +-
 iroh-sync/Cargo.toml           | 6 +++---
 iroh-sync/src/net/codec.rs     | 6 +++---
 4 files changed, 8 insertions(+), 8 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index 3d3f6cb76a9..f10d4ec4e30 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -2664,7 +2664,7 @@ dependencies = [
  "derive_more",
  "ed25519-dalek",
  "flume",
- "futures",
+ "futures-util",
  "hex",
  "iroh-base",
  "iroh-blake3",
diff --git a/iroh-net/src/bin/iroh-relay.rs b/iroh-net/src/bin/iroh-relay.rs
index 5e45cc13358..9228348518a 100644
--- a/iroh-net/src/bin/iroh-relay.rs
+++ b/iroh-net/src/bin/iroh-relay.rs
@@ -4,11 +4,11 @@
 
 use std::{
     borrow::Cow,
+    future::Future,
     net::{IpAddr, Ipv6Addr, SocketAddr},
     path::{Path, PathBuf},
     pin::Pin,
     sync::Arc,
-    future::Future,
 };
 
 use anyhow::{anyhow, bail, Context as _, Result};
diff --git a/iroh-sync/Cargo.toml b/iroh-sync/Cargo.toml
index 0bdc4fb6150..dd13d0048d6 100644
--- a/iroh-sync/Cargo.toml
+++ b/iroh-sync/Cargo.toml
@@ -47,7 +47,7 @@ iroh-net = { version = "0.12.0", optional = true, path = "../iroh-net" }
 tokio-util = { version = "0.7", optional = true, features = ["codec", "io-util", "io"] }
 tokio-stream = { version = "0.1", optional = true, features = ["sync"]}
 quinn = { version = "0.10", optional = true }
-futures = { version = "0.3", optional = true }
+futures-util = { version = "0.3.25", optional = true }
 lru = "0.12"
 
 [dev-dependencies]
@@ -60,5 +60,5 @@ test-strategy = "0.3.1"
 
 [features]
 default = ["net", "metrics"]
-net = ["iroh-net", "tokio/io-util", "tokio-stream", "tokio-util", "quinn", "futures"]
-metrics = ["iroh-metrics"]
+net = ["dep:iroh-net", "tokio/io-util", "dep:tokio-stream", "dep:tokio-util", "dep:quinn", "dep:futures-lite"]
+metrics = ["dep:iroh-metrics"]
diff --git a/iroh-sync/src/net/codec.rs b/iroh-sync/src/net/codec.rs
index e64a2ebb4df..5f63a712b2e 100644
--- a/iroh-sync/src/net/codec.rs
+++ b/iroh-sync/src/net/codec.rs
@@ -2,7 +2,7 @@ use std::future::Future;
 
 use anyhow::{anyhow, ensure};
 use bytes::{Buf, BufMut, BytesMut};
-use futures::SinkExt;
+use futures_util::SinkExt;
 use iroh_net::key::PublicKey;
 use serde::{Deserialize, Serialize};
 use tokio::io::{AsyncRead, AsyncWrite};
@@ -382,7 +382,7 @@ mod tests {
                 &mut bob_writer,
                 &mut bob_reader,
                 bob_handle2,
-                |_namespace, _peer| futures::future::ready(AcceptOutcome::Allow),
+                |_namespace, _peer| std::future::ready(AcceptOutcome::Allow),
                 alice_peer_id,
             )
             .await
@@ -602,7 +602,7 @@ mod tests {
                 &mut bob_writer,
                 &mut bob_reader,
                 bob_handle,
-                |_namespace, _peer| futures::future::ready(AcceptOutcome::Allow),
+                |_namespace, _peer| std::future::ready(AcceptOutcome::Allow),
                 alice_node_pubkey,
             )
             .await

From ca287a9cd151e1e0f7d8a969dd13a74573d55150 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 22 Mar 2024 13:25:54 +0100
Subject: [PATCH 04/25] refactor(iroh-gossip): avoid futures crate

---
 Cargo.lock                  | 2 +-
 iroh-gossip/Cargo.toml      | 4 ++--
 iroh-gossip/src/net.rs      | 6 +++---
 iroh-gossip/src/net/util.rs | 2 +-
 4 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index f10d4ec4e30..e8f485de2cc 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -2496,7 +2496,7 @@ dependencies = [
  "data-encoding",
  "derive_more",
  "ed25519-dalek",
- "futures",
+ "futures-lite",
  "genawaiter",
  "indexmap 2.2.5",
  "iroh-base",
diff --git a/iroh-gossip/Cargo.toml b/iroh-gossip/Cargo.toml
index 7f73dedc82b..9b270d02909 100644
--- a/iroh-gossip/Cargo.toml
+++ b/iroh-gossip/Cargo.toml
@@ -32,7 +32,7 @@ iroh-metrics = { version = "0.12.0", path = "../iroh-metrics" }
 iroh-base = { version = "0.12.0", path = "../iroh-base" }
 
 # net dependencies (optional)
-futures = { version = "0.3.25", optional = true }
+futures-lite = { version = "2.3", optional = true }
 iroh-net = { path = "../iroh-net", version = "0.12.0", optional = true }
 quinn = { version = "0.10", optional = true }
 tokio = { version = "1", optional = true, features = ["io-util", "sync", "rt", "macros", "net", "fs"] }
@@ -49,7 +49,7 @@ url = "2.4.0"
 
 [features]
 default = ["net"]
-net = ["futures", "iroh-net", "quinn", "tokio", "tokio-util"]
+net = ["dep:futures-lite", "dep:iroh-net", "dep:quinn", "dep:tokio", "dep:tokio-util"]
 
 [[example]]
 name = "chat"
diff --git a/iroh-gossip/src/net.rs b/iroh-gossip/src/net.rs
index 071eb6c7a8a..7564c153b6c 100644
--- a/iroh-gossip/src/net.rs
+++ b/iroh-gossip/src/net.rs
@@ -2,7 +2,7 @@
 
 use anyhow::{anyhow, Context};
 use bytes::{Bytes, BytesMut};
-use futures::{stream::Stream, FutureExt};
+use futures_lite::stream::Stream;
 use genawaiter::sync::{Co, Gen};
 use iroh_net::{
     dialer::Dialer, key::PublicKey, magic_endpoint::get_remote_node_id, AddrInfo, MagicEndpoint,
@@ -10,7 +10,7 @@ use iroh_net::{
 };
 use rand::rngs::StdRng;
 use rand_core::SeedableRng;
-use std::{collections::HashMap, future::Future, sync::Arc, task::Poll, time::Instant};
+use std::{collections::HashMap, future::Future, pin::Pin, sync::Arc, task::Poll, time::Instant};
 use tokio::{
     sync::{broadcast, mpsc, oneshot},
     task::JoinHandle,
@@ -274,7 +274,7 @@ impl Future for JoinTopicFut {
         mut self: std::pin::Pin<&mut Self>,
         cx: &mut std::task::Context<'_>,
     ) -> std::task::Poll<Self::Output> {
-        let res = self.0.poll_unpin(cx);
+        let res = Pin::new(&mut self.0).poll(cx);
         match res {
             Poll::Pending => Poll::Pending,
             Poll::Ready(Err(_err)) => Poll::Ready(Err(anyhow!("gossip actor dropped"))),
diff --git a/iroh-gossip/src/net/util.rs b/iroh-gossip/src/net/util.rs
index a53cd4be121..11013002929 100644
--- a/iroh-gossip/src/net/util.rs
+++ b/iroh-gossip/src/net/util.rs
@@ -120,7 +120,7 @@ impl<T> Timers<T> {
                 sleep.await;
                 self.map.drain_until(instant)
             }
-            None => futures::future::pending().await,
+            None => std::future::pending().await,
         }
     }
 }

From c7b9ffbd3bd83cfe7ec9fefadc923347b631b7f0 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 22 Mar 2024 13:37:44 +0100
Subject: [PATCH 05/25] fixup: iroh-bytes downloader

---
 iroh-bytes/Cargo.toml                    |  8 +++++---
 iroh-bytes/src/downloader.rs             | 10 ++++------
 iroh-bytes/src/downloader/get.rs         |  2 +-
 iroh-bytes/src/downloader/test.rs        |  6 +++---
 iroh-bytes/src/downloader/test/dialer.rs |  2 +-
 iroh-bytes/src/downloader/test/getter.rs |  3 ++-
 6 files changed, 16 insertions(+), 15 deletions(-)

diff --git a/iroh-bytes/Cargo.toml b/iroh-bytes/Cargo.toml
index b578c95bed0..59bb19fefb9 100644
--- a/iroh-bytes/Cargo.toml
+++ b/iroh-bytes/Cargo.toml
@@ -53,6 +53,7 @@ tracing-futures = "0.2.5"
 [dev-dependencies]
 http-body = "0.4.5"
 iroh-test = { path = "../iroh-test" }
+futures-buffered = "0.2.4"
 proptest = "1.0.0"
 serde_json = "1.0.107"
 serde_test = "1.0.176"
@@ -64,9 +65,10 @@ tempfile = "3.10.0"
 
 [features]
 default = ["fs-store"]
-fs-store = ["reflink-copy", "redb"]
-downloader = ["iroh-net", "parking_lot", "tokio-util/time"]
-metrics = ["iroh-metrics"]
+fs-store = ["dep:reflink-copy", "redb"]
+redb = ["dep:redb"]
+downloader = ["dep:iroh-net", "dep:parking_lot", "tokio-util/time"]
+metrics = ["dep:iroh-metrics"]
 
 [[example]]
 name = "provide-bytes"
diff --git a/iroh-bytes/src/downloader.rs b/iroh-bytes/src/downloader.rs
index 1963bb306be..9244f58c21b 100644
--- a/iroh-bytes/src/downloader.rs
+++ b/iroh-bytes/src/downloader.rs
@@ -37,7 +37,7 @@ use std::{
 
 use crate::{get::Stats, protocol::RangeSpecSeq, store::Store, Hash, HashAndFormat};
 use bao_tree::ChunkRanges;
-use futures::{future::LocalBoxFuture, FutureExt, StreamExt};
+use futures_lite::{future::BoxedLocal, StreamExt, Stream};
 use iroh_net::{MagicEndpoint, NodeId};
 use tokio::{
     sync::{mpsc, oneshot},
@@ -64,9 +64,7 @@ const SERVICE_CHANNEL_CAPACITY: usize = 128;
 pub type Id = u64;
 
 /// Trait modeling a dialer. This allows for IO-less testing.
-pub trait Dialer:
-    futures::Stream<Item = (NodeId, anyhow::Result<Self::Connection>)> + Unpin
-{
+pub trait Dialer: Stream<Item = (NodeId, anyhow::Result<Self::Connection>)> + Unpin {
     /// Type of connections returned by the Dialer.
     type Connection: Clone;
     /// Dial a node.
@@ -89,7 +87,7 @@ pub enum FailureAction {
 }
 
 /// Future of a get request.
-type GetFut = LocalBoxFuture<'static, Result<Stats, FailureAction>>;
+type GetFut = BoxedLocal<Result<Stats, FailureAction>>;
 
 /// Trait modelling performing a single request over a connection. This allows for IO-less testing.
 pub trait Getter {
@@ -205,7 +203,7 @@ impl std::future::Future for DownloadHandle {
         use std::task::Poll::*;
         // make it easier on holders of the handle to poll the result, removing the receiver error
         // from the middle
-        match self.receiver.poll_unpin(cx) {
+        match std::pin::Pin::new(&mut self.receiver).poll(cx) {
             Ready(Ok(result)) => Ready(result),
             Ready(Err(recv_err)) => Ready(Err(anyhow::anyhow!("oneshot error: {recv_err}"))),
             Pending => Pending,
diff --git a/iroh-bytes/src/downloader/get.rs b/iroh-bytes/src/downloader/get.rs
index 334064bdeea..2e87c3e9264 100644
--- a/iroh-bytes/src/downloader/get.rs
+++ b/iroh-bytes/src/downloader/get.rs
@@ -5,7 +5,7 @@ use crate::{
     store::Store,
     util::progress::IgnoreProgressSender,
 };
-use futures::FutureExt;
+use futures_lite::FutureExt;
 #[cfg(feature = "metrics")]
 use iroh_metrics::{inc, inc_by};
 
diff --git a/iroh-bytes/src/downloader/test.rs b/iroh-bytes/src/downloader/test.rs
index 6f7029ad2fa..9a8e0019bab 100644
--- a/iroh-bytes/src/downloader/test.rs
+++ b/iroh-bytes/src/downloader/test.rs
@@ -81,7 +81,7 @@ async fn deduplication() {
         handles.push(h);
     }
     assert!(
-        futures::future::join_all(handles)
+        futures_buffered::join_all(handles)
             .await
             .into_iter()
             .all(|r| r.is_ok()),
@@ -168,7 +168,7 @@ async fn max_concurrent_requests() {
     }
 
     assert!(
-        futures::future::join_all(handles)
+        futures_buffered::join_all(handles)
             .await
             .into_iter()
             .all(|r| r.is_ok()),
@@ -211,7 +211,7 @@ async fn max_concurrent_requests_per_peer() {
         handles.push(h);
     }
 
-    futures::future::join_all(handles).await;
+    futures_buffered::join_all(handles).await;
 }
 
 /// Tests that providers are preferred over candidates.
diff --git a/iroh-bytes/src/downloader/test/dialer.rs b/iroh-bytes/src/downloader/test/dialer.rs
index 5463b01b7c2..a509a36948a 100644
--- a/iroh-bytes/src/downloader/test/dialer.rs
+++ b/iroh-bytes/src/downloader/test/dialer.rs
@@ -62,7 +62,7 @@ impl Dialer for TestingDialer {
     }
 }
 
-impl futures::Stream for TestingDialer {
+impl Stream for TestingDialer {
     type Item = (NodeId, anyhow::Result<NodeId>);
 
     fn poll_next(self: std::pin::Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
diff --git a/iroh-bytes/src/downloader/test/getter.rs b/iroh-bytes/src/downloader/test/getter.rs
index 78f2307469d..289d4b4aadb 100644
--- a/iroh-bytes/src/downloader/test/getter.rs
+++ b/iroh-bytes/src/downloader/test/getter.rs
@@ -2,6 +2,7 @@
 
 use std::{sync::Arc, time::Duration};
 
+use futures_lite::FutureExt;
 use parking_lot::RwLock;
 
 use super::*;
@@ -30,7 +31,7 @@ impl Getter for TestingGetter {
             tokio::time::sleep(request_duration).await;
             Ok(Stats::default())
         }
-        .boxed_local()
+            .boxed_local()
     }
 }
 

From 010bce4597073f8294ba68bd92c28e568e82b45e Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 22 Mar 2024 15:05:20 +0100
Subject: [PATCH 06/25] refactor(iroh): avoid futures crate

---
 Cargo.lock                               |  4 +-
 iroh-bytes/src/downloader.rs             |  2 +-
 iroh-bytes/src/downloader/test/getter.rs |  2 +-
 iroh/Cargo.toml                          |  4 +-
 iroh/src/client.rs                       |  2 +-
 iroh/src/client/authors.rs               |  4 +-
 iroh/src/client/blobs.rs                 | 20 ++++---
 iroh/src/client/docs.rs                  | 18 +++---
 iroh/src/client/node.rs                  |  4 +-
 iroh/src/client/tags.rs                  |  4 +-
 iroh/src/node.rs                         | 17 +++---
 iroh/src/node/builder.rs                 | 12 +++-
 iroh/src/node/rpc.rs                     | 20 ++++---
 iroh/src/sync_engine.rs                  | 72 ++++++++++--------------
 iroh/src/sync_engine/gossip.rs           | 16 ++++--
 iroh/src/sync_engine/live.rs             |  4 +-
 iroh/src/sync_engine/rpc.rs              | 16 +++---
 iroh/tests/gc.rs                         |  2 +-
 iroh/tests/provide.rs                    |  4 +-
 iroh/tests/sync.rs                       |  4 +-
 20 files changed, 122 insertions(+), 109 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index e8f485de2cc..a7ca3c518a1 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -2304,7 +2304,9 @@ dependencies = [
  "data-encoding",
  "derive_more",
  "flume",
- "futures",
+ "futures-buffered",
+ "futures-lite",
+ "futures-util",
  "genawaiter",
  "hashlink",
  "hex",
diff --git a/iroh-bytes/src/downloader.rs b/iroh-bytes/src/downloader.rs
index 9244f58c21b..50fa52d5d90 100644
--- a/iroh-bytes/src/downloader.rs
+++ b/iroh-bytes/src/downloader.rs
@@ -37,7 +37,7 @@ use std::{
 
 use crate::{get::Stats, protocol::RangeSpecSeq, store::Store, Hash, HashAndFormat};
 use bao_tree::ChunkRanges;
-use futures_lite::{future::BoxedLocal, StreamExt, Stream};
+use futures_lite::{future::BoxedLocal, Stream, StreamExt};
 use iroh_net::{MagicEndpoint, NodeId};
 use tokio::{
     sync::{mpsc, oneshot},
diff --git a/iroh-bytes/src/downloader/test/getter.rs b/iroh-bytes/src/downloader/test/getter.rs
index 289d4b4aadb..f777c129466 100644
--- a/iroh-bytes/src/downloader/test/getter.rs
+++ b/iroh-bytes/src/downloader/test/getter.rs
@@ -31,7 +31,7 @@ impl Getter for TestingGetter {
             tokio::time::sleep(request_duration).await;
             Ok(Stats::default())
         }
-            .boxed_local()
+        .boxed_local()
     }
 }
 
diff --git a/iroh/Cargo.toml b/iroh/Cargo.toml
index 9e9c228c842..6b248c5aad6 100644
--- a/iroh/Cargo.toml
+++ b/iroh/Cargo.toml
@@ -22,7 +22,9 @@ bytes = "1"
 data-encoding = "2.4.0"
 derive_more = { version = "1.0.0-beta.1", features = ["debug", "display", "from", "try_into", "from_str"] }
 flume = "0.11"
-futures = "0.3.25"
+futures-buffered = "0.2.4"
+futures-lite = "2.3"
+futures-util = "0.3"
 genawaiter = { version = "0.99", default-features = false, features = ["futures03"] }
 hashlink = "0.8.4"
 hex = { version = "0.4.3" }
diff --git a/iroh/src/client.rs b/iroh/src/client.rs
index 56f85827b0d..8159739793a 100644
--- a/iroh/src/client.rs
+++ b/iroh/src/client.rs
@@ -2,7 +2,7 @@
 //!
 //! TODO: Contains only iroh sync related methods. Add other methods.
 
-use futures::{Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use quic_rpc::{RpcClient, ServiceConnection};
 
 use crate::rpc_protocol::ProviderService;
diff --git a/iroh/src/client/authors.rs b/iroh/src/client/authors.rs
index 19a3ba75b66..67950db179e 100644
--- a/iroh/src/client/authors.rs
+++ b/iroh/src/client/authors.rs
@@ -1,5 +1,5 @@
 use anyhow::Result;
-use futures::{Stream, TryStreamExt};
+use futures_lite::{stream::StreamExt, Stream};
 use iroh_sync::AuthorId;
 use quic_rpc::{RpcClient, ServiceConnection};
 
@@ -26,6 +26,6 @@ where
     /// List document authors for which we have a secret key.
     pub async fn list(&self) -> Result<impl Stream<Item = Result<AuthorId>>> {
         let stream = self.rpc.server_streaming(AuthorListRequest {}).await?;
-        Ok(flatten(stream).map_ok(|res| res.author_id))
+        Ok(flatten(stream).map(|res| res.map(|res| res.author_id)))
     }
 }
diff --git a/iroh/src/client/blobs.rs b/iroh/src/client/blobs.rs
index 90c7be8956e..afca401b13f 100644
--- a/iroh/src/client/blobs.rs
+++ b/iroh/src/client/blobs.rs
@@ -1,4 +1,5 @@
 use std::{
+    future::Future,
     io,
     path::PathBuf,
     pin::Pin,
@@ -8,7 +9,8 @@ use std::{
 
 use anyhow::{anyhow, Result};
 use bytes::Bytes;
-use futures::{Future, SinkExt, Stream, StreamExt, TryStreamExt};
+use futures_lite::{Stream, StreamExt};
+use futures_util::SinkExt;
 use iroh_base::ticket::BlobTicket;
 use iroh_bytes::{
     export::ExportProgress,
@@ -181,7 +183,7 @@ where
 
     /// Write a blob by passing bytes.
     pub async fn add_bytes(&self, bytes: impl Into<Bytes>) -> anyhow::Result<BlobAddOutcome> {
-        let input = futures::stream::once(futures::future::ready(Ok(bytes.into())));
+        let input = futures_lite::stream::once(Ok(bytes.into()));
         self.add_stream(input, SetTagOption::Auto).await?.await
     }
 
@@ -191,7 +193,7 @@ where
         bytes: impl Into<Bytes>,
         name: impl Into<Tag>,
     ) -> anyhow::Result<BlobAddOutcome> {
-        let input = futures::stream::once(futures::future::ready(Ok(bytes.into())));
+        let input = futures_lite::stream::once(Ok(bytes.into()));
         self.add_stream(input, SetTagOption::Named(name.into()))
             .await?
             .await
@@ -208,7 +210,7 @@ where
             .rpc
             .server_streaming(BlobValidateRequest { repair })
             .await?;
-        Ok(stream.map_err(anyhow::Error::from))
+        Ok(stream.map(|res| res.map_err(anyhow::Error::from)))
     }
 
     /// Validate hashes on the running node.
@@ -229,7 +231,7 @@ where
     pub async fn download(&self, req: BlobDownloadRequest) -> Result<BlobDownloadProgress> {
         let stream = self.rpc.server_streaming(req).await?;
         Ok(BlobDownloadProgress::new(
-            stream.map_err(anyhow::Error::from),
+            stream.map(|res| res.map_err(anyhow::Error::from)),
         ))
     }
 
@@ -429,7 +431,7 @@ impl BlobAddProgress {
 impl Stream for BlobAddProgress {
     type Item = Result<AddProgress>;
     fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
-        self.stream.poll_next_unpin(cx)
+        Pin::new(&mut self.stream).poll_next(cx)
     }
 }
 
@@ -438,7 +440,7 @@ impl Future for BlobAddProgress {
 
     fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
         loop {
-            match self.stream.poll_next_unpin(cx) {
+            match Pin::new(&mut self.stream).poll_next(cx) {
                 Poll::Pending => return Poll::Pending,
                 Poll::Ready(None) => {
                     return Poll::Ready(Err(anyhow!("Response stream ended prematurely")))
@@ -534,7 +536,7 @@ impl BlobDownloadProgress {
 impl Stream for BlobDownloadProgress {
     type Item = Result<DownloadProgress>;
     fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
-        self.stream.poll_next_unpin(cx)
+        Pin::new(&mut self.stream).poll_next(cx)
     }
 }
 
@@ -543,7 +545,7 @@ impl Future for BlobDownloadProgress {
 
     fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
         loop {
-            match self.stream.poll_next_unpin(cx) {
+            match Pin::new(&mut self.stream).poll_next(cx) {
                 Poll::Pending => return Poll::Pending,
                 Poll::Ready(None) => {
                     return Poll::Ready(Err(anyhow!("Response stream ended prematurely")))
diff --git a/iroh/src/client/docs.rs b/iroh/src/client/docs.rs
index ef37ed1320c..29ffbdfd8c9 100644
--- a/iroh/src/client/docs.rs
+++ b/iroh/src/client/docs.rs
@@ -7,7 +7,7 @@ use std::{
 
 use anyhow::{anyhow, Context as _, Result};
 use bytes::Bytes;
-use futures::{Stream, StreamExt, TryStreamExt};
+use futures_lite::{Stream, StreamExt};
 use iroh_base::key::PublicKey;
 use iroh_bytes::{export::ExportProgress, store::ExportMode, Hash};
 use iroh_net::NodeAddr;
@@ -72,7 +72,7 @@ where
     /// List all documents.
     pub async fn list(&self) -> Result<impl Stream<Item = Result<(NamespaceId, CapabilityKind)>>> {
         let stream = self.rpc.server_streaming(DocListRequest {}).await?;
-        Ok(flatten(stream).map_ok(|res| (res.id, res.capability)))
+        Ok(flatten(stream).map(|res| res.map(|res| (res.id, res.capability))))
     }
 
     /// Get a [`Doc`] client for a single document. Return None if the document cannot be found.
@@ -293,7 +293,7 @@ where
                 query: query.into(),
             })
             .await?;
-        Ok(flatten(stream).map_ok(|res| res.entry.into()))
+        Ok(flatten(stream).map(|res| res.map(|res| res.entry.into())))
     }
 
     /// Get a single entry.
@@ -340,9 +340,10 @@ where
             .rpc
             .server_streaming(DocSubscribeRequest { doc_id: self.id() })
             .await?;
-        Ok(flatten(stream)
-            .map_ok(|res| res.event.into())
-            .map_err(Into::into))
+        Ok(flatten(stream).map(|res| match res {
+            Ok(res) => Ok(res.event.into()),
+            Err(err) => Err(err.into()),
+        }))
     }
 
     /// Get status info for this document
@@ -588,7 +589,7 @@ pub struct DocImportFileOutcome {
 impl Stream for DocImportFileProgress {
     type Item = Result<DocImportProgress>;
     fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
-        self.stream.poll_next_unpin(cx)
+        Pin::new(&mut self.stream).poll_next(cx)
     }
 }
 
@@ -653,8 +654,9 @@ pub struct DocExportFileOutcome {
 
 impl Stream for DocExportFileProgress {
     type Item = Result<ExportProgress>;
+
     fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
-        self.stream.poll_next_unpin(cx)
+        Pin::new(&mut self.stream).poll_next(cx)
     }
 }
 
diff --git a/iroh/src/client/node.rs b/iroh/src/client/node.rs
index 30291400f46..6a4d15ff92a 100644
--- a/iroh/src/client/node.rs
+++ b/iroh/src/client/node.rs
@@ -1,7 +1,7 @@
 use std::collections::BTreeMap;
 
 use anyhow::Result;
-use futures::{Stream, TryStreamExt};
+use futures_lite::{Stream, StreamExt};
 use iroh_base::key::PublicKey;
 use iroh_net::magic_endpoint::ConnectionInfo;
 use quic_rpc::{RpcClient, ServiceConnection};
@@ -32,7 +32,7 @@ where
     /// Get information about the different connections we have made
     pub async fn connections(&self) -> Result<impl Stream<Item = Result<ConnectionInfo>>> {
         let stream = self.rpc.server_streaming(NodeConnectionsRequest {}).await?;
-        Ok(flatten(stream).map_ok(|res| res.conn_info))
+        Ok(flatten(stream).map(|res| res.map(|res| res.conn_info)))
     }
 
     /// Get connection information about a node
diff --git a/iroh/src/client/tags.rs b/iroh/src/client/tags.rs
index db5db7d5b7a..0d5a402a3c1 100644
--- a/iroh/src/client/tags.rs
+++ b/iroh/src/client/tags.rs
@@ -1,5 +1,5 @@
 use anyhow::Result;
-use futures::{Stream, TryStreamExt};
+use futures_lite::{Stream, StreamExt};
 use iroh_bytes::Tag;
 use quic_rpc::{RpcClient, ServiceConnection};
 
@@ -18,7 +18,7 @@ where
     /// List all tags.
     pub async fn list(&self) -> Result<impl Stream<Item = Result<ListTagsResponse>>> {
         let stream = self.rpc.server_streaming(ListTagsRequest).await?;
-        Ok(stream.map_err(anyhow::Error::from))
+        Ok(stream.map(|res| res.map_err(anyhow::Error::from)))
     }
 
     /// Delete a tag.
diff --git a/iroh/src/node.rs b/iroh/src/node.rs
index ceaa9b12c16..a7a28f50223 100644
--- a/iroh/src/node.rs
+++ b/iroh/src/node.rs
@@ -14,8 +14,7 @@ use std::sync::Arc;
 use std::task::Poll;
 
 use anyhow::{anyhow, Result};
-use futures::future::{BoxFuture, Shared};
-use futures::{FutureExt, StreamExt};
+use futures_lite::{future::Boxed as BoxFuture, FutureExt, StreamExt};
 use iroh_bytes::store::Store as BaoStore;
 use iroh_bytes::BlobFormat;
 use iroh_bytes::Hash;
@@ -45,7 +44,7 @@ mod rpc_status;
 pub use builder::{Builder, GcPolicy, StorageConfig};
 pub use rpc_status::RpcStatus;
 
-type EventCallback = Box<dyn Fn(Event) -> BoxFuture<'static, ()> + 'static + Sync + Send>;
+type EventCallback = Box<dyn Fn(Event) -> BoxFuture<()> + 'static + Sync + Send>;
 
 #[derive(Default, derive_more::Debug, Clone)]
 struct Callbacks(#[debug("..")] Arc<RwLock<Vec<EventCallback>>>);
@@ -66,8 +65,9 @@ impl Callbacks {
 
 impl iroh_bytes::provider::EventSender for Callbacks {
     fn send(&self, event: iroh_bytes::provider::Event) -> BoxFuture<()> {
+        let this = self.clone();
         async move {
-            let cbs = self.0.read().await;
+            let cbs = this.0.read().await;
             for cb in &*cbs {
                 cb(Event::ByteProvide(event.clone())).await;
             }
@@ -89,7 +89,7 @@ impl iroh_bytes::provider::EventSender for Callbacks {
 #[derive(Debug, Clone)]
 pub struct Node<D> {
     inner: Arc<NodeInner<D>>,
-    task: Shared<BoxFuture<'static, Result<(), Arc<JoinError>>>>,
+    task: (), // Arc<BoxFuture<anyhow::Result<()>>>,
     client: crate::client::mem::Iroh,
 }
 
@@ -101,7 +101,7 @@ struct NodeInner<D> {
     cancel_token: CancellationToken,
     controller: FlumeConnection<ProviderResponse, ProviderRequest>,
     #[debug("callbacks: Sender<Box<dyn Fn(Event)>>")]
-    cb_sender: mpsc::Sender<Box<dyn Fn(Event) -> BoxFuture<'static, ()> + Send + Sync + 'static>>,
+    cb_sender: mpsc::Sender<Box<dyn Fn(Event) -> BoxFuture<()> + Send + Sync + 'static>>,
     callbacks: Callbacks,
     #[allow(dead_code)]
     gc_task: Option<AbortingJoinHandle<()>>,
@@ -189,7 +189,7 @@ impl<D: BaoStore> Node<D> {
     /// progress.
     ///
     /// Warning: The callback must complete quickly, as otherwise it will block ongoing work.
-    pub async fn subscribe<F: Fn(Event) -> BoxFuture<'static, ()> + Send + Sync + 'static>(
+    pub async fn subscribe<F: Fn(Event) -> BoxFuture<()> + Send + Sync + 'static>(
         &self,
         cb: F,
     ) -> Result<()> {
@@ -253,7 +253,8 @@ impl<D> Future for Node<D> {
     type Output = Result<(), Arc<JoinError>>;
 
     fn poll(mut self: Pin<&mut Self>, cx: &mut std::task::Context<'_>) -> Poll<Self::Output> {
-        Pin::new(&mut self.task).poll(cx)
+        // Pin::new(&mut self.task).poll(cx)
+        todo!()
     }
 }
 
diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs
index bc5aa4dfe76..77f49df7aa8 100644
--- a/iroh/src/node/builder.rs
+++ b/iroh/src/node/builder.rs
@@ -7,7 +7,7 @@ use std::{
 };
 
 use anyhow::{bail, Context, Result};
-use futures::{FutureExt, StreamExt, TryFutureExt};
+use futures_lite::{FutureExt, StreamExt};
 use iroh_base::key::SecretKey;
 use iroh_bytes::{
     downloader::Downloader,
@@ -382,9 +382,17 @@ where
             )
         };
 
+        let task = Arc::new(
+            async move {
+                task.await?;
+                anyhow::Ok(())
+            }
+            .boxed(),
+        );
+
         let node = Node {
             inner,
-            task: task.map_err(Arc::new).boxed().shared(),
+            task: (),
             client,
         };
 
diff --git a/iroh/src/node/rpc.rs b/iroh/src/node/rpc.rs
index d20fd12ddd8..0aef68ca24d 100644
--- a/iroh/src/node/rpc.rs
+++ b/iroh/src/node/rpc.rs
@@ -1,10 +1,12 @@
 use std::fmt::Debug;
+use std::future::Future;
 use std::io;
 use std::sync::{Arc, Mutex};
 use std::time::Duration;
 
 use anyhow::{anyhow, Result};
-use futures::{Future, FutureExt, Stream, StreamExt};
+use futures_buffered::BufferedStreamExt;
+use futures_lite::{FutureExt, Stream, StreamExt};
 use genawaiter::sync::{Co, Gen};
 use iroh_base::rpc::RpcResult;
 use iroh_bytes::export::ExportProgress;
@@ -234,10 +236,11 @@ impl<D: BaoStore> Handler<D> {
                     .await
                 }
                 DocSubscribe(msg) => {
-                    chan.server_streaming(msg, handler, |handler, req| {
-                        async move { handler.inner.sync.doc_subscribe(req) }.flatten_stream()
+                    todo!()
+                    /*chan.server_streaming(msg, handler, |handler, req| {
+                        handler.inner.sync.doc_subscribe(req)
                     })
-                    .await
+                    .await*/
                 }
                 DocSetDownloadPolicy(msg) => {
                     chan.rpc(msg, handler, |handler, req| async move {
@@ -651,7 +654,6 @@ impl<D: BaoStore> Handler<D> {
         progress: flume::Sender<AddProgress>,
     ) -> anyhow::Result<()> {
         use crate::rpc_protocol::WrapOption;
-        use futures::TryStreamExt;
         use iroh_bytes::store::ImportMode;
         use std::collections::BTreeMap;
 
@@ -701,7 +703,7 @@ impl<D: BaoStore> Handler<D> {
             // import all files below root recursively
             let data_sources = crate::util::fs::scan_path(root, wrap)?;
             const IO_PARALLELISM: usize = 4;
-            let result: Vec<_> = futures::stream::iter(data_sources)
+            let result: Vec<_> = futures_lite::stream::iter(data_sources)
                 .map(|source| {
                     let import_progress = import_progress.clone();
                     let db = self.inner.db.clone();
@@ -719,8 +721,8 @@ impl<D: BaoStore> Handler<D> {
                         io::Result::Ok((name, hash, size, tag))
                     }
                 })
-                .buffered(IO_PARALLELISM)
-                .try_collect::<Vec<_>>()
+                .buffered_ordered(IO_PARALLELISM)
+                .try_collect()
                 .await?;
 
             // create a collection
@@ -807,7 +809,7 @@ impl<D: BaoStore> Handler<D> {
     }
 
     fn node_watch(self, _: NodeWatchRequest) -> impl Stream<Item = NodeWatchResponse> {
-        futures::stream::unfold((), |()| async move {
+        futures_lite::stream::unfold((), |()| async move {
             tokio::time::sleep(HEALTH_POLL_WAIT).await;
             Some((
                 NodeWatchResponse {
diff --git a/iroh/src/sync_engine.rs b/iroh/src/sync_engine.rs
index 6a8cae118a0..67e56efee52 100644
--- a/iroh/src/sync_engine.rs
+++ b/iroh/src/sync_engine.rs
@@ -5,10 +5,7 @@
 use std::{io, sync::Arc};
 
 use anyhow::Result;
-use futures::{
-    future::{BoxFuture, FutureExt, Shared},
-    Stream, TryStreamExt,
-};
+use futures_lite::{future::Boxed as BoxFuture, FutureExt, Stream, StreamExt};
 use iroh_bytes::downloader::Downloader;
 use iroh_bytes::{store::EntryStatus, Hash};
 use iroh_gossip::net::Gossip;
@@ -16,7 +13,6 @@ use iroh_net::{key::PublicKey, MagicEndpoint, NodeAddr};
 use iroh_sync::{actor::SyncHandle, ContentStatus, ContentStatusCallback, Entry, NamespaceId};
 use serde::{Deserialize, Serialize};
 use tokio::sync::{mpsc, oneshot};
-use tokio_stream::StreamExt;
 use tracing::{error, error_span, Instrument};
 
 mod gossip;
@@ -46,7 +42,8 @@ pub struct SyncEngine {
     pub(crate) endpoint: MagicEndpoint,
     pub(crate) sync: SyncHandle,
     to_live_actor: mpsc::Sender<ToLiveActor>,
-    tasks_fut: Shared<BoxFuture<'static, ()>>,
+    #[debug("Arc<BoxFuture<()>>")]
+    tasks_fut: (), // Arc<BoxFuture<()>>,
     #[debug("ContentStatusCallback")]
     content_status_cb: ContentStatusCallback,
 }
@@ -121,14 +118,13 @@ impl SyncEngine {
                 }
             }
         }
-        .boxed()
-        .shared();
+        .boxed();
 
         Self {
             endpoint,
             sync,
             to_live_actor: live_actor_tx,
-            tasks_fut,
+            tasks_fut: (),
             content_status_cb,
         }
     }
@@ -182,49 +178,40 @@ impl SyncEngine {
     }
 
     /// Subscribe to replica and sync progress events.
-    pub fn subscribe(
+    pub async fn subscribe(
         &self,
         namespace: NamespaceId,
-    ) -> impl Stream<Item = Result<LiveEvent>> + Unpin + 'static {
+    ) -> Result<impl Stream<Item = Result<LiveEvent>> + Unpin + 'static> {
         let content_status_cb = self.content_status_cb.clone();
 
         // Create a future that sends channel senders to the respective actors.
         // We clone `self` so that the future does not capture any lifetimes.
         let this = self.clone();
-        let fut = async move {
-            // Subscribe to insert events from the replica.
-            let replica_events = {
-                let (s, r) = flume::bounded(SUBSCRIBE_CHANNEL_CAP);
-                this.sync.subscribe(namespace, s).await?;
-                r.into_stream()
-                    .map(move |ev| LiveEvent::from_replica_event(ev, &content_status_cb))
-            };
 
-            // Subscribe to events from the [`live::Actor`].
-            let sync_events = {
-                let (s, r) = flume::bounded(SUBSCRIBE_CHANNEL_CAP);
-                let (reply, reply_rx) = oneshot::channel();
-                this.to_live_actor
-                    .send(ToLiveActor::Subscribe {
-                        namespace,
-                        sender: s,
-                        reply,
-                    })
-                    .await?;
-                reply_rx.await??;
-                r.into_stream().map(|event| Ok(LiveEvent::from(event)))
-            };
+        // Subscribe to insert events from the replica.
+        let a = {
+            let (s, r) = flume::bounded(SUBSCRIBE_CHANNEL_CAP);
+            this.sync.subscribe(namespace, s).await?;
+            r.into_stream()
+                .map(move |ev| LiveEvent::from_replica_event(ev, &content_status_cb))
+        };
 
-            // Merge the two receivers into a single stream.
-            let stream = replica_events.merge(sync_events);
-            // We need type annotations for the error type here.
-            Result::<_, anyhow::Error>::Ok(stream)
+        // Subscribe to events from the [`live::Actor`].
+        let b = {
+            let (s, r) = flume::bounded(SUBSCRIBE_CHANNEL_CAP);
+            let (reply, reply_rx) = oneshot::channel();
+            this.to_live_actor
+                .send(ToLiveActor::Subscribe {
+                    namespace,
+                    sender: s,
+                    reply,
+                })
+                .await?;
+            reply_rx.await??;
+            r.into_stream().map(|event| Ok(LiveEvent::from(event)))
         };
 
-        // Flatten the future into a single stream. If the future errors, the error will be
-        // returned from the first call to [`Stream::next`].
-        // We first pin the future so that the resulting stream is `Unpin`.
-        Box::pin(fut).into_stream().try_flatten()
+        Ok(a.or(b))
     }
 
     /// Handle an incoming iroh-sync connection.
@@ -238,7 +225,8 @@ impl SyncEngine {
     /// Shutdown the sync engine.
     pub async fn shutdown(&self) -> Result<()> {
         self.to_live_actor.send(ToLiveActor::Shutdown).await?;
-        self.tasks_fut.clone().await;
+        // TODO
+        // self.tasks_fut.clone().await;
         Ok(())
     }
 }
diff --git a/iroh/src/sync_engine/gossip.rs b/iroh/src/sync_engine/gossip.rs
index 8daa8781f81..c8f6edd32dc 100644
--- a/iroh/src/sync_engine/gossip.rs
+++ b/iroh/src/sync_engine/gossip.rs
@@ -1,7 +1,7 @@
 use std::collections::HashSet;
 
 use anyhow::{anyhow, Context, Result};
-use futures::{stream::StreamExt, FutureExt};
+use futures_lite::{FutureExt, StreamExt};
 use iroh_gossip::{
     net::{Event, Gossip},
     proto::TopicId,
@@ -112,12 +112,16 @@ impl GossipActor {
                 return Ok(false);
             }
             ToGossipActor::Join { namespace, peers } => {
+                let gossip = self.gossip.clone();
                 // join gossip for the topic to receive and send message
-                let fut = self
-                    .gossip
-                    .join(namespace.into(), peers)
-                    .await?
-                    .map(move |res| (namespace, res));
+                let fut = async move {
+                    let res = gossip.join(namespace.into(), peers).await;
+                    let res = match res {
+                        Ok(fut) => fut.await,
+                        Err(err) => Err(err.into()),
+                    };
+                    (namespace, res)
+                };
                 self.want_join.insert(namespace);
                 self.pending_joins.spawn(fut);
             }
diff --git a/iroh/src/sync_engine/live.rs b/iroh/src/sync_engine/live.rs
index 57b9444d87d..ec911dd54fd 100644
--- a/iroh/src/sync_engine/live.rs
+++ b/iroh/src/sync_engine/live.rs
@@ -3,7 +3,7 @@
 use std::{collections::HashMap, time::SystemTime};
 
 use anyhow::{Context, Result};
-use futures::FutureExt;
+use futures_lite::FutureExt;
 use iroh_bytes::downloader::{DownloadKind, Downloader, Role};
 use iroh_bytes::{store::EntryStatus, Hash};
 use iroh_gossip::{net::Gossip, proto::TopicId};
@@ -753,7 +753,7 @@ impl Subscribers {
 
     async fn send(&mut self, event: Event) -> bool {
         let futs = self.0.iter().map(|sender| sender.send_async(event.clone()));
-        let res = futures::future::join_all(futs).await;
+        let res = futures_buffered::join_all(futs).await;
         // reverse the order so removing does not shift remaining indices
         for (i, res) in res.into_iter().enumerate().rev() {
             if res.is_err() {
diff --git a/iroh/src/sync_engine/rpc.rs b/iroh/src/sync_engine/rpc.rs
index 4b3afffa3ce..aabc28d2a2e 100644
--- a/iroh/src/sync_engine/rpc.rs
+++ b/iroh/src/sync_engine/rpc.rs
@@ -1,12 +1,13 @@
 //! This module contains an impl block on [`SyncEngine`] with handlers for RPC requests
 
 use anyhow::anyhow;
-use futures::Stream;
+use futures_lite::Stream;
 use iroh_bytes::{store::Store as BaoStore, BlobFormat};
 use iroh_sync::{Author, NamespaceSecret};
 use tokio_stream::StreamExt;
 
 use crate::rpc_protocol::{DocGetSyncPeersRequest, DocGetSyncPeersResponse};
+use crate::sync_engine::LiveEvent;
 use crate::{
     rpc_protocol::{
         AuthorCreateRequest, AuthorCreateResponse, AuthorListRequest, AuthorListResponse,
@@ -123,15 +124,16 @@ impl SyncEngine {
         }))
     }
 
-    pub fn doc_subscribe(
+    pub async fn doc_subscribe(
         &self,
         req: DocSubscribeRequest,
-    ) -> impl Stream<Item = RpcResult<DocSubscribeResponse>> {
-        let stream = self.subscribe(req.doc_id);
-        stream.map(|res| {
-            res.map(|event| DocSubscribeResponse { event })
+    ) -> RpcResult<impl Stream<Item = RpcResult<DocSubscribeResponse>>> {
+        let stream = self.subscribe(req.doc_id).await?;
+
+        Ok(stream.map(|el| {
+            el.map(|event| DocSubscribeResponse { event })
                 .map_err(Into::into)
-        })
+        }))
     }
 
     pub async fn doc_import(&self, req: DocImportRequest) -> RpcResult<DocImportResponse> {
diff --git a/iroh/tests/gc.rs b/iroh/tests/gc.rs
index aa4f2ffb97d..12db9226ae3 100644
--- a/iroh/tests/gc.rs
+++ b/iroh/tests/gc.rs
@@ -3,7 +3,7 @@ use std::{io::Cursor, time::Duration};
 use anyhow::Result;
 use bao_tree::{blake3, io::sync::Outboard, ChunkRanges};
 use bytes::Bytes;
-use futures::FutureExt;
+use futures_lite::FutureExt;
 use iroh::node::{self, Node};
 use rand::RngCore;
 
diff --git a/iroh/tests/provide.rs b/iroh/tests/provide.rs
index dffacb86cdc..3911ce051c2 100644
--- a/iroh/tests/provide.rs
+++ b/iroh/tests/provide.rs
@@ -7,7 +7,7 @@ use std::{
 
 use anyhow::{anyhow, Context, Result};
 use bytes::Bytes;
-use futures::FutureExt;
+use futures_lite::FutureExt;
 use iroh::{
     dial::Options,
     node::{Builder, Event},
@@ -169,7 +169,7 @@ async fn multiple_clients() -> Result<()> {
         }));
     }
 
-    futures::future::try_join_all(tasks).await?;
+    futures_buffered::try_join_all(tasks).await?;
     Ok(())
 }
 
diff --git a/iroh/tests/sync.rs b/iroh/tests/sync.rs
index c7096aaa0a8..b5042e3f67e 100644
--- a/iroh/tests/sync.rs
+++ b/iroh/tests/sync.rs
@@ -7,7 +7,7 @@ use std::{
 
 use anyhow::{anyhow, bail, Context, Result};
 use bytes::Bytes;
-use futures::{Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use iroh::{
     client::{mem::Doc, Entry, LiveEvent},
     node::{Builder, Node},
@@ -58,7 +58,7 @@ async fn spawn_nodes(
     for i in 0..n {
         futs.push(spawn_node(i, &mut rng));
     }
-    futures::future::join_all(futs).await.into_iter().collect()
+    futures_buffered::join_all(futs).await.into_iter().collect()
 }
 
 pub fn test_rng(seed: &[u8]) -> rand_chacha::ChaCha12Rng {

From e93404700ea315a15f78e917e4890a9d13092824 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 22 Mar 2024 15:35:05 +0100
Subject: [PATCH 07/25] refactor(iroh-cli): avoid futures crate

---
 Cargo.lock                       |   8 +-
 iroh-cli/Cargo.toml              |   3 +-
 iroh-cli/src/commands/#start.rs# | 277 +++++++++++++++++++++++++++++++
 iroh-cli/src/commands/author.rs  |   2 +-
 iroh-cli/src/commands/blob.rs    |   2 +-
 iroh-cli/src/commands/doc.rs     |  38 +++--
 iroh-cli/src/commands/doctor.rs  |   4 +-
 iroh-cli/src/commands/node.rs    |   2 +-
 iroh-cli/src/commands/start.rs   |   5 +-
 iroh-cli/src/commands/tag.rs     |   2 +-
 iroh/src/node/builder.rs         |   4 +-
 11 files changed, 315 insertions(+), 32 deletions(-)
 create mode 100644 iroh-cli/src/commands/#start.rs#

diff --git a/Cargo.lock b/Cargo.lock
index a7ca3c518a1..eecefbec634 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -320,9 +320,9 @@ dependencies = [
 
 [[package]]
 name = "bao-tree"
-version = "0.11.1"
+version = "0.10.2"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "bdae307defb220bd2698a42495e226ff89e3173f024abfc2182129603e74b5c7"
+checksum = "a82f048f1b3b5ce0c7900e59aaeee3041dab048c80e5ee17ddf2823013b155c6"
 dependencies = [
  "bytes",
  "futures",
@@ -2453,8 +2453,8 @@ dependencies = [
  "dialoguer",
  "dirs-next",
  "duct",
- "flume",
- "futures",
+ "futures-buffered",
+ "futures-lite",
  "hex",
  "human-time",
  "indicatif",
diff --git a/iroh-cli/Cargo.toml b/iroh-cli/Cargo.toml
index 88a056e74af..d237f962001 100644
--- a/iroh-cli/Cargo.toml
+++ b/iroh-cli/Cargo.toml
@@ -29,7 +29,8 @@ console = { version = "0.15.5" }
 derive_more = { version = "1.0.0-beta.1", features = ["display"] }
 dialoguer = { version = "0.11.0", default-features = false }
 dirs-next = { version = "2.0.0" }
-futures = "0.3.30"
+futures-buffered = "0.2.4"
+futures-lite = "2.3"
 hex = "0.4.3"
 human-time = { version = "0.1.6" }
 indicatif = { version = "0.17", features = ["tokio"] }
diff --git a/iroh-cli/src/commands/#start.rs# b/iroh-cli/src/commands/#start.rs#
new file mode 100644
index 00000000000..9cad900e5a7
--- /dev/null
+++ b/iroh-cli/src/commands/#start.rs#
@@ -0,0 +1,277 @@
+use std::{net::SocketAddr, path::Path, time::Duration, future::Future};
+
+use anyhow::Result;
+use colored::Colorize;
+use indicatif::{ProgressBar, ProgressDrawTarget, ProgressStyle};
+use iroh::node::Node;
+use iroh::{
+    net::relay::{RelayMap, RelayMode},
+    node::RpcStatus,
+};
+use tracing::{info_span, Instrument};
+
+use crate::config::NodeConfig;
+
+/// Whether to stop the node after running a command or run forever until stopped.
+#[derive(Debug, Copy, Clone, Eq, PartialEq)]
+pub enum RunType {
+    /// Run a single command, and then shutdown the node. Allow to abort with Ctrl-C.
+    SingleCommandAbortable,
+    /// Run a single command, and then shutdown the node. Do not abort on Ctrl-C (expects Ctrl-C to be handled internally).
+    SingleCommandNoAbort,
+    /// Run until manually stopped (through Ctrl-C or shutdown RPC command)
+    UntilStopped,
+}
+
+#[derive(thiserror::Error, Debug)]
+#[error("iroh is already running on port {0}")]
+pub struct AlreadyRunningError(u16);
+
+pub async fn run_with_command<F, T>(
+    config: &NodeConfig,
+    iroh_data_root: &Path,
+    run_type: RunType,
+    command: F,
+) -> Result<()>
+where
+    F: FnOnce(iroh::client::mem::Iroh) -> T + Send + 'static,
+    T: Future<Output = Result<()>> + 'static,
+{
+    let metrics_fut = start_metrics_server(config.metrics_addr);
+
+    let res = run_with_command_inner(config, iroh_data_root, run_type, command).await;
+
+    if let Some(metrics_fut) = metrics_fut {
+        metrics_fut.abort();
+    }
+
+    let (clear_rpc, res) = match res {
+        Ok(()) => (true, res),
+        Err(e) => match e.downcast::<AlreadyRunningError>() {
+            // iroh is already running in a different process, do no remove the rpc lockfile
+            Ok(already_running) => (false, Err(already_running.into())),
+            Err(e) => (true, Err(e)),
+        },
+    };
+
+    if clear_rpc {
+        RpcStatus::clear(iroh_data_root).await?;
+    }
+
+    res
+}
+
+async fn run_with_command_inner<F, T>(
+    config: &NodeConfig,
+    iroh_data_root: &Path,
+    run_type: RunType,
+    command: F,
+) -> Result<()>
+where
+    F: FnOnce(iroh::client::mem::Iroh) -> T + Send + 'static,
+    T: Future<Output = Result<()>> + 'static,
+{
+    let relay_map = config.relay_map()?;
+
+    let spinner = create_spinner("Iroh booting...");
+    let node = start_node(iroh_data_root, relay_map).await?;
+    drop(spinner);
+
+    eprintln!("{}", welcome_message(&node)?);
+
+    let client = node.client().clone();
+
+    let mut command_task = node.local_pool_handle().spawn_pinned(move || {
+        async move {
+            match command(client).await {
+                Err(err) => Err(err),
+                Ok(()) => {
+                    // keep the task open forever if not running in single-command mode
+                    if run_type == RunType::UntilStopped {
+                        futures_lite::future::pending().await
+                    }
+                    Ok(())
+                }
+            }
+        }
+        .instrument(info_span!("command"))
+    });
+
+    let node2 = node.clone();
+    tokio::select! {
+        biased;
+        // always abort on signal-c
+        _ = tokio::signal::ctrl_c(), if run_type != RunType::SingleCommandNoAbort => {
+            command_task.abort();
+            node.shutdown();
+            // node.await?;
+        }
+        // abort if the command task finishes (will run forever if not in single-command mode)
+        res = &mut command_task => {
+            node.shutdown();
+            // let _ = node.await;
+            res??;
+        }
+        // abort if the node future completes (shutdown called or error)
+        res = node2 => {
+            command_task.abort();
+            res?;
+        }
+    }
+    Ok(())
+}
+
+pub(crate) async fn start_node(
+    iroh_data_root: &Path,
+    relay_map: Option<RelayMap>,
+) -> Result<Node<iroh::bytes::store::fs::Store>> {
+    let rpc_status = RpcStatus::load(iroh_data_root).await?;
+    match rpc_status {
+        RpcStatus::Running { port, .. } => {
+            return Err(AlreadyRunningError(port).into());
+        }
+        RpcStatus::Stopped => {
+            // all good, we can go ahead
+        }
+    }
+
+    let relay_mode = match relay_map {
+        None => RelayMode::Default,
+        Some(relay_map) => RelayMode::Custom(relay_map),
+    };
+
+    Node::persistent(iroh_data_root)
+        .await?
+        .relay_mode(relay_mode)
+        .enable_rpc()
+        .await?
+        .spawn()
+        .await
+}
+
+fn welcome_message<B: iroh::bytes::store::Store>(node: &Node<B>) -> Result<String> {
+    let msg = format!(
+        "{}\nNode ID: {}\n",
+        "Iroh is running".green(),
+        node.node_id()
+    );
+
+    Ok(msg)
+}
+
+/// Create a nice spinner.
+fn create_spinner(msg: &'static str) -> ProgressBar {
+    let pb = ProgressBar::new_spinner();
+    pb.enable_steady_tick(Duration::from_millis(80));
+    pb.set_draw_target(ProgressDrawTarget::stderr());
+    pb.set_style(
+        ProgressStyle::with_template("{spinner:.blue} {msg}")
+            .unwrap()
+            .tick_strings(&["⠋", "⠙", "⠹", "⠸", "⠼", "⠴", "⠦", "⠧", "⠇", "⠏"]),
+    );
+    pb.set_message(msg);
+    pb.with_finish(indicatif::ProgressFinish::AndClear)
+}
+
+pub fn start_metrics_server(
+    metrics_addr: Option<SocketAddr>,
+) -> Option<tokio::task::JoinHandle<()>> {
+    // doesn't start the server if the address is None
+    if let Some(metrics_addr) = metrics_addr {
+        // metrics are initilaized in iroh::node::Node::spawn
+        // here we only start the server
+        return Some(tokio::task::spawn(async move {
+            if let Err(e) = iroh_metrics::metrics::start_metrics_server(metrics_addr).await {
+                eprintln!("Failed to start metrics server: {e}");
+            }
+        }));
+    }
+    tracing::info!("Metrics server not started, no address provided");
+    None
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use anyhow::bail;
+    use iroh::util::path::IrohPaths;
+
+    #[tokio::test]
+    async fn test_run_rpc_lock_file() -> Result<()> {
+        let data_dir = tempfile::TempDir::with_prefix("rpc-lock-file-")?;
+        let lock_file_path = data_dir
+            .path()
+            .join(IrohPaths::RpcLock.with_root(data_dir.path()));
+        let data_dir_path = data_dir.path().to_path_buf();
+
+        let (ready_s, ready_r) = tokio::sync::oneshot::channel();
+        let (close_s, close_r) = tokio::sync::oneshot::channel();
+
+        // run the first start command, using channels to coordinate so we know when the node has fully booted up, and when we need to shut the node down
+        let fut1 = run_with_command(
+                &NodeConfig::default(),
+                &data_dir_path,
+                RunType::SingleCommandAbortable,
+                |_| async move {
+                    // inform the test the node is booted up
+                    ready_s.send(()).unwrap();
+
+                    // wait until the test tells us to shut down the node
+                    close_r.await?;
+                    Ok(())
+                },
+            )
+            .await;
+
+        // allow ample time for iroh to boot up
+        tokio::time::timeout(Duration::from_millis(20000), {
+            fut1.race(ready_r)
+        }).await.unwrap();
+         
+            bail!("First `run_with_command` call never started");
+        }
+
+        // ensure the rpc lock file exists
+        if !lock_file_path.try_exists()? {
+            start.abort();
+            bail!("First `run_with_command` call never made the rpc lockfile");
+        }
+
+        // run the second command, this should fail
+        if run_with_command(
+            &NodeConfig::default(),
+            data_dir.path(),
+            RunType::SingleCommandAbortable,
+            |_| async move { Ok(()) },
+        )
+        .await
+        .is_ok()
+        {
+            start.abort();
+            bail!("Second `run_with_command` call should return error");
+        }
+
+        // ensure the rpc lock file still exists
+        if !lock_file_path.try_exists()? {
+            start.abort();
+            bail!("Second `run_with_command` removed the rpc lockfile");
+        }
+
+        // inform the node it should close
+        close_s.send(()).unwrap();
+
+        // wait for the node to close
+        if tokio::time::timeout(Duration::from_millis(1000), start)
+            .await
+            .is_err()
+        {
+            bail!("First `run_with_command` never closed");
+        }
+
+        // ensure the lockfile no longer exists
+        if lock_file_path.try_exists()? {
+            bail!("First `run_with_command` closed without removing the rpc lockfile");
+        }
+        Ok(())
+    }
+}
diff --git a/iroh-cli/src/commands/author.rs b/iroh-cli/src/commands/author.rs
index c584d9d0285..c512292e42e 100644
--- a/iroh-cli/src/commands/author.rs
+++ b/iroh-cli/src/commands/author.rs
@@ -1,6 +1,6 @@
 use anyhow::{bail, Result};
 use clap::Parser;
-use futures::TryStreamExt;
+use futures_lite::StreamExt;
 use iroh::base::base32::fmt_short;
 
 use iroh::sync::AuthorId;
diff --git a/iroh-cli/src/commands/blob.rs b/iroh-cli/src/commands/blob.rs
index 5c8fbc49d89..09aa6aee166 100644
--- a/iroh-cli/src/commands/blob.rs
+++ b/iroh-cli/src/commands/blob.rs
@@ -8,7 +8,7 @@ use std::{
 use anyhow::{anyhow, bail, ensure, Context, Result};
 use clap::Subcommand;
 use console::{style, Emoji};
-use futures::{Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use indicatif::{
     HumanBytes, HumanDuration, MultiProgress, ProgressBar, ProgressDrawTarget, ProgressState,
     ProgressStyle,
diff --git a/iroh-cli/src/commands/doc.rs b/iroh-cli/src/commands/doc.rs
index c04439d6b55..c19dabee89d 100644
--- a/iroh-cli/src/commands/doc.rs
+++ b/iroh-cli/src/commands/doc.rs
@@ -10,7 +10,8 @@ use anyhow::{anyhow, bail, Context, Result};
 use clap::Parser;
 use colored::Colorize;
 use dialoguer::Confirm;
-use futures::{Stream, StreamExt, TryStreamExt};
+use futures_buffered::BufferedStreamExt;
+use futures_lite::{Stream, StreamExt};
 use indicatif::{HumanBytes, HumanDuration, MultiProgress, ProgressBar, ProgressStyle};
 use iroh::base::base32::fmt_short;
 use quic_rpc::ServiceConnection;
@@ -799,8 +800,11 @@ where
         (String, u64, Option<Hash>, u64),
     >::new()));
 
-    let _stats: Vec<u64> = blob_add_progress
-        .filter_map(|item| async {
+    let doc2 = doc.clone();
+    let imp2 = task_imp.clone();
+
+    let _stats: Vec<_> = blob_add_progress
+        .filter_map(|item| {
             let item = match item.context("Error adding files") {
                 Err(e) => return Some(Err(e)),
                 Ok(item) => item,
@@ -871,20 +875,22 @@ where
                 }
             }
         })
-        .try_chunks(1024)
-        .map_ok(|chunks| {
-            futures::stream::iter(chunks.into_iter().map(|(key, hash, size)| {
-                let doc = doc.clone();
-                let imp = task_imp.clone();
-                Ok(async move {
-                    doc.set_hash(author_id, key, hash, size).await?;
-                    imp.import_progress();
-                    anyhow::Ok(size)
-                })
-            }))
+        .map(move |res| {
+            let doc = doc2.clone();
+            let imp = imp2.clone();
+            async move {
+                match res {
+                    Ok((key, hash, size)) => {
+                        let doc = doc.clone();
+                        doc.set_hash(author_id, key, hash, size).await?;
+                        imp.import_progress();
+                        Ok(size)
+                    }
+                    Err(err) => Err(err),
+                }
+            }
         })
-        .try_flatten()
-        .try_buffer_unordered(64)
+        .buffered_unordered(128)
         .try_collect()
         .await?;
 
diff --git a/iroh-cli/src/commands/doctor.rs b/iroh-cli/src/commands/doctor.rs
index dba86dcbeac..e929670f074 100644
--- a/iroh-cli/src/commands/doctor.rs
+++ b/iroh-cli/src/commands/doctor.rs
@@ -14,7 +14,7 @@ use crate::config::{iroh_data_root, NodeConfig};
 
 use anyhow::Context;
 use clap::Subcommand;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use indicatif::{HumanBytes, MultiProgress, ProgressBar};
 use iroh::{
     base::ticket::Ticket,
@@ -215,7 +215,7 @@ fn update_pb(
             }
         })
     } else {
-        tokio::spawn(futures::future::ready(()))
+        tokio::spawn(std::future::ready(()))
     }
 }
 
diff --git a/iroh-cli/src/commands/node.rs b/iroh-cli/src/commands/node.rs
index 695e3e8dcc4..e7c75b92efd 100644
--- a/iroh-cli/src/commands/node.rs
+++ b/iroh-cli/src/commands/node.rs
@@ -5,7 +5,7 @@ use clap::Subcommand;
 use colored::Colorize;
 use comfy_table::Table;
 use comfy_table::{presets::NOTHING, Cell};
-use futures::{Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use human_time::ToHumanTimeString;
 use iroh::client::Iroh;
 use iroh::net::{key::PublicKey, magic_endpoint::ConnectionInfo, magicsock::DirectAddrInfo};
diff --git a/iroh-cli/src/commands/start.rs b/iroh-cli/src/commands/start.rs
index ad523507a2f..2a56b40e13b 100644
--- a/iroh-cli/src/commands/start.rs
+++ b/iroh-cli/src/commands/start.rs
@@ -1,9 +1,8 @@
-use std::{net::SocketAddr, path::Path, time::Duration};
+use std::{future::Future, net::SocketAddr, path::Path, time::Duration};
 
 use crate::config::NodeConfig;
 use anyhow::Result;
 use colored::Colorize;
-use futures::Future;
 use indicatif::{ProgressBar, ProgressDrawTarget, ProgressStyle};
 use iroh::node::Node;
 use iroh::{
@@ -88,7 +87,7 @@ where
                 Ok(()) => {
                     // keep the task open forever if not running in single-command mode
                     if run_type == RunType::UntilStopped {
-                        futures::future::pending().await
+                        futures_lite::future::pending().await
                     }
                     Ok(())
                 }
diff --git a/iroh-cli/src/commands/tag.rs b/iroh-cli/src/commands/tag.rs
index f3b2d011f76..69edf005c34 100644
--- a/iroh-cli/src/commands/tag.rs
+++ b/iroh-cli/src/commands/tag.rs
@@ -1,7 +1,7 @@
 use anyhow::Result;
 use bytes::Bytes;
 use clap::Subcommand;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use iroh::bytes::Tag;
 use iroh::{client::Iroh, rpc_protocol::ProviderService};
 use quic_rpc::ServiceConnection;
diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs
index 77f49df7aa8..b831d291f14 100644
--- a/iroh/src/node/builder.rs
+++ b/iroh/src/node/builder.rs
@@ -382,13 +382,13 @@ where
             )
         };
 
-        let task = Arc::new(
+        /*let task = Arc::new(
             async move {
                 task.await?;
                 anyhow::Ok(())
             }
             .boxed(),
-        );
+        );*/
 
         let node = Node {
             inner,

From ab582b1d320003375848247c7a82df07449ef1bf Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 25 Mar 2024 13:20:23 +0100
Subject: [PATCH 08/25] fixup

---
 Cargo.lock                      | 5 +++--
 iroh-bytes/src/store/traits.rs  | 6 +++---
 iroh-bytes/src/util/progress.rs | 8 +++++---
 3 files changed, 11 insertions(+), 8 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index eecefbec634..78e4c76b1ff 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -320,9 +320,9 @@ dependencies = [
 
 [[package]]
 name = "bao-tree"
-version = "0.10.2"
+version = "0.11.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a82f048f1b3b5ce0c7900e59aaeee3041dab048c80e5ee17ddf2823013b155c6"
+checksum = "bdae307defb220bd2698a42495e226ff89e3173f024abfc2182129603e74b5c7"
 dependencies = [
  "bytes",
  "futures",
@@ -2453,6 +2453,7 @@ dependencies = [
  "dialoguer",
  "dirs-next",
  "duct",
+ "flume",
  "futures-buffered",
  "futures-lite",
  "hex",
diff --git a/iroh-bytes/src/store/traits.rs b/iroh-bytes/src/store/traits.rs
index 3f4b1c24d68..5b664692d5c 100644
--- a/iroh-bytes/src/store/traits.rs
+++ b/iroh-bytes/src/store/traits.rs
@@ -6,7 +6,7 @@ use bao_tree::{
     BaoTree, ByteNum, ChunkRanges,
 };
 use bytes::Bytes;
-use futures_lite::Stream;
+use futures_lite::{Stream, StreamExt};
 use genawaiter::rc::{Co, Gen};
 use iroh_base::rpc::RpcError;
 use iroh_io::AsyncSliceReader;
@@ -430,7 +430,7 @@ async fn validate_impl(
         total: complete.len() as u64,
     })
     .await?;
-    let complete_result = futures::stream::iter(complete)
+    let complete_result = futures_lite::stream::iter(complete)
         .map(|hash| {
             let store = store.clone();
             let tx = tx.clone();
@@ -479,7 +479,7 @@ async fn validate_impl(
         .buffered_unordered(validate_parallelism)
         .collect::<Vec<_>>()
         .await;
-    let partial_result = futures::stream::iter(partial)
+    let partial_result = futures_lite::stream::iter(partial)
         .map(|hash| {
             let store = store.clone();
             let tx = tx.clone();
diff --git a/iroh-bytes/src/util/progress.rs b/iroh-bytes/src/util/progress.rs
index 5e0731185e1..2c370af9fbf 100644
--- a/iroh-bytes/src/util/progress.rs
+++ b/iroh-bytes/src/util/progress.rs
@@ -146,13 +146,15 @@ impl<T> std::fmt::Debug for BoxedProgressSender<T> {
     }
 }
 
+type BoxFuture<'a, T> = std::pin::Pin<Box<dyn Future<Output = T> + Send + 'a>>;
+
 /// Boxable progress sender
 trait BoxableProgressSender<T>: IdGenerator + std::fmt::Debug + Send + Sync + 'static {
     /// Send a message and wait if the receiver is full.
     ///
     /// Use this to send important progress messages where delivery must be guaranteed.
     #[must_use]
-    fn send(&self, msg: T) -> BoxFuture<ProgressSendResult<()>>;
+    fn send(&self, msg: T) -> BoxFuture<'_, ProgressSendResult<()>>;
 
     /// Try to send a message and drop it if the receiver is full.
     ///
@@ -168,8 +170,8 @@ trait BoxableProgressSender<T>: IdGenerator + std::fmt::Debug + Send + Sync + 's
 impl<I: ProgressSender + IdGenerator> BoxableProgressSender<I::Msg>
     for BoxableProgressSenderWrapper<I>
 {
-    fn send(&self, msg: I::Msg) -> BoxFuture<ProgressSendResult<()>> {
-        self.0.send(msg).boxed()
+    fn send(&self, msg: I::Msg) -> BoxFuture<'_, ProgressSendResult<()>> {
+        Box::pin(self.0.send(msg))
     }
 
     fn try_send(&self, msg: I::Msg) -> ProgressSendResult<()> {

From 887eb6cd80a00618ca528d191a9c4ff05ec02ce9 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 25 Mar 2024 13:37:19 +0100
Subject: [PATCH 09/25] update shutdown logic

---
 iroh-cli/src/commands/start.rs       | 12 ++----------
 iroh/examples/collection-provide.rs  |  2 +-
 iroh/examples/hello-world-provide.rs |  2 +-
 iroh/src/client/blobs.rs             | 10 ++++++----
 iroh/src/node.rs                     | 29 +++++++++++-----------------
 iroh/src/node/builder.rs             | 12 ++----------
 iroh/src/node/rpc.rs                 |  2 +-
 iroh/src/sync_engine.rs              | 21 ++++++++++----------
 iroh/src/sync_engine/gossip.rs       |  2 +-
 iroh/src/sync_engine/rpc.rs          |  1 -
 iroh/tests/gc.rs                     |  6 ++----
 iroh/tests/provide.rs                | 18 ++++++++---------
 iroh/tests/sync.rs                   |  8 ++++----
 13 files changed, 51 insertions(+), 74 deletions(-)

diff --git a/iroh-cli/src/commands/start.rs b/iroh-cli/src/commands/start.rs
index 2a56b40e13b..196022bdce5 100644
--- a/iroh-cli/src/commands/start.rs
+++ b/iroh-cli/src/commands/start.rs
@@ -96,26 +96,18 @@ where
         .instrument(info_span!("command"))
     });
 
-    let node2 = node.clone();
     tokio::select! {
         biased;
         // always abort on signal-c
         _ = tokio::signal::ctrl_c(), if run_type != RunType::SingleCommandNoAbort => {
             command_task.abort();
-            node.shutdown();
-            node.await?;
+            node.shutdown().await?;
         }
         // abort if the command task finishes (will run forever if not in single-command mode)
         res = &mut command_task => {
-            node.shutdown();
-            let _ = node.await;
+            let _ = node.shutdown().await;
             res??;
         }
-        // abort if the node future completes (shutdown called or error)
-        res = node2 => {
-            command_task.abort();
-            res?;
-        }
     }
     Ok(())
 }
diff --git a/iroh/examples/collection-provide.rs b/iroh/examples/collection-provide.rs
index 1ef27561c3b..3269ced626a 100644
--- a/iroh/examples/collection-provide.rs
+++ b/iroh/examples/collection-provide.rs
@@ -66,6 +66,6 @@ async fn main() -> anyhow::Result<()> {
     println!("\tcargo run --example collection-fetch {}", ticket);
     // wait for the node to finish, this will block indefinitely
     // stop with SIGINT (ctrl+c)
-    node.await?;
+    node.shutdown().await?;
     Ok(())
 }
diff --git a/iroh/examples/hello-world-provide.rs b/iroh/examples/hello-world-provide.rs
index 663895e5f49..0f0be772cdb 100644
--- a/iroh/examples/hello-world-provide.rs
+++ b/iroh/examples/hello-world-provide.rs
@@ -48,6 +48,6 @@ async fn main() -> anyhow::Result<()> {
     println!("\t cargo run --example hello-world-fetch {}", ticket);
     // wait for the node to finish, this will block indefinitely
     // stop with SIGINT (ctrl+c)
-    node.await?;
+    node.shutdown().await?;
     Ok(())
 }
diff --git a/iroh/src/client/blobs.rs b/iroh/src/client/blobs.rs
index afca401b13f..bdab2fc7926 100644
--- a/iroh/src/client/blobs.rs
+++ b/iroh/src/client/blobs.rs
@@ -224,7 +224,7 @@ where
             .rpc
             .server_streaming(BlobConsistencyCheckRequest { repair })
             .await?;
-        Ok(stream.map_err(anyhow::Error::from))
+        Ok(stream.map(|r| r.map_err(anyhow::Error::from)))
     }
 
     /// Download a blob from another node and add it to the local database.
@@ -258,7 +258,9 @@ where
             mode,
         };
         let stream = self.rpc.server_streaming(req).await?;
-        Ok(BlobExportProgress::new(stream.map_err(anyhow::Error::from)))
+        Ok(BlobExportProgress::new(
+            stream.map(|r| r.map_err(anyhow::Error::from)),
+        ))
     }
 
     /// List all complete blobs.
@@ -624,7 +626,7 @@ impl BlobExportProgress {
 impl Stream for BlobExportProgress {
     type Item = Result<ExportProgress>;
     fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
-        self.stream.poll_next_unpin(cx)
+        Pin::new(&mut self.stream).poll_next(cx)
     }
 }
 
@@ -633,7 +635,7 @@ impl Future for BlobExportProgress {
 
     fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
         loop {
-            match self.stream.poll_next_unpin(cx) {
+            match Pin::new(&mut self.stream).poll_next(cx) {
                 Poll::Pending => return Poll::Pending,
                 Poll::Ready(None) => {
                     return Poll::Ready(Err(anyhow!("Response stream ended prematurely")))
diff --git a/iroh/src/node.rs b/iroh/src/node.rs
index a7a28f50223..1fc7e4d3950 100644
--- a/iroh/src/node.rs
+++ b/iroh/src/node.rs
@@ -6,12 +6,9 @@
 //!
 //! To shut down the node, call [`Node::shutdown`].
 use std::fmt::Debug;
-use std::future::Future;
 use std::net::SocketAddr;
 use std::path::Path;
-use std::pin::Pin;
 use std::sync::Arc;
-use std::task::Poll;
 
 use anyhow::{anyhow, Result};
 use futures_lite::{future::Boxed as BoxFuture, FutureExt, StreamExt};
@@ -28,7 +25,7 @@ use iroh_net::{
 use quic_rpc::transport::flume::FlumeConnection;
 use quic_rpc::RpcClient;
 use tokio::sync::{mpsc, RwLock};
-use tokio::task::JoinError;
+use tokio::task::JoinHandle;
 use tokio_util::sync::CancellationToken;
 use tokio_util::task::LocalPoolHandle;
 use tracing::debug;
@@ -89,7 +86,7 @@ impl iroh_bytes::provider::EventSender for Callbacks {
 #[derive(Debug, Clone)]
 pub struct Node<D> {
     inner: Arc<NodeInner<D>>,
-    task: (), // Arc<BoxFuture<anyhow::Result<()>>>,
+    task: Arc<JoinHandle<()>>,
     client: crate::client::mem::Iroh,
 }
 
@@ -234,12 +231,18 @@ impl<D: BaoStore> Node<D> {
     /// Aborts the node.
     ///
     /// This does not gracefully terminate currently: all connections are closed and
-    /// anything in-transit is lost.  The task will stop running and awaiting this
-    /// [`Node`] will complete.
+    /// anything in-transit is lost.  The task will stop running.
+    /// If this is the last copy of the `Node`, this will finish once the task is
+    /// fully shutdown.
     ///
     /// The shutdown behaviour will become more graceful in the future.
-    pub fn shutdown(&self) {
+    pub async fn shutdown(self) -> Result<()> {
         self.inner.cancel_token.cancel();
+
+        if let Ok(task) = Arc::try_unwrap(self.task) {
+            task.await?;
+        }
+        Ok(())
     }
 
     /// Returns a token that can be used to cancel the node.
@@ -248,16 +251,6 @@ impl<D: BaoStore> Node<D> {
     }
 }
 
-/// The future completes when the spawned tokio task finishes.
-impl<D> Future for Node<D> {
-    type Output = Result<(), Arc<JoinError>>;
-
-    fn poll(mut self: Pin<&mut Self>, cx: &mut std::task::Context<'_>) -> Poll<Self::Output> {
-        // Pin::new(&mut self.task).poll(cx)
-        todo!()
-    }
-}
-
 impl<D> std::ops::Deref for Node<D> {
     type Target = crate::client::mem::Iroh;
 
diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs
index b831d291f14..79f68050768 100644
--- a/iroh/src/node/builder.rs
+++ b/iroh/src/node/builder.rs
@@ -7,7 +7,7 @@ use std::{
 };
 
 use anyhow::{bail, Context, Result};
-use futures_lite::{FutureExt, StreamExt};
+use futures_lite::StreamExt;
 use iroh_base::key::SecretKey;
 use iroh_bytes::{
     downloader::Downloader,
@@ -382,17 +382,9 @@ where
             )
         };
 
-        /*let task = Arc::new(
-            async move {
-                task.await?;
-                anyhow::Ok(())
-            }
-            .boxed(),
-        );*/
-
         let node = Node {
             inner,
-            task: (),
+            task: Arc::new(task),
             client,
         };
 
diff --git a/iroh/src/node/rpc.rs b/iroh/src/node/rpc.rs
index 0aef68ca24d..4d346705e77 100644
--- a/iroh/src/node/rpc.rs
+++ b/iroh/src/node/rpc.rs
@@ -6,7 +6,7 @@ use std::time::Duration;
 
 use anyhow::{anyhow, Result};
 use futures_buffered::BufferedStreamExt;
-use futures_lite::{FutureExt, Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use genawaiter::sync::{Co, Gen};
 use iroh_base::rpc::RpcResult;
 use iroh_bytes::export::ExportProgress;
diff --git a/iroh/src/sync_engine.rs b/iroh/src/sync_engine.rs
index 67e56efee52..32cb1c6b94b 100644
--- a/iroh/src/sync_engine.rs
+++ b/iroh/src/sync_engine.rs
@@ -5,7 +5,7 @@
 use std::{io, sync::Arc};
 
 use anyhow::Result;
-use futures_lite::{future::Boxed as BoxFuture, FutureExt, Stream, StreamExt};
+use futures_lite::{Stream, StreamExt};
 use iroh_bytes::downloader::Downloader;
 use iroh_bytes::{store::EntryStatus, Hash};
 use iroh_gossip::net::Gossip;
@@ -13,6 +13,7 @@ use iroh_net::{key::PublicKey, MagicEndpoint, NodeAddr};
 use iroh_sync::{actor::SyncHandle, ContentStatus, ContentStatusCallback, Entry, NamespaceId};
 use serde::{Deserialize, Serialize};
 use tokio::sync::{mpsc, oneshot};
+use tokio::task::JoinHandle;
 use tracing::{error, error_span, Instrument};
 
 mod gossip;
@@ -42,8 +43,8 @@ pub struct SyncEngine {
     pub(crate) endpoint: MagicEndpoint,
     pub(crate) sync: SyncHandle,
     to_live_actor: mpsc::Sender<ToLiveActor>,
-    #[debug("Arc<BoxFuture<()>>")]
-    tasks_fut: (), // Arc<BoxFuture<()>>,
+    #[debug("Arc<JoinHandle<()>>")]
+    tasks: Arc<JoinHandle<()>>,
     #[debug("ContentStatusCallback")]
     content_status_cb: ContentStatusCallback,
 }
@@ -107,7 +108,7 @@ impl SyncEngine {
             }
             .instrument(error_span!("sync", %me)),
         );
-        let tasks_fut = async move {
+        let tasks = tokio::task::spawn(async move {
             if let Err(err) = live_actor_task.await {
                 error!("Error while joining actor task: {err:?}");
             }
@@ -117,14 +118,13 @@ impl SyncEngine {
                     error!("Error while joining gossip recv task task: {err:?}");
                 }
             }
-        }
-        .boxed();
+        });
 
         Self {
             endpoint,
             sync,
             to_live_actor: live_actor_tx,
-            tasks_fut: (),
+            tasks: Arc::new(tasks),
             content_status_cb,
         }
     }
@@ -223,10 +223,11 @@ impl SyncEngine {
     }
 
     /// Shutdown the sync engine.
-    pub async fn shutdown(&self) -> Result<()> {
+    pub async fn shutdown(self) -> Result<()> {
         self.to_live_actor.send(ToLiveActor::Shutdown).await?;
-        // TODO
-        // self.tasks_fut.clone().await;
+        if let Ok(tasks) = Arc::try_unwrap(self.tasks) {
+            tasks.await?;
+        }
         Ok(())
     }
 }
diff --git a/iroh/src/sync_engine/gossip.rs b/iroh/src/sync_engine/gossip.rs
index c8f6edd32dc..0a6eb76d771 100644
--- a/iroh/src/sync_engine/gossip.rs
+++ b/iroh/src/sync_engine/gossip.rs
@@ -1,7 +1,7 @@
 use std::collections::HashSet;
 
 use anyhow::{anyhow, Context, Result};
-use futures_lite::{FutureExt, StreamExt};
+use futures_lite::StreamExt;
 use iroh_gossip::{
     net::{Event, Gossip},
     proto::TopicId,
diff --git a/iroh/src/sync_engine/rpc.rs b/iroh/src/sync_engine/rpc.rs
index aabc28d2a2e..41aced676d6 100644
--- a/iroh/src/sync_engine/rpc.rs
+++ b/iroh/src/sync_engine/rpc.rs
@@ -7,7 +7,6 @@ use iroh_sync::{Author, NamespaceSecret};
 use tokio_stream::StreamExt;
 
 use crate::rpc_protocol::{DocGetSyncPeersRequest, DocGetSyncPeersResponse};
-use crate::sync_engine::LiveEvent;
 use crate::{
     rpc_protocol::{
         AuthorCreateRequest, AuthorCreateResponse, AuthorListRequest, AuthorListResponse,
diff --git a/iroh/tests/gc.rs b/iroh/tests/gc.rs
index 12db9226ae3..e6f79e0dd22 100644
--- a/iroh/tests/gc.rs
+++ b/iroh/tests/gc.rs
@@ -121,8 +121,7 @@ async fn gc_basics() -> Result<()> {
     step(&evs).await;
     assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::NotFound);
 
-    node.shutdown();
-    node.await?;
+    node.shutdown().await?;
     Ok(())
 }
 
@@ -180,8 +179,7 @@ async fn gc_hashseq_impl() -> Result<()> {
     assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::NotFound);
     assert_eq!(bao_store.entry_status(&hr).await?, EntryStatus::NotFound);
 
-    node.shutdown();
-    node.await?;
+    node.shutdown().await?;
     Ok(())
 }
 
diff --git a/iroh/tests/provide.rs b/iroh/tests/provide.rs
index 3911ce051c2..72bd3261583 100644
--- a/iroh/tests/provide.rs
+++ b/iroh/tests/provide.rs
@@ -5,7 +5,7 @@ use std::{
     time::{Duration, Instant},
 };
 
-use anyhow::{anyhow, Context, Result};
+use anyhow::{anyhow, Result};
 use bytes::Bytes;
 use futures_lite::FutureExt;
 use iroh::{
@@ -263,8 +263,7 @@ where
     .await
     .expect("duration expired");
 
-    node.shutdown();
-    node.await?;
+    node.shutdown().await?;
 
     assert_events(events, num_blobs + 1);
 
@@ -314,7 +313,7 @@ async fn test_server_close() {
     let child_hash = db.insert(b"hello there");
     let collection = Collection::from_iter([("hello", child_hash)]);
     let hash = db.insert_many(collection.to_blobs()).unwrap();
-    let mut node = test_node(db).spawn().await.unwrap();
+    let node = test_node(db).spawn().await.unwrap();
     let node_addr = node.local_endpoint_addresses().await.unwrap();
     let peer_id = node.node_id();
 
@@ -337,11 +336,12 @@ async fn test_server_close() {
         loop {
             tokio::select! {
                 biased;
-                res = &mut node => break res.context("provider failed"),
                 maybe_event = events_recv.recv() => {
                     match maybe_event {
                         Some(event) => match event {
-                            Event::ByteProvide(provider::Event::TransferCompleted { .. }) => node.shutdown(),
+                            Event::ByteProvide(provider::Event::TransferCompleted { .. }) => {
+                                return node.shutdown().await;
+                            },
                             Event::ByteProvide(provider::Event::TransferAborted { .. }) => {
                                 break Err(anyhow!("transfer aborted"));
                             }
@@ -353,9 +353,9 @@ async fn test_server_close() {
             }
         }
     })
-        .await
-        .expect("supervisor timeout")
-        .expect("supervisor failed");
+    .await
+    .expect("supervisor timeout")
+    .expect("supervisor failed");
 }
 
 /// create an in memory test database containing the given entries and an iroh collection of all entries
diff --git a/iroh/tests/sync.rs b/iroh/tests/sync.rs
index b5042e3f67e..e4fe503c1ec 100644
--- a/iroh/tests/sync.rs
+++ b/iroh/tests/sync.rs
@@ -115,7 +115,7 @@ async fn sync_simple() -> Result<()> {
     .await;
 
     for node in nodes {
-        node.shutdown();
+        node.shutdown().await?;
     }
     Ok(())
 }
@@ -136,7 +136,7 @@ async fn sync_subscribe_no_sync() -> Result<()> {
         matches!(event, Some(Ok(LiveEvent::InsertLocal { .. }))),
         "expected InsertLocal but got {event:?}"
     );
-    node.shutdown();
+    node.shutdown().await?;
     Ok(())
 }
 
@@ -389,7 +389,7 @@ async fn sync_full_basic() -> Result<()> {
 
     info!("shutdown");
     for node in nodes {
-        node.shutdown();
+        node.shutdown().await?;
     }
 
     Ok(())
@@ -878,7 +878,7 @@ async fn doc_delete() -> Result<()> {
     tokio::time::sleep(Duration::from_millis(200)).await;
     let bytes = client.blobs.read_to_bytes(hash).await;
     assert!(bytes.is_err());
-    node.shutdown();
+    node.shutdown().await?;
     Ok(())
 }
 

From 871f1804cfc5ca5cb521f813106dad10da02d8a4 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 25 Mar 2024 14:47:31 +0100
Subject: [PATCH 10/25] fixup

---
 iroh-net/src/net/interfaces/linux.rs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/iroh-net/src/net/interfaces/linux.rs b/iroh-net/src/net/interfaces/linux.rs
index d0e4d6f23ee..8b0d139913e 100644
--- a/iroh-net/src/net/interfaces/linux.rs
+++ b/iroh-net/src/net/interfaces/linux.rs
@@ -2,7 +2,7 @@
 
 use anyhow::{anyhow, Result};
 #[cfg(not(target_os = "android"))]
-use futures::TryStreamExt;
+use futures_lite::StreamExt;
 use tokio::fs::File;
 use tokio::io::{AsyncBufReadExt, BufReader};
 

From 6ee446be57a8e999c554da68604b1af0300befd5 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 25 Mar 2024 17:04:25 +0100
Subject: [PATCH 11/25] cleanup

---
 iroh-cli/src/commands/#start.rs#  | 277 ------------------------------
 iroh-net/src/relay/client_conn.rs |   1 -
 iroh-sync/Cargo.toml              |   2 +-
 iroh/src/node/rpc.rs              |   2 +-
 4 files changed, 2 insertions(+), 280 deletions(-)
 delete mode 100644 iroh-cli/src/commands/#start.rs#

diff --git a/iroh-cli/src/commands/#start.rs# b/iroh-cli/src/commands/#start.rs#
deleted file mode 100644
index 9cad900e5a7..00000000000
--- a/iroh-cli/src/commands/#start.rs#
+++ /dev/null
@@ -1,277 +0,0 @@
-use std::{net::SocketAddr, path::Path, time::Duration, future::Future};
-
-use anyhow::Result;
-use colored::Colorize;
-use indicatif::{ProgressBar, ProgressDrawTarget, ProgressStyle};
-use iroh::node::Node;
-use iroh::{
-    net::relay::{RelayMap, RelayMode},
-    node::RpcStatus,
-};
-use tracing::{info_span, Instrument};
-
-use crate::config::NodeConfig;
-
-/// Whether to stop the node after running a command or run forever until stopped.
-#[derive(Debug, Copy, Clone, Eq, PartialEq)]
-pub enum RunType {
-    /// Run a single command, and then shutdown the node. Allow to abort with Ctrl-C.
-    SingleCommandAbortable,
-    /// Run a single command, and then shutdown the node. Do not abort on Ctrl-C (expects Ctrl-C to be handled internally).
-    SingleCommandNoAbort,
-    /// Run until manually stopped (through Ctrl-C or shutdown RPC command)
-    UntilStopped,
-}
-
-#[derive(thiserror::Error, Debug)]
-#[error("iroh is already running on port {0}")]
-pub struct AlreadyRunningError(u16);
-
-pub async fn run_with_command<F, T>(
-    config: &NodeConfig,
-    iroh_data_root: &Path,
-    run_type: RunType,
-    command: F,
-) -> Result<()>
-where
-    F: FnOnce(iroh::client::mem::Iroh) -> T + Send + 'static,
-    T: Future<Output = Result<()>> + 'static,
-{
-    let metrics_fut = start_metrics_server(config.metrics_addr);
-
-    let res = run_with_command_inner(config, iroh_data_root, run_type, command).await;
-
-    if let Some(metrics_fut) = metrics_fut {
-        metrics_fut.abort();
-    }
-
-    let (clear_rpc, res) = match res {
-        Ok(()) => (true, res),
-        Err(e) => match e.downcast::<AlreadyRunningError>() {
-            // iroh is already running in a different process, do no remove the rpc lockfile
-            Ok(already_running) => (false, Err(already_running.into())),
-            Err(e) => (true, Err(e)),
-        },
-    };
-
-    if clear_rpc {
-        RpcStatus::clear(iroh_data_root).await?;
-    }
-
-    res
-}
-
-async fn run_with_command_inner<F, T>(
-    config: &NodeConfig,
-    iroh_data_root: &Path,
-    run_type: RunType,
-    command: F,
-) -> Result<()>
-where
-    F: FnOnce(iroh::client::mem::Iroh) -> T + Send + 'static,
-    T: Future<Output = Result<()>> + 'static,
-{
-    let relay_map = config.relay_map()?;
-
-    let spinner = create_spinner("Iroh booting...");
-    let node = start_node(iroh_data_root, relay_map).await?;
-    drop(spinner);
-
-    eprintln!("{}", welcome_message(&node)?);
-
-    let client = node.client().clone();
-
-    let mut command_task = node.local_pool_handle().spawn_pinned(move || {
-        async move {
-            match command(client).await {
-                Err(err) => Err(err),
-                Ok(()) => {
-                    // keep the task open forever if not running in single-command mode
-                    if run_type == RunType::UntilStopped {
-                        futures_lite::future::pending().await
-                    }
-                    Ok(())
-                }
-            }
-        }
-        .instrument(info_span!("command"))
-    });
-
-    let node2 = node.clone();
-    tokio::select! {
-        biased;
-        // always abort on signal-c
-        _ = tokio::signal::ctrl_c(), if run_type != RunType::SingleCommandNoAbort => {
-            command_task.abort();
-            node.shutdown();
-            // node.await?;
-        }
-        // abort if the command task finishes (will run forever if not in single-command mode)
-        res = &mut command_task => {
-            node.shutdown();
-            // let _ = node.await;
-            res??;
-        }
-        // abort if the node future completes (shutdown called or error)
-        res = node2 => {
-            command_task.abort();
-            res?;
-        }
-    }
-    Ok(())
-}
-
-pub(crate) async fn start_node(
-    iroh_data_root: &Path,
-    relay_map: Option<RelayMap>,
-) -> Result<Node<iroh::bytes::store::fs::Store>> {
-    let rpc_status = RpcStatus::load(iroh_data_root).await?;
-    match rpc_status {
-        RpcStatus::Running { port, .. } => {
-            return Err(AlreadyRunningError(port).into());
-        }
-        RpcStatus::Stopped => {
-            // all good, we can go ahead
-        }
-    }
-
-    let relay_mode = match relay_map {
-        None => RelayMode::Default,
-        Some(relay_map) => RelayMode::Custom(relay_map),
-    };
-
-    Node::persistent(iroh_data_root)
-        .await?
-        .relay_mode(relay_mode)
-        .enable_rpc()
-        .await?
-        .spawn()
-        .await
-}
-
-fn welcome_message<B: iroh::bytes::store::Store>(node: &Node<B>) -> Result<String> {
-    let msg = format!(
-        "{}\nNode ID: {}\n",
-        "Iroh is running".green(),
-        node.node_id()
-    );
-
-    Ok(msg)
-}
-
-/// Create a nice spinner.
-fn create_spinner(msg: &'static str) -> ProgressBar {
-    let pb = ProgressBar::new_spinner();
-    pb.enable_steady_tick(Duration::from_millis(80));
-    pb.set_draw_target(ProgressDrawTarget::stderr());
-    pb.set_style(
-        ProgressStyle::with_template("{spinner:.blue} {msg}")
-            .unwrap()
-            .tick_strings(&["⠋", "⠙", "⠹", "⠸", "⠼", "⠴", "⠦", "⠧", "⠇", "⠏"]),
-    );
-    pb.set_message(msg);
-    pb.with_finish(indicatif::ProgressFinish::AndClear)
-}
-
-pub fn start_metrics_server(
-    metrics_addr: Option<SocketAddr>,
-) -> Option<tokio::task::JoinHandle<()>> {
-    // doesn't start the server if the address is None
-    if let Some(metrics_addr) = metrics_addr {
-        // metrics are initilaized in iroh::node::Node::spawn
-        // here we only start the server
-        return Some(tokio::task::spawn(async move {
-            if let Err(e) = iroh_metrics::metrics::start_metrics_server(metrics_addr).await {
-                eprintln!("Failed to start metrics server: {e}");
-            }
-        }));
-    }
-    tracing::info!("Metrics server not started, no address provided");
-    None
-}
-
-#[cfg(test)]
-mod tests {
-    use super::*;
-    use anyhow::bail;
-    use iroh::util::path::IrohPaths;
-
-    #[tokio::test]
-    async fn test_run_rpc_lock_file() -> Result<()> {
-        let data_dir = tempfile::TempDir::with_prefix("rpc-lock-file-")?;
-        let lock_file_path = data_dir
-            .path()
-            .join(IrohPaths::RpcLock.with_root(data_dir.path()));
-        let data_dir_path = data_dir.path().to_path_buf();
-
-        let (ready_s, ready_r) = tokio::sync::oneshot::channel();
-        let (close_s, close_r) = tokio::sync::oneshot::channel();
-
-        // run the first start command, using channels to coordinate so we know when the node has fully booted up, and when we need to shut the node down
-        let fut1 = run_with_command(
-                &NodeConfig::default(),
-                &data_dir_path,
-                RunType::SingleCommandAbortable,
-                |_| async move {
-                    // inform the test the node is booted up
-                    ready_s.send(()).unwrap();
-
-                    // wait until the test tells us to shut down the node
-                    close_r.await?;
-                    Ok(())
-                },
-            )
-            .await;
-
-        // allow ample time for iroh to boot up
-        tokio::time::timeout(Duration::from_millis(20000), {
-            fut1.race(ready_r)
-        }).await.unwrap();
-         
-            bail!("First `run_with_command` call never started");
-        }
-
-        // ensure the rpc lock file exists
-        if !lock_file_path.try_exists()? {
-            start.abort();
-            bail!("First `run_with_command` call never made the rpc lockfile");
-        }
-
-        // run the second command, this should fail
-        if run_with_command(
-            &NodeConfig::default(),
-            data_dir.path(),
-            RunType::SingleCommandAbortable,
-            |_| async move { Ok(()) },
-        )
-        .await
-        .is_ok()
-        {
-            start.abort();
-            bail!("Second `run_with_command` call should return error");
-        }
-
-        // ensure the rpc lock file still exists
-        if !lock_file_path.try_exists()? {
-            start.abort();
-            bail!("Second `run_with_command` removed the rpc lockfile");
-        }
-
-        // inform the node it should close
-        close_s.send(()).unwrap();
-
-        // wait for the node to close
-        if tokio::time::timeout(Duration::from_millis(1000), start)
-            .await
-            .is_err()
-        {
-            bail!("First `run_with_command` never closed");
-        }
-
-        // ensure the lockfile no longer exists
-        if lock_file_path.try_exists()? {
-            bail!("First `run_with_command` closed without removing the rpc lockfile");
-        }
-        Ok(())
-    }
-}
diff --git a/iroh-net/src/relay/client_conn.rs b/iroh-net/src/relay/client_conn.rs
index 2446668438d..c3d61c5bf90 100644
--- a/iroh-net/src/relay/client_conn.rs
+++ b/iroh-net/src/relay/client_conn.rs
@@ -6,7 +6,6 @@ use anyhow::{Context, Result};
 use bytes::Bytes;
 use futures_lite::StreamExt;
 use futures_util::SinkExt;
-use tokio::io::{AsyncRead, AsyncWrite};
 use tokio::sync::mpsc;
 use tokio_util::codec::Framed;
 use tokio_util::sync::CancellationToken;
diff --git a/iroh-sync/Cargo.toml b/iroh-sync/Cargo.toml
index dd13d0048d6..9f0037fb44a 100644
--- a/iroh-sync/Cargo.toml
+++ b/iroh-sync/Cargo.toml
@@ -60,5 +60,5 @@ test-strategy = "0.3.1"
 
 [features]
 default = ["net", "metrics"]
-net = ["dep:iroh-net", "tokio/io-util", "dep:tokio-stream", "dep:tokio-util", "dep:quinn", "dep:futures-lite"]
+net = ["dep:iroh-net", "tokio/io-util", "dep:tokio-stream", "dep:tokio-util", "dep:quinn", "dep:futures-util"]
 metrics = ["dep:iroh-metrics"]
diff --git a/iroh/src/node/rpc.rs b/iroh/src/node/rpc.rs
index 4d346705e77..e89c3d628a0 100644
--- a/iroh/src/node/rpc.rs
+++ b/iroh/src/node/rpc.rs
@@ -235,7 +235,7 @@ impl<D: BaoStore> Handler<D> {
                     })
                     .await
                 }
-                DocSubscribe(msg) => {
+                DocSubscribe(_msg) => {
                     todo!()
                     /*chan.server_streaming(msg, handler, |handler, req| {
                         handler.inner.sync.doc_subscribe(req)

From 8de42839090fbb597784d45cbf6498f3ee5a9255 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 25 Mar 2024 17:41:34 +0100
Subject: [PATCH 12/25] more fixes

---
 iroh-bytes/src/util/progress.rs  |  2 +-
 iroh-net/src/magicsock.rs        |  2 +-
 iroh-net/src/net/netmon/linux.rs |  2 +-
 iroh/examples/rpc.rs             |  3 ++-
 iroh/src/node.rs                 |  2 +-
 iroh/tests/gc.rs                 | 12 +++++-------
 6 files changed, 11 insertions(+), 12 deletions(-)

diff --git a/iroh-bytes/src/util/progress.rs b/iroh-bytes/src/util/progress.rs
index 2c370af9fbf..182fbc1f514 100644
--- a/iroh-bytes/src/util/progress.rs
+++ b/iroh-bytes/src/util/progress.rs
@@ -57,7 +57,7 @@ use iroh_io::AsyncSliceWriter;
 /// operation that reports progress of type `B`. If you have a transformation for
 /// every `B` to an `A`, you can use the [ProgressSender::with_map] method to transform the message.
 ///
-/// This is similar to the [futures::SinkExt::with] method.
+/// This is similar to the [futures_util::SinkExt::with] method.
 ///
 /// # Filtering the message type
 ///
diff --git a/iroh-net/src/magicsock.rs b/iroh-net/src/magicsock.rs
index cae0c9ab8e5..3fc45c17803 100644
--- a/iroh-net/src/magicsock.rs
+++ b/iroh-net/src/magicsock.rs
@@ -1312,7 +1312,7 @@ impl MagicSock {
     ///
     /// To get the current endpoints, drop the stream after the first item was received:
     /// ```
-    /// use futures::StreamExt;
+    /// use futures_lite::StreamExt;
     /// use iroh_net::magicsock::MagicSock;
     ///
     /// # let rt = tokio::runtime::Builder::new_current_thread().enable_all().build().unwrap();
diff --git a/iroh-net/src/net/netmon/linux.rs b/iroh-net/src/net/netmon/linux.rs
index 0cdc1a81ce5..f1b98dec113 100644
--- a/iroh-net/src/net/netmon/linux.rs
+++ b/iroh-net/src/net/netmon/linux.rs
@@ -4,7 +4,7 @@ use std::{
 };
 
 use anyhow::Result;
-use futures::StreamExt;
+use futures_lite::StreamExt;
 use netlink_packet_core::NetlinkPayload;
 use netlink_packet_route::{address, constants::*, route, RtnlMessage};
 use netlink_sys::{AsyncSocket, SocketAddr};
diff --git a/iroh/examples/rpc.rs b/iroh/examples/rpc.rs
index 2aacfa758bc..1d1aa9550f4 100644
--- a/iroh/examples/rpc.rs
+++ b/iroh/examples/rpc.rs
@@ -40,7 +40,8 @@ where
     }
     // wait for the node to finish, this will block indefinitely
     // stop with SIGINT (ctrl+c)
-    node.await?;
+    node.shutdown().await?;
+
     Ok(())
 }
 
diff --git a/iroh/src/node.rs b/iroh/src/node.rs
index 1fc7e4d3950..520ada2a2a0 100644
--- a/iroh/src/node.rs
+++ b/iroh/src/node.rs
@@ -278,7 +278,7 @@ mod tests {
 
     use anyhow::{bail, Context};
     use bytes::Bytes;
-    use futures::StreamExt;
+    use futures_lite::StreamExt;
     use iroh_bytes::provider::AddProgress;
 
     use crate::rpc_protocol::{BlobAddPathRequest, BlobAddPathResponse, SetTagOption, WrapOption};
diff --git a/iroh/tests/gc.rs b/iroh/tests/gc.rs
index e6f79e0dd22..b94fa41917a 100644
--- a/iroh/tests/gc.rs
+++ b/iroh/tests/gc.rs
@@ -194,7 +194,7 @@ mod file {
         ChunkRanges,
     };
     use bytes::Bytes;
-    use futures::StreamExt;
+    use futures_lite::StreamExt;
     use iroh_io::AsyncSliceReaderExt;
     use testdir::testdir;
 
@@ -376,8 +376,8 @@ mod file {
         assert!(!path(&hr).exists());
         assert!(!outboard_path(&hr).exists());
 
-        node.shutdown();
-        node.await?;
+        node.shutdown().await?;
+
         Ok(())
     }
 
@@ -471,8 +471,7 @@ mod file {
         assert!(!path(&h1).exists());
         assert!(!outboard_path(&h1).exists());
 
-        node.shutdown();
-        node.await?;
+        node.shutdown().await?;
         Ok(())
     }
 
@@ -514,8 +513,7 @@ mod file {
             assert!(dir.join(format!("data/{}.data", h.to_hex())).exists());
         }
 
-        node.shutdown();
-        node.await?;
+        node.shutdown().await?;
         Ok(())
     }
 }

From 3f422f13a5837747cc7c11fcca51b35310031423 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 25 Mar 2024 17:52:29 +0100
Subject: [PATCH 13/25] more fixes

---
 iroh-bytes/src/util/progress.rs      | 2 +-
 iroh-net/src/net/interfaces/linux.rs | 9 +++++----
 2 files changed, 6 insertions(+), 5 deletions(-)

diff --git a/iroh-bytes/src/util/progress.rs b/iroh-bytes/src/util/progress.rs
index 182fbc1f514..40925ee965e 100644
--- a/iroh-bytes/src/util/progress.rs
+++ b/iroh-bytes/src/util/progress.rs
@@ -57,7 +57,7 @@ use iroh_io::AsyncSliceWriter;
 /// operation that reports progress of type `B`. If you have a transformation for
 /// every `B` to an `A`, you can use the [ProgressSender::with_map] method to transform the message.
 ///
-/// This is similar to the [futures_util::SinkExt::with] method.
+/// This is similar to the `futures::SinkExt::with` method.
 ///
 /// # Filtering the message type
 ///
diff --git a/iroh-net/src/net/interfaces/linux.rs b/iroh-net/src/net/interfaces/linux.rs
index 8b0d139913e..01187900d43 100644
--- a/iroh-net/src/net/interfaces/linux.rs
+++ b/iroh-net/src/net/interfaces/linux.rs
@@ -133,7 +133,8 @@ async fn default_route_netlink_family(
     family: rtnetlink::IpVersion,
 ) -> Result<Option<(String, u32)>> {
     let mut routes = handle.route().get(family).execute();
-    while let Some(route) = routes.try_next().await? {
+    while let Some(route) = routes.next().await {
+        let route = route?;
         if route.gateway().is_none() {
             // A default route has a gateway.
             continue;
@@ -158,9 +159,9 @@ async fn default_route_netlink_family(
 async fn iface_by_index(handle: &rtnetlink::Handle, index: u32) -> Result<String> {
     let mut links = handle.link().get().match_index(index).execute();
     let msg = links
-        .try_next()
-        .await?
-        .ok_or_else(|| anyhow!("No netlink response"))?;
+        .next()
+        .ok_or_else(|| anyhow!("No netlink response"))?
+        .await?;
     for nla in msg.nlas {
         if let netlink_packet_route::link::nlas::Nla::IfName(name) = nla {
             return Ok(name);

From 00492f6c18e49b3769a6b2d4257cb19a2c3cdb9d Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 25 Mar 2024 18:04:44 +0100
Subject: [PATCH 14/25] fixup

---
 iroh-net/src/net/interfaces/linux.rs | 1 +
 1 file changed, 1 insertion(+)

diff --git a/iroh-net/src/net/interfaces/linux.rs b/iroh-net/src/net/interfaces/linux.rs
index 01187900d43..8bde4be1a3e 100644
--- a/iroh-net/src/net/interfaces/linux.rs
+++ b/iroh-net/src/net/interfaces/linux.rs
@@ -160,6 +160,7 @@ async fn iface_by_index(handle: &rtnetlink::Handle, index: u32) -> Result<String
     let mut links = handle.link().get().match_index(index).execute();
     let msg = links
         .next()
+        .await
         .ok_or_else(|| anyhow!("No netlink response"))?
         .await?;
     for nla in msg.nlas {

From 9c838e005dbf0bf4b699375932461109036789e7 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 5 Apr 2024 13:36:29 +0200
Subject: [PATCH 15/25] first pass of merge updates

---
 iroh-net/src/relay/clients.rs  |  6 +++++-
 iroh/src/client/authors.rs     | 32 +++++++-------------------------
 iroh/src/client/docs.rs        |  2 +-
 iroh/src/node.rs               |  3 +--
 iroh/src/node/builder.rs       |  7 ++++---
 iroh/src/sync_engine.rs        | 27 +++++++++------------------
 iroh/src/sync_engine/gossip.rs |  2 +-
 7 files changed, 28 insertions(+), 51 deletions(-)

diff --git a/iroh-net/src/relay/clients.rs b/iroh-net/src/relay/clients.rs
index 224deceba64..a8660ed659d 100644
--- a/iroh-net/src/relay/clients.rs
+++ b/iroh-net/src/relay/clients.rs
@@ -153,7 +153,11 @@ impl Clients {
         for (_, client) in self.inner.drain() {
             handles.spawn(async move { client.shutdown_await().await }.instrument(Span::current()));
         }
-        while let Some(_) = handles.join_next().await {}
+        while let Some(t) = handles.join_next().await {
+            if let Err(err) = t {
+                tracing::trace!("shutdown error: {:?}", err);
+            }
+        }
     }
 
     /// Record that `src` sent or forwarded a packet to `dst`
diff --git a/iroh/src/client/authors.rs b/iroh/src/client/authors.rs
index 5de388bf27b..8f74d0d1076 100644
--- a/iroh/src/client/authors.rs
+++ b/iroh/src/client/authors.rs
@@ -69,15 +69,8 @@ mod tests {
 
         let author_id = node.authors.create().await?;
 
-        assert_eq!(
-            node.authors
-                .list()
-                .await?
-                .try_collect::<Vec<_>>()
-                .await?
-                .len(),
-            1
-        );
+        let authors: Vec<_> = node.authors.list().await?.try_collect().await?;
+        assert_eq!(authors.len(), 1);
 
         let author = node
             .authors
@@ -85,24 +78,13 @@ mod tests {
             .await?
             .expect("should have author");
         node.authors.delete(author_id).await?;
-        assert!(node
-            .authors
-            .list()
-            .await?
-            .try_collect::<Vec<_>>()
-            .await?
-            .is_empty());
+        let authors: Vec<_> = node.authors.list().await?.try_collect().await?;
+        assert!(authors.is_empty());
 
         node.authors.import(author).await?;
-        assert_eq!(
-            node.authors
-                .list()
-                .await?
-                .try_collect::<Vec<_>>()
-                .await?
-                .len(),
-            1
-        );
+
+        let authors: Vec<_> = node.authors.list().await?.try_collect().await?;
+        assert_eq!(authors.len(), 1);
 
         Ok(())
     }
diff --git a/iroh/src/client/docs.rs b/iroh/src/client/docs.rs
index 29ffbdfd8c9..c4c6a7bdfa6 100644
--- a/iroh/src/client/docs.rs
+++ b/iroh/src/client/docs.rs
@@ -342,7 +342,7 @@ where
             .await?;
         Ok(flatten(stream).map(|res| match res {
             Ok(res) => Ok(res.event.into()),
-            Err(err) => Err(err.into()),
+            Err(err) => Err(err),
         }))
     }
 
diff --git a/iroh/src/node.rs b/iroh/src/node.rs
index 30060db74f0..2314296194c 100644
--- a/iroh/src/node.rs
+++ b/iroh/src/node.rs
@@ -392,8 +392,7 @@ mod tests {
             let iroh = Node::persistent(iroh_root.path()).await?.spawn().await?;
             let doc = iroh.docs.create().await?;
             drop(doc);
-            iroh.shutdown();
-            iroh.await?;
+            iroh.shutdown().await?;
         }
 
         let iroh = Node::persistent(iroh_root.path()).await?.spawn().await?;
diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs
index f1f854014f0..72c87e8601f 100644
--- a/iroh/src/node/builder.rs
+++ b/iroh/src/node/builder.rs
@@ -435,9 +435,10 @@ where
                 _ = cancel_token.cancelled() => {
                     // clean shutdown of the blobs db to close the write transaction
                     handler.inner.db.shutdown().await;
-                    if let Err(err) = handler.inner.sync.shutdown().await {
-                        warn!("sync shutdown error: {:?}", err);
-                    }
+                    todo!();
+                    // if let Err(err) = handler.inner.sync.shutdown().await {
+                    // warn!("sync shutdown error: {:?}", err);
+                    // }
                     break
                 },
                 // handle rpc requests. This will do nothing if rpc is not configured, since
diff --git a/iroh/src/sync_engine.rs b/iroh/src/sync_engine.rs
index 32cb1c6b94b..d24591fada5 100644
--- a/iroh/src/sync_engine.rs
+++ b/iroh/src/sync_engine.rs
@@ -13,7 +13,7 @@ use iroh_net::{key::PublicKey, MagicEndpoint, NodeAddr};
 use iroh_sync::{actor::SyncHandle, ContentStatus, ContentStatusCallback, Entry, NamespaceId};
 use serde::{Deserialize, Serialize};
 use tokio::sync::{mpsc, oneshot};
-use tokio::task::JoinHandle;
+use tokio::task::JoinSet;
 use tracing::{error, error_span, Instrument};
 
 mod gossip;
@@ -44,7 +44,7 @@ pub struct SyncEngine {
     pub(crate) sync: SyncHandle,
     to_live_actor: mpsc::Sender<ToLiveActor>,
     #[debug("Arc<JoinHandle<()>>")]
-    tasks: Arc<JoinHandle<()>>,
+    tasks: Arc<JoinSet<()>>,
     #[debug("ContentStatusCallback")]
     content_status_cb: ContentStatusCallback,
 }
@@ -92,7 +92,8 @@ impl SyncEngine {
             downloader,
             live_actor_tx.clone(),
         );
-        let live_actor_task = tokio::task::spawn(
+        let mut tasks = JoinSet::new();
+        tasks.spawn(
             async move {
                 if let Err(err) = actor.run().await {
                     error!("sync actor failed: {err:?}");
@@ -100,7 +101,7 @@ impl SyncEngine {
             }
             .instrument(error_span!("sync", %me)),
         );
-        let gossip_actor_task = tokio::task::spawn(
+        tasks.spawn(
             async move {
                 if let Err(err) = gossip_actor.run().await {
                     error!("gossip recv actor failed: {err:?}");
@@ -108,18 +109,6 @@ impl SyncEngine {
             }
             .instrument(error_span!("sync", %me)),
         );
-        let tasks = tokio::task::spawn(async move {
-            if let Err(err) = live_actor_task.await {
-                error!("Error while joining actor task: {err:?}");
-            }
-            gossip_actor_task.abort();
-            if let Err(err) = gossip_actor_task.await {
-                if !err.is_cancelled() {
-                    error!("Error while joining gossip recv task task: {err:?}");
-                }
-            }
-        });
-
         Self {
             endpoint,
             sync,
@@ -225,8 +214,10 @@ impl SyncEngine {
     /// Shutdown the sync engine.
     pub async fn shutdown(self) -> Result<()> {
         self.to_live_actor.send(ToLiveActor::Shutdown).await?;
-        if let Ok(tasks) = Arc::try_unwrap(self.tasks) {
-            tasks.await?;
+        if let Ok(mut tasks) = Arc::try_unwrap(self.tasks) {
+            while let Some(t) = tasks.join_next().await {
+                t?;
+            }
         }
         Ok(())
     }
diff --git a/iroh/src/sync_engine/gossip.rs b/iroh/src/sync_engine/gossip.rs
index 0a6eb76d771..a70f462b82e 100644
--- a/iroh/src/sync_engine/gossip.rs
+++ b/iroh/src/sync_engine/gossip.rs
@@ -118,7 +118,7 @@ impl GossipActor {
                     let res = gossip.join(namespace.into(), peers).await;
                     let res = match res {
                         Ok(fut) => fut.await,
-                        Err(err) => Err(err.into()),
+                        Err(err) => Err(err),
                     };
                     (namespace, res)
                 };

From 82559e2eafb46e531058e77f8034481082fdf4e7 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 5 Apr 2024 13:43:33 +0200
Subject: [PATCH 16/25] fix shutdown

---
 iroh/src/node/builder.rs | 8 ++++----
 iroh/src/sync_engine.rs  | 6 +++++-
 2 files changed, 9 insertions(+), 5 deletions(-)

diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs
index 72c87e8601f..f1d9b16906d 100644
--- a/iroh/src/node/builder.rs
+++ b/iroh/src/node/builder.rs
@@ -435,10 +435,10 @@ where
                 _ = cancel_token.cancelled() => {
                     // clean shutdown of the blobs db to close the write transaction
                     handler.inner.db.shutdown().await;
-                    todo!();
-                    // if let Err(err) = handler.inner.sync.shutdown().await {
-                    // warn!("sync shutdown error: {:?}", err);
-                    // }
+
+                    if let Err(err) = handler.inner.sync.start_shutdown().await {
+                        warn!("sync shutdown error: {:?}", err);
+                    }
                     break
                 },
                 // handle rpc requests. This will do nothing if rpc is not configured, since
diff --git a/iroh/src/sync_engine.rs b/iroh/src/sync_engine.rs
index d24591fada5..414ef3b3d50 100644
--- a/iroh/src/sync_engine.rs
+++ b/iroh/src/sync_engine.rs
@@ -43,7 +43,6 @@ pub struct SyncEngine {
     pub(crate) endpoint: MagicEndpoint,
     pub(crate) sync: SyncHandle,
     to_live_actor: mpsc::Sender<ToLiveActor>,
-    #[debug("Arc<JoinHandle<()>>")]
     tasks: Arc<JoinSet<()>>,
     #[debug("ContentStatusCallback")]
     content_status_cb: ContentStatusCallback,
@@ -211,6 +210,11 @@ impl SyncEngine {
         Ok(())
     }
 
+    pub(crate) async fn start_shutdown(&self) -> Result<()> {
+        self.to_live_actor.send(ToLiveActor::Shutdown).await?;
+        Ok(())
+    }
+
     /// Shutdown the sync engine.
     pub async fn shutdown(self) -> Result<()> {
         self.to_live_actor.send(ToLiveActor::Shutdown).await?;

From dbe2299bc57bf8d4ad28e37a27ad096503a320b0 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 22 Apr 2024 10:07:40 +0200
Subject: [PATCH 17/25] merge fixup

---
 Cargo.lock                         |  1 +
 iroh-bytes/Cargo.toml              |  3 ++-
 iroh-bytes/src/get/db.rs           |  2 +-
 iroh-bytes/src/provider.rs         |  3 +--
 iroh-bytes/src/store/bao_file.rs   |  2 +-
 iroh-net/src/discovery.rs          |  2 +-
 iroh-net/src/discovery/dns.rs      | 17 +++++++++--------
 iroh-net/src/magicsock/node_map.rs |  2 +-
 iroh-net/src/relay/client.rs       |  2 +-
 iroh-net/src/relay/codec.rs        |  1 -
 iroh-net/src/relay/server.rs       |  1 -
 iroh-net/src/test_utils.rs         |  8 ++++----
 12 files changed, 22 insertions(+), 22 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index 420d8b7e1a4..13dcc5acc42 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -2717,6 +2717,7 @@ dependencies = [
  "iroh-metrics",
  "iroh-net",
  "iroh-test",
+ "lru",
  "num_enum",
  "postcard",
  "proptest",
diff --git a/iroh-bytes/Cargo.toml b/iroh-bytes/Cargo.toml
index 741477132df..fc13a329b5e 100644
--- a/iroh-bytes/Cargo.toml
+++ b/iroh-bytes/Cargo.toml
@@ -64,9 +64,10 @@ tempfile = "3.10.0"
 
 [features]
 default = ["fs-store"]
-fs-store = ["dep:reflink-copy", "dep:redb", "dep:redb_v1", "dep:tempfile"]
+fs-store = ["dep:reflink-copy", "redb", "dep:redb_v1", "dep:tempfile"]
 downloader = ["dep:iroh-net", "dep:parking_lot", "tokio-util/time"]
 metrics = ["dep:iroh-metrics"]
+redb = ["dep:redb"]
 
 [[example]]
 name = "provide-bytes"
diff --git a/iroh-bytes/src/get/db.rs b/iroh-bytes/src/get/db.rs
index 78650973037..dadef7d8576 100644
--- a/iroh-bytes/src/get/db.rs
+++ b/iroh-bytes/src/get/db.rs
@@ -27,7 +27,7 @@ use crate::{
     BlobFormat, HashAndFormat,
 };
 use anyhow::anyhow;
-use bao_tree::ChunkRanges;
+use bao_tree::{ChunkNum, ChunkRanges};
 use iroh_io::AsyncSliceReader;
 use tracing::trace;
 
diff --git a/iroh-bytes/src/provider.rs b/iroh-bytes/src/provider.rs
index a11cef93161..52371cba3a6 100644
--- a/iroh-bytes/src/provider.rs
+++ b/iroh-bytes/src/provider.rs
@@ -4,9 +4,8 @@ use std::time::Duration;
 
 use anyhow::{Context, Result};
 use bao_tree::io::fsm::{encode_ranges_validated, Outboard};
-use futures_lite::future::Boxed as BoxFuture;
 use bao_tree::io::EncodeError;
-use futures::future::BoxFuture;
+use futures_lite::future::Boxed as BoxFuture;
 use iroh_base::rpc::RpcError;
 use iroh_io::stats::{
     SliceReaderStats, StreamWriterStats, TrackingSliceReader, TrackingStreamWriter,
diff --git a/iroh-bytes/src/store/bao_file.rs b/iroh-bytes/src/store/bao_file.rs
index 0db9c560bad..8d61bb90dbc 100644
--- a/iroh-bytes/src/store/bao_file.rs
+++ b/iroh-bytes/src/store/bao_file.rs
@@ -872,9 +872,9 @@ pub mod test_support {
 mod tests {
     use std::io::Write;
 
+    use bao_tree::{blake3, ChunkNum, ChunkRanges};
     use bao_tree::{ChunkNum, ChunkRanges};
     use futures_lite::StreamExt;
-    use bao_tree::{blake3, ChunkNum, ChunkRanges};
     use iroh_io::TokioStreamReader;
     use tests::test_support::{
         decode_response_into_batch, local, make_wire_data, random_test_data, trickle, validate,
diff --git a/iroh-net/src/discovery.rs b/iroh-net/src/discovery.rs
index 9efa740f956..27d6dbfd9bd 100644
--- a/iroh-net/src/discovery.rs
+++ b/iroh-net/src/discovery.rs
@@ -814,7 +814,7 @@ mod test_dns_pkarr {
             ) -> impl Future<Output = Result<()>> + Send {
                 const TTL: u32 = 30;
                 let res = self.resolve_dns(query, reply, TTL);
-                futures::future::ready(res)
+                std::future::ready(res)
             }
         }
     }
diff --git a/iroh-net/src/discovery/dns.rs b/iroh-net/src/discovery/dns.rs
index befb00cc48f..e791edc7d30 100644
--- a/iroh-net/src/discovery/dns.rs
+++ b/iroh-net/src/discovery/dns.rs
@@ -1,13 +1,12 @@
 //! DNS node discovery for iroh-net
 
+use anyhow::Result;
+use futures_lite::stream::Boxed as BoxStream;
+
 use crate::{
     discovery::{Discovery, DiscoveryItem},
-    MagicEndpoint, NodeId,
+    dns, MagicEndpoint, NodeId,
 };
-use anyhow::Result;
-use futures::{future::FutureExt, stream::BoxStream, StreamExt};
-
-use crate::dns;
 
 /// The n0 testing DNS node origin
 pub const N0_DNS_NODE_ORIGIN: &str = "dns.iroh.link";
@@ -54,17 +53,19 @@ impl Discovery for DnsDiscovery {
         &self,
         ep: MagicEndpoint,
         node_id: NodeId,
-    ) -> Option<BoxStream<'_, Result<DiscoveryItem>>> {
+    ) -> Option<BoxStream<Result<DiscoveryItem>>> {
         let resolver = ep.dns_resolver().clone();
+        let origin_domain = self.origin_domain.clone();
         let fut = async move {
             let node_addr =
-                dns::node_info::lookup_by_id(&resolver, &node_id, &self.origin_domain).await?;
+                dns::node_info::lookup_by_id(&resolver, &node_id, &origin_domain).await?;
             Ok(DiscoveryItem {
                 provenance: "dns",
                 last_updated: None,
                 addr_info: node_addr.info,
             })
         };
-        Some(fut.into_stream().boxed())
+        let stream = futures_lite::stream::once_future(fut);
+        Some(Box::pin(stream))
     }
 }
diff --git a/iroh-net/src/magicsock/node_map.rs b/iroh-net/src/magicsock/node_map.rs
index 99fdf52e2c9..c990a04cf04 100644
--- a/iroh-net/src/magicsock/node_map.rs
+++ b/iroh-net/src/magicsock/node_map.rs
@@ -9,7 +9,7 @@ use std::{
 };
 
 use anyhow::{ensure, Context as _};
-use futures::Stream;
+use futures_lite::stream::Stream;
 use iroh_metrics::inc;
 use parking_lot::Mutex;
 use stun_rs::TransactionId;
diff --git a/iroh-net/src/relay/client.rs b/iroh-net/src/relay/client.rs
index 78a7860dece..cbee0aea0a3 100644
--- a/iroh-net/src/relay/client.rs
+++ b/iroh-net/src/relay/client.rs
@@ -5,7 +5,7 @@ use std::time::Duration;
 
 use anyhow::{anyhow, bail, ensure, Result};
 use bytes::Bytes;
-use futures_lite::{Stream, StreamExt};
+use futures_lite::StreamExt;
 use futures_sink::Sink;
 use futures_util::sink::SinkExt;
 use tokio::io::{AsyncRead, AsyncWrite};
diff --git a/iroh-net/src/relay/codec.rs b/iroh-net/src/relay/codec.rs
index ee2b5528638..8b4e02f4934 100644
--- a/iroh-net/src/relay/codec.rs
+++ b/iroh-net/src/relay/codec.rs
@@ -5,7 +5,6 @@ use bytes::{Buf, BufMut, Bytes, BytesMut};
 use futures_lite::{Stream, StreamExt};
 use futures_sink::Sink;
 use futures_util::SinkExt;
-use futures::{Sink, SinkExt, Stream, StreamExt};
 use iroh_base::key::{Signature, PUBLIC_KEY_LENGTH};
 use tokio_util::codec::{Decoder, Encoder};
 
diff --git a/iroh-net/src/relay/server.rs b/iroh-net/src/relay/server.rs
index acff15a475d..dd4c8d62656 100644
--- a/iroh-net/src/relay/server.rs
+++ b/iroh-net/src/relay/server.rs
@@ -5,7 +5,6 @@ use std::sync::Arc;
 use std::task::{Context, Poll};
 use std::time::Duration;
 
-use futures_util::SinkExt;
 use anyhow::{bail, Context as _, Result};
 use hyper::HeaderMap;
 use iroh_metrics::core::UsageStatsReport;
diff --git a/iroh-net/src/test_utils.rs b/iroh-net/src/test_utils.rs
index 652d81a09d6..77d6025ef42 100644
--- a/iroh-net/src/test_utils.rs
+++ b/iroh-net/src/test_utils.rs
@@ -65,10 +65,11 @@ pub async fn run_relay_server() -> Result<(RelayMap, RelayUrl, CleanupDropGuard)
 
 #[cfg(test)]
 pub(crate) mod dns_server {
+    use std::future::Future;
     use std::net::{Ipv4Addr, SocketAddr};
 
     use anyhow::{ensure, Result};
-    use futures::{future::BoxFuture, Future};
+    use futures_lite::future::Boxed as BoxFuture;
     use hickory_proto::{
         op::{header::MessageType, Message},
         serialize::binary::BinDecodable,
@@ -87,9 +88,8 @@ pub(crate) mod dns_server {
         ) -> impl Future<Output = Result<()>> + Send;
     }
 
-    pub type QueryHandlerFunction = Box<
-        dyn Fn(&Message, &mut Message) -> BoxFuture<'static, Result<()>> + Send + Sync + 'static,
-    >;
+    pub type QueryHandlerFunction =
+        Box<dyn Fn(&Message, &mut Message) -> BoxFuture<Result<()>> + Send + Sync + 'static>;
     impl QueryHandler for QueryHandlerFunction {
         fn resolve(
             &self,

From a3e894b9cf5e340a693b72b7a812a4f617893464 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 22 Apr 2024 10:53:07 +0200
Subject: [PATCH 18/25] fixup: iroh-dns-server

---
 Cargo.lock                      | 2 +-
 iroh-dns-server/Cargo.toml      | 2 +-
 iroh-dns-server/src/http/tls.rs | 4 ++--
 iroh-dns-server/src/main.rs     | 8 +++++---
 4 files changed, 9 insertions(+), 7 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index 13dcc5acc42..796b8c5d1c6 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -2503,7 +2503,7 @@ dependencies = [
  "clap",
  "derive_more",
  "dirs-next",
- "futures",
+ "futures-lite",
  "governor",
  "hickory-proto",
  "hickory-resolver",
diff --git a/iroh-dns-server/Cargo.toml b/iroh-dns-server/Cargo.toml
index 589bb6370f5..095f480f273 100644
--- a/iroh-dns-server/Cargo.toml
+++ b/iroh-dns-server/Cargo.toml
@@ -19,7 +19,7 @@ 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"
+futures-lite = "2.3.0"
 governor = "0.6.3"
 hickory-proto = "0.24.0"
 hickory-server = { version = "0.24.0", features = ["dns-over-rustls"] }
diff --git a/iroh-dns-server/src/http/tls.rs b/iroh-dns-server/src/http/tls.rs
index 4e079aefd49..1133d498d56 100644
--- a/iroh-dns-server/src/http/tls.rs
+++ b/iroh-dns-server/src/http/tls.rs
@@ -10,7 +10,7 @@ use axum_server::{
     accept::Accept,
     tls_rustls::{RustlsAcceptor, RustlsConfig},
 };
-use futures::{future::BoxFuture, FutureExt};
+use futures_lite::{future::Boxed as BoxFuture, FutureExt};
 use serde::{Deserialize, Serialize};
 use tokio::io::{AsyncRead, AsyncWrite};
 use tokio_rustls_acme::{axum::AxumAcceptor, caches::DirCache, AcmeConfig};
@@ -62,7 +62,7 @@ impl<I: AsyncRead + AsyncWrite + Unpin + Send + 'static, S: Send + 'static> Acce
 {
     type Stream = tokio_rustls::server::TlsStream<I>;
     type Service = S;
-    type Future = BoxFuture<'static, io::Result<(Self::Stream, Self::Service)>>;
+    type Future = BoxFuture<io::Result<(Self::Stream, Self::Service)>>;
 
     fn accept(&self, stream: I, service: S) -> Self::Future {
         match self {
diff --git a/iroh-dns-server/src/main.rs b/iroh-dns-server/src/main.rs
index 6a7f88d6738..511a4c58db7 100644
--- a/iroh-dns-server/src/main.rs
+++ b/iroh-dns-server/src/main.rs
@@ -1,14 +1,16 @@
 #![allow(unused_imports)]
 
+use std::future::Future;
+use std::net::{Ipv4Addr, SocketAddr};
+use std::path::PathBuf;
+
 use anyhow::Result;
 use axum::{routing::get, Router};
 use clap::Parser;
-use futures::{Future, FutureExt};
+use futures_lite::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};

From dda6dde2e90f3503a8dd58a6dff18e8ea4b53f6f Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Wed, 24 Apr 2024 19:22:07 +0200
Subject: [PATCH 19/25] try updating to new quic rpc

---
 Cargo.lock               | 234 ++++++++++++++++++++++-----------------
 iroh-cli/Cargo.toml      |   2 +-
 iroh/Cargo.toml          |   2 +-
 iroh/src/client/docs.rs  |   4 +-
 iroh/src/node/rpc.rs     |   5 +-
 iroh/src/rpc_protocol.rs |   9 +-
 iroh/tests/sync.rs       |   7 +-
 7 files changed, 146 insertions(+), 117 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index cef2fe82401..b445b64c5af 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -200,7 +200,7 @@ checksum = "7378575ff571966e99a744addeff0bff98b8ada0dedf1956d59e634db95eaac1"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
  "synstructure 0.13.1",
 ]
 
@@ -223,7 +223,7 @@ checksum = "7b18050c2cd6fe86c3a76584ef5e0baf286d038cda203eb6223df2cc413565f7"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -234,7 +234,7 @@ checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -277,7 +277,7 @@ dependencies = [
  "http 1.1.0",
  "http-body 1.0.0",
  "http-body-util",
- "hyper 1.2.0",
+ "hyper 1.3.1",
  "hyper-util",
  "itoa",
  "matchit",
@@ -328,7 +328,7 @@ dependencies = [
  "heck 0.4.1",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -343,7 +343,7 @@ dependencies = [
  "http 1.1.0",
  "http-body 1.0.0",
  "http-body-util",
- "hyper 1.2.0",
+ "hyper 1.3.1",
  "hyper-util",
  "pin-project-lite",
  "rustls",
@@ -547,9 +547,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5"
 
 [[package]]
 name = "cc"
-version = "1.0.94"
+version = "1.0.95"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "17f6e324229dc011159fcc089755d1e2e216a90d43a7dea6853ca740b84f35e7"
+checksum = "d32a725bc159af97c3e629873bb9f88fb8cf8a4867175f76dc987815ea07c83b"
 
 [[package]]
 name = "cfg-if"
@@ -652,7 +652,7 @@ dependencies = [
  "heck 0.5.0",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -984,7 +984,7 @@ checksum = "f46882e17999c6cc590af592290432be3bce0428cb0d5f8b6715e4dc7b383eb3"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1008,7 +1008,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "strsim 0.10.0",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1019,7 +1019,7 @@ checksum = "a668eda54683121533a393014d8692171709ff57a7d61f187b6e782719f8933f"
 dependencies = [
  "darling_core",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1105,7 +1105,7 @@ checksum = "5fe87ce4529967e0ba1dcf8450bab64d97dfd5010a6256187ffe2e43e6f0e049"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1135,7 +1135,7 @@ checksum = "2bba3e9872d7c58ce7ef0fcf1844fcc3e23ef2a58377b50df35dd98e42a5726e"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
  "unicode-xid",
 ]
 
@@ -1218,7 +1218,7 @@ checksum = "487585f4d0c6655fe74905e2504d8ad6908e4db67f744eb140876906c2f3175d"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1363,7 +1363,7 @@ dependencies = [
  "heck 0.4.1",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1376,7 +1376,7 @@ dependencies = [
  "num-traits",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1396,7 +1396,7 @@ checksum = "5c785274071b1b420972453b306eeca06acf4633829db4223b58a2a8c5953bc4"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1486,9 +1486,9 @@ dependencies = [
 
 [[package]]
 name = "fiat-crypto"
-version = "0.2.7"
+version = "0.2.8"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c007b1ae3abe1cb6f85a16305acd418b7ca6343b953633fee2b76d8f108b830f"
+checksum = "38793c55593b33412e3ae40c2c9781ffaa6f438f6f8c10f24e71846fbd7ae01e"
 
 [[package]]
 name = "flume"
@@ -1607,7 +1607,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -1876,9 +1876,9 @@ checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70"
 
 [[package]]
 name = "hickory-proto"
-version = "0.24.0"
+version = "0.24.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "091a6fbccf4860009355e3efc52ff4acf37a63489aad7435372d44ceeb6fbbcf"
+checksum = "07698b8420e2f0d6447a436ba999ec85d8fbf2a398bbd737b82cac4a2e96e512"
 dependencies = [
  "async-trait",
  "cfg-if",
@@ -1905,9 +1905,9 @@ dependencies = [
 
 [[package]]
 name = "hickory-resolver"
-version = "0.24.0"
+version = "0.24.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "35b8f021164e6a984c9030023544c57789c51760065cd510572fedcfb04164e8"
+checksum = "28757f23aa75c98f254cf0405e6d8c25b831b32921b050a66692427679b1f243"
 dependencies = [
  "cfg-if",
  "futures-util",
@@ -1929,9 +1929,9 @@ dependencies = [
 
 [[package]]
 name = "hickory-server"
-version = "0.24.0"
+version = "0.24.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "4fbbb45bc4dcb456445732c705e3cfdc7393b8bcae5c36ecec36b9d76bd67cb5"
+checksum = "9be0e43c556b9b3fdb6c7c71a9a32153a2275d02419e3de809e520bfcfe40c37"
 dependencies = [
  "async-trait",
  "bytes",
@@ -2114,9 +2114,9 @@ dependencies = [
 
 [[package]]
 name = "hyper"
-version = "1.2.0"
+version = "1.3.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "186548d73ac615b32a73aafe38fb4f56c0d340e110e5a200bcadbaf2e199263a"
+checksum = "fe575dd17d0862a9a33781c8c4696a55c320909004a67a00fb286ba8b1bc496d"
 dependencies = [
  "bytes",
  "futures-channel",
@@ -2157,7 +2157,7 @@ dependencies = [
  "futures-util",
  "http 1.1.0",
  "http-body 1.0.0",
- "hyper 1.2.0",
+ "hyper 1.3.1",
  "pin-project-lite",
  "socket2",
  "tokio",
@@ -2379,7 +2379,7 @@ dependencies = [
  "proptest",
  "rand",
  "rand_core",
- "redb 2.0.0",
+ "redb 2.1.0",
  "serde",
  "serde-error",
  "serde_json",
@@ -2393,9 +2393,9 @@ dependencies = [
 
 [[package]]
 name = "iroh-blake3"
-version = "1.4.4"
+version = "1.4.5"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "6eb52cd11b3de4407f29579ebcd10fd746b0bd8ab758a2afac69baf88e96bede"
+checksum = "efbba31f40a650f58fa28dd585a8ca76d8ae3ba63aacab4c8269004a0c803930"
 dependencies = [
  "arrayref",
  "arrayvec",
@@ -2436,7 +2436,7 @@ dependencies = [
  "range-collections",
  "rcgen 0.12.1",
  "redb 1.5.1",
- "redb 2.0.0",
+ "redb 2.1.0",
  "reflink-copy",
  "rustls",
  "self_cell",
@@ -2528,7 +2528,7 @@ dependencies = [
  "parking_lot",
  "pkarr",
  "rcgen 0.12.1",
- "redb 2.0.0",
+ "redb 2.1.0",
  "regex",
  "rustls",
  "rustls-pemfile 1.0.4",
@@ -2599,7 +2599,7 @@ dependencies = [
  "anyhow",
  "erased_set",
  "http-body-util",
- "hyper 1.2.0",
+ "hyper 1.3.1",
  "hyper-util",
  "once_cell",
  "prometheus-client",
@@ -2639,7 +2639,7 @@ dependencies = [
  "hostname",
  "http 1.1.0",
  "http-body-util",
- "hyper 1.2.0",
+ "hyper 1.3.1",
  "hyper-util",
  "igd-next",
  "iroh-base",
@@ -2738,7 +2738,7 @@ dependencies = [
  "rand_chacha",
  "rand_core",
  "redb 1.5.1",
- "redb 2.0.0",
+ "redb 2.1.0",
  "self_cell",
  "serde",
  "strum 0.25.0",
@@ -3227,7 +3227,7 @@ dependencies = [
  "proc-macro-crate",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -3447,7 +3447,7 @@ dependencies = [
  "pest_meta",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -3478,7 +3478,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -3583,7 +3583,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "regex",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -3767,9 +3767,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068"
 
 [[package]]
 name = "proc-macro2"
-version = "1.0.80"
+version = "1.0.81"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a56dea16b0a29e94408b9aa5e2940a4eedbd128a1ba20e8f7ae60fd3d465af0e"
+checksum = "3d1597b0c024618f09a9c3b8655b7e430397a36d23fdafec26d6965e9eec3eba"
 dependencies = [
  "unicode-ident",
 ]
@@ -3794,7 +3794,7 @@ checksum = "440f724eba9f6996b75d63681b0a92b06947f1457076d503a4d2e2c8f56442b8"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -3834,14 +3834,16 @@ dependencies = [
 
 [[package]]
 name = "quic-rpc"
-version = "0.7.0"
+version = "0.8.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "30d960d0b328db80274cbcbaf4bf53728eff1f43df4fc36c34eb79a97753c7d9"
+checksum = "b7af964f8ee0a3ff7f03b8bc3ffce8cd604d9c2e5805b45d1446f774c71fb07e"
 dependencies = [
  "bincode",
  "educe",
  "flume",
- "futures",
+ "futures-lite",
+ "futures-sink",
+ "futures-util",
  "pin-project",
  "quinn",
  "serde",
@@ -4049,9 +4051,9 @@ dependencies = [
 
 [[package]]
 name = "redb"
-version = "2.0.0"
+version = "2.1.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "a1100a056c5dcdd4e5513d5333385223b26ef1bf92f31eb38f407e8c20549256"
+checksum = "ed7508e692a49b6b2290b56540384ccae9b1fb4d77065640b165835b56ffe3bb"
 dependencies = [
  "libc",
 ]
@@ -4093,18 +4095,18 @@ checksum = "5fddb4f8d99b0a2ebafc65a87a69a7b9875e4b1ae1f00db265d300ef7f28bccc"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
 name = "reflink-copy"
-version = "0.1.15"
+version = "0.1.17"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "52b1349400e2ffd64a9fb5ed9008e33c0b8ef86bd5bae8f73080839c7082f1d5"
+checksum = "7c3138c30c59ed9b8572f82bed97ea591ecd7e45012566046cc39e72679cff22"
 dependencies = [
  "cfg-if",
  "rustix",
- "windows 0.54.0",
+ "windows 0.56.0",
 ]
 
 [[package]]
@@ -4319,9 +4321,9 @@ dependencies = [
 
 [[package]]
 name = "rustix"
-version = "0.38.32"
+version = "0.38.34"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "65e04861e65f21776e67888bfbea442b3642beaa0138fdb1dd7a84a52dffdb89"
+checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f"
 dependencies = [
  "bitflags 2.5.0",
  "errno",
@@ -4375,9 +4377,9 @@ dependencies = [
 
 [[package]]
 name = "rustls-pki-types"
-version = "1.4.1"
+version = "1.5.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "ecd36cc4259e3e4514335c4a138c6b43171a8d61d8f5c9348f9fc7529416f247"
+checksum = "beb461507cee2c2ff151784c52762cf4d9ff6a61f3e80968600ed24fa837fa54"
 
 [[package]]
 name = "rustls-webpki"
@@ -4533,9 +4535,9 @@ dependencies = [
 
 [[package]]
 name = "serde"
-version = "1.0.197"
+version = "1.0.198"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3fb1c873e1b9b056a4dc4c0c198b24c3ffa059243875552b2bd0933b1aee4ce2"
+checksum = "9846a40c979031340571da2545a4e5b7c4163bdae79b301d5f86d03979451fcc"
 dependencies = [
  "serde_derive",
 ]
@@ -4551,20 +4553,20 @@ dependencies = [
 
 [[package]]
 name = "serde_derive"
-version = "1.0.197"
+version = "1.0.198"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "7eb0b34b42edc17f6b7cac84a52a1c5f0e1bb2227e997ca9011ea3dd34e8610b"
+checksum = "e88edab869b01783ba905e7d0153f9fc1a6505a96e4ad3018011eedb838566d9"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
 name = "serde_json"
-version = "1.0.115"
+version = "1.0.116"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "12dc5c46daa8e9fdf4f5e71b6cf9a53f2487da0e86e55808e2d35539666497dd"
+checksum = "3e17db7126d17feb94eb3fad46bf1a96b034e8aacbc2e775fe81505f8b0b2813"
 dependencies = [
  "itoa",
  "ryu",
@@ -4638,7 +4640,7 @@ dependencies = [
  "darling",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -4709,9 +4711,9 @@ dependencies = [
 
 [[package]]
 name = "signal-hook-registry"
-version = "1.4.1"
+version = "1.4.2"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1"
+checksum = "a9e9e0b4211b72e7b8b6e85c807d36c212bdb33ea8587f7569562a84df5465b1"
 dependencies = [
  "libc",
 ]
@@ -4883,7 +4885,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "struct_iterable_internal",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -4901,7 +4903,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "structmeta-derive",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -4912,7 +4914,7 @@ checksum = "a60bcaff7397072dca0017d1db428e30d5002e00b6847703e2e42005c95fbe00"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -4943,7 +4945,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "rustversion",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -4956,7 +4958,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "rustversion",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -5018,9 +5020,9 @@ dependencies = [
 
 [[package]]
 name = "syn"
-version = "2.0.59"
+version = "2.0.60"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "4a6531ffc7b071655e4ce2e04bd464c4830bb585a61cabb96cf808f05172615a"
+checksum = "909518bc7b1c9b779f1bbf07f2929d35af9f0f37e47c6e9ef7f9dddc1e1821f3"
 dependencies = [
  "proc-macro2",
  "quote",
@@ -5070,7 +5072,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -5129,7 +5131,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "structmeta",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -5148,22 +5150,22 @@ dependencies = [
 
 [[package]]
 name = "thiserror"
-version = "1.0.58"
+version = "1.0.59"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "03468839009160513471e86a034bb2c5c0e4baae3b43f79ffc55c4a5427b3297"
+checksum = "f0126ad08bff79f29fc3ae6a55cc72352056dfff61e3ff8bb7129476d44b23aa"
 dependencies = [
  "thiserror-impl",
 ]
 
 [[package]]
 name = "thiserror-impl"
-version = "1.0.58"
+version = "1.0.59"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c61f3ba182994efc43764a46c018c347bc492c79f024e705f46567b418f6d4f7"
+checksum = "d1cd413b5d558b4c5bf3680e324a6fa5014e7b7c067a51e69dbdf47eb7148b66"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -5261,7 +5263,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -5357,7 +5359,7 @@ dependencies = [
  "serde",
  "serde_spanned",
  "toml_datetime",
- "toml_edit 0.22.9",
+ "toml_edit 0.22.12",
 ]
 
 [[package]]
@@ -5382,9 +5384,9 @@ dependencies = [
 
 [[package]]
 name = "toml_edit"
-version = "0.22.9"
+version = "0.22.12"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "8e40bb779c5187258fd7aad0eb68cb8706a0a81fa712fbea808ab43c4b8374c4"
+checksum = "d3328d4f68a705b2a4498da1d580585d39a6510f98318a2cec3018a7ec61ddef"
 dependencies = [
  "indexmap 2.2.6",
  "serde",
@@ -5486,7 +5488,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -5733,7 +5735,7 @@ dependencies = [
  "once_cell",
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
  "wasm-bindgen-shared",
 ]
 
@@ -5767,7 +5769,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
  "wasm-bindgen-backend",
  "wasm-bindgen-shared",
 ]
@@ -5841,11 +5843,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6"
 
 [[package]]
 name = "winapi-util"
-version = "0.1.6"
+version = "0.1.7"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "f29e6f9198ba0d26b4c9f07dbe6f9ed633e1f3d5b8b414090084349e46a52596"
+checksum = "134306a13c5647ad6453e8deaec55d3a44d6021970129e6188735e74bf546697"
 dependencies = [
- "winapi",
+ "windows-sys 0.52.0",
 ]
 
 [[package]]
@@ -5880,18 +5882,18 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
 checksum = "e48a53791691ab099e5e2ad123536d0fff50652600abaf43bbf952894110d0be"
 dependencies = [
  "windows-core 0.52.0",
- "windows-implement",
- "windows-interface",
+ "windows-implement 0.52.0",
+ "windows-interface 0.52.0",
  "windows-targets 0.52.5",
 ]
 
 [[package]]
 name = "windows"
-version = "0.54.0"
+version = "0.56.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "9252e5725dbed82865af151df558e754e4a3c2c30818359eb17465f1346a1b49"
+checksum = "1de69df01bdf1ead2f4ac895dc77c9351aefff65b2f3db429a343f9cbf05e132"
 dependencies = [
- "windows-core 0.54.0",
+ "windows-core 0.56.0",
  "windows-targets 0.52.5",
 ]
 
@@ -5915,10 +5917,12 @@ dependencies = [
 
 [[package]]
 name = "windows-core"
-version = "0.54.0"
+version = "0.56.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "12661b9c89351d684a50a8a643ce5f608e20243b9fb84687800163429f161d65"
+checksum = "4698e52ed2d08f8658ab0c39512a7c00ee5fe2688c65f8c0a4f06750d729f2a6"
 dependencies = [
+ "windows-implement 0.56.0",
+ "windows-interface 0.56.0",
  "windows-result",
  "windows-targets 0.52.5",
 ]
@@ -5931,7 +5935,18 @@ checksum = "12168c33176773b86799be25e2a2ba07c7aab9968b37541f1094dbd7a60c8946"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
+]
+
+[[package]]
+name = "windows-implement"
+version = "0.56.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "f6fc35f58ecd95a9b71c4f2329b911016e6bec66b3f2e6a4aad86bd2e99e2f9b"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -5942,7 +5957,18 @@ checksum = "9d8dc32e0095a7eeccebd0e3f09e9509365ecb3fc6ac4d6f5f14a3f6392942d1"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
+]
+
+[[package]]
+name = "windows-interface"
+version = "0.56.0"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "08990546bf4edef8f431fa6326e032865f27138718c587dc21bc0265bbcb57cc"
+dependencies = [
+ "proc-macro2",
+ "quote",
+ "syn 2.0.60",
 ]
 
 [[package]]
@@ -6222,11 +6248,11 @@ checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.59",
+ "syn 2.0.60",
 ]
 
 [[package]]
 name = "zeroize"
-version = "1.7.0"
+version = "1.8.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d"
+checksum = "63381fa6624bf92130a6b87c0d07380116f80b565c42cf0d754136f0238359ef"
diff --git a/iroh-cli/Cargo.toml b/iroh-cli/Cargo.toml
index 07d57df1615..04744d5533f 100644
--- a/iroh-cli/Cargo.toml
+++ b/iroh-cli/Cargo.toml
@@ -43,7 +43,7 @@ iroh-metrics = { version = "0.14.0", path = "../iroh-metrics" }
 parking_lot = "0.12.1"
 portable-atomic = "1"
 postcard = "1.0.8"
-quic-rpc = { version = "0.7.0", features = ["flume-transport", "quinn-transport"] }
+quic-rpc = { version = "0.8.0", features = ["flume-transport", "quinn-transport"] }
 quinn = "0.10.2"
 rand = "0.8.5"
 rustyline = "12.0.0"
diff --git a/iroh/Cargo.toml b/iroh/Cargo.toml
index b74738c130d..81c5bab4d26 100644
--- a/iroh/Cargo.toml
+++ b/iroh/Cargo.toml
@@ -37,7 +37,7 @@ iroh-sync = { version = "0.14.0", path = "../iroh-sync" }
 iroh-gossip = { version = "0.14.0", path = "../iroh-gossip" }
 parking_lot = "0.12.1"
 postcard = { version = "1", default-features = false, features = ["alloc", "use-std", "experimental-derive"] }
-quic-rpc = { version = "0.7.0", default-features = false, features = ["flume-transport", "quinn-transport"] }
+quic-rpc = { version = "0.8.0", default-features = false, features = ["flume-transport", "quinn-transport"] }
 quinn = "0.10"
 rand = "0.8"
 serde = { version = "1", features = ["derive"] }
diff --git a/iroh/src/client/docs.rs b/iroh/src/client/docs.rs
index c4c6a7bdfa6..93c7331b42e 100644
--- a/iroh/src/client/docs.rs
+++ b/iroh/src/client/docs.rs
@@ -338,9 +338,9 @@ where
         let stream = self
             .0
             .rpc
-            .server_streaming(DocSubscribeRequest { doc_id: self.id() })
+            .try_server_streaming(DocSubscribeRequest { doc_id: self.id() })
             .await?;
-        Ok(flatten(stream).map(|res| match res {
+        Ok(stream.map(|res| match res {
             Ok(res) => Ok(res.event.into()),
             Err(err) => Err(err),
         }))
diff --git a/iroh/src/node/rpc.rs b/iroh/src/node/rpc.rs
index 917dcf54d75..a1701679c81 100644
--- a/iroh/src/node/rpc.rs
+++ b/iroh/src/node/rpc.rs
@@ -254,11 +254,10 @@ impl<D: BaoStore> Handler<D> {
                     .await
                 }
                 DocSubscribe(_msg) => {
-                    todo!()
-                    /*chan.server_streaming(msg, handler, |handler, req| {
+                    chan.try_server_streaming(msg, handler, |handler, req| {
                         handler.inner.sync.doc_subscribe(req)
                     })
-                    .await*/
+                    .await
                 }
                 DocSetDownloadPolicy(msg) => {
                     chan.rpc(msg, handler, |handler, req| async move {
diff --git a/iroh/src/rpc_protocol.rs b/iroh/src/rpc_protocol.rs
index ce71da856db..80ead92d112 100644
--- a/iroh/src/rpc_protocol.rs
+++ b/iroh/src/rpc_protocol.rs
@@ -29,6 +29,7 @@ use iroh_sync::{
 };
 use quic_rpc::{
     message::{BidiStreaming, BidiStreamingMsg, Msg, RpcMsg, ServerStreaming, ServerStreamingMsg},
+    pattern::try_server_streaming::{TryServerStreaming, TryServerStreamingMsg},
     Service,
 };
 use serde::{Deserialize, Serialize};
@@ -565,11 +566,13 @@ pub struct DocSubscribeRequest {
 }
 
 impl Msg<ProviderService> for DocSubscribeRequest {
-    type Pattern = ServerStreaming;
+    type Pattern = TryServerStreaming;
 }
 
-impl ServerStreamingMsg<ProviderService> for DocSubscribeRequest {
-    type Response = RpcResult<DocSubscribeResponse>;
+impl TryServerStreamingMsg<ProviderService> for DocSubscribeRequest {
+    type Item = DocSubscribeResponse;
+    type ItemError = RpcError;
+    type CreateError = anyhow::Error;
 }
 
 /// Response to [`DocSubscribeRequest`]
diff --git a/iroh/tests/sync.rs b/iroh/tests/sync.rs
index 7ce718b78f6..72ebe99dad6 100644
--- a/iroh/tests/sync.rs
+++ b/iroh/tests/sync.rs
@@ -7,7 +7,8 @@ use std::{
 
 use anyhow::{anyhow, bail, Context, Result};
 use bytes::Bytes;
-use futures_lite::{Stream, StreamExt};
+use futures_lite::Stream;
+use futures_util::{FutureExt, StreamExt, TryStreamExt};
 use iroh::{
     client::{mem::Doc, Entry, LiveEvent},
     node::{Builder, Node},
@@ -797,7 +798,7 @@ async fn sync_big() -> Result<()> {
 
     info!("shutdown");
     for node in nodes {
-        node.shutdown();
+        node.shutdown().await?;
     }
 
     Ok(())
@@ -844,7 +845,7 @@ async fn publish(
 async fn collect_futures<T>(
     futs: impl IntoIterator<Item = impl Future<Output = anyhow::Result<T>>>,
 ) -> anyhow::Result<Vec<T>> {
-    futures::future::join_all(futs)
+    futures_buffered::join_all(futs)
         .await
         .into_iter()
         .collect::<Result<Vec<_>>>()

From e9aa5d9f296e32371c2b9e791c74fb9d52a169fa Mon Sep 17 00:00:00 2001
From: Ruediger Klaehn <rklaehn@protonmail.com>
Date: Thu, 25 Apr 2024 19:38:04 +0300
Subject: [PATCH 20/25] Fix compile errors

---
 iroh/src/client/docs.rs  | 2 +-
 iroh/src/node/rpc.rs     | 6 +++---
 iroh/src/rpc_protocol.rs | 5 +++--
 3 files changed, 7 insertions(+), 6 deletions(-)

diff --git a/iroh/src/client/docs.rs b/iroh/src/client/docs.rs
index 93c7331b42e..7a5f63a4995 100644
--- a/iroh/src/client/docs.rs
+++ b/iroh/src/client/docs.rs
@@ -342,7 +342,7 @@ where
             .await?;
         Ok(stream.map(|res| match res {
             Ok(res) => Ok(res.event.into()),
-            Err(err) => Err(err),
+            Err(err) => Err(err.into()),
         }))
     }
 
diff --git a/iroh/src/node/rpc.rs b/iroh/src/node/rpc.rs
index a1701679c81..29941fc6801 100644
--- a/iroh/src/node/rpc.rs
+++ b/iroh/src/node/rpc.rs
@@ -253,9 +253,9 @@ impl<D: BaoStore> Handler<D> {
                     })
                     .await
                 }
-                DocSubscribe(_msg) => {
-                    chan.try_server_streaming(msg, handler, |handler, req| {
-                        handler.inner.sync.doc_subscribe(req)
+                DocSubscribe(msg) => {
+                    chan.try_server_streaming(msg, handler, |handler, req| async move {
+                        handler.inner.sync.doc_subscribe(req).await
                     })
                     .await
                 }
diff --git a/iroh/src/rpc_protocol.rs b/iroh/src/rpc_protocol.rs
index 80ead92d112..24c95ead956 100644
--- a/iroh/src/rpc_protocol.rs
+++ b/iroh/src/rpc_protocol.rs
@@ -29,7 +29,7 @@ use iroh_sync::{
 };
 use quic_rpc::{
     message::{BidiStreaming, BidiStreamingMsg, Msg, RpcMsg, ServerStreaming, ServerStreamingMsg},
-    pattern::try_server_streaming::{TryServerStreaming, TryServerStreamingMsg},
+    pattern::try_server_streaming::{StreamCreated, TryServerStreaming, TryServerStreamingMsg},
     Service,
 };
 use serde::{Deserialize, Serialize};
@@ -572,7 +572,7 @@ impl Msg<ProviderService> for DocSubscribeRequest {
 impl TryServerStreamingMsg<ProviderService> for DocSubscribeRequest {
     type Item = DocSubscribeResponse;
     type ItemError = RpcError;
-    type CreateError = anyhow::Error;
+    type CreateError = RpcError;
 }
 
 /// Response to [`DocSubscribeRequest`]
@@ -1224,6 +1224,7 @@ pub enum ProviderResponse {
     DocGetDownloadPolicy(RpcResult<DocGetDownloadPolicyResponse>),
     DocSetDownloadPolicy(RpcResult<DocSetDownloadPolicyResponse>),
     DocGetSyncPeers(RpcResult<DocGetSyncPeersResponse>),
+    StreamCreated(RpcResult<StreamCreated>),
 
     AuthorList(RpcResult<AuthorListResponse>),
     AuthorCreate(RpcResult<AuthorCreateResponse>),

From 416fc57f1db86686f1b49286ac3a19d548d3a96a Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 26 Apr 2024 09:35:31 +0200
Subject: [PATCH 21/25] fix linux

---
 iroh-net/src/net/interfaces/linux.rs | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)

diff --git a/iroh-net/src/net/interfaces/linux.rs b/iroh-net/src/net/interfaces/linux.rs
index 8bde4be1a3e..de15895d15a 100644
--- a/iroh-net/src/net/interfaces/linux.rs
+++ b/iroh-net/src/net/interfaces/linux.rs
@@ -2,7 +2,7 @@
 
 use anyhow::{anyhow, Result};
 #[cfg(not(target_os = "android"))]
-use futures_lite::StreamExt;
+use futures_util::TryStreamExt;
 use tokio::fs::File;
 use tokio::io::{AsyncBufReadExt, BufReader};
 
@@ -133,8 +133,7 @@ async fn default_route_netlink_family(
     family: rtnetlink::IpVersion,
 ) -> Result<Option<(String, u32)>> {
     let mut routes = handle.route().get(family).execute();
-    while let Some(route) = routes.next().await {
-        let route = route?;
+    while let Some(route) = routes.try_next().await? {
         if route.gateway().is_none() {
             // A default route has a gateway.
             continue;
@@ -159,10 +158,10 @@ async fn default_route_netlink_family(
 async fn iface_by_index(handle: &rtnetlink::Handle, index: u32) -> Result<String> {
     let mut links = handle.link().get().match_index(index).execute();
     let msg = links
-        .next()
-        .await
-        .ok_or_else(|| anyhow!("No netlink response"))?
-        .await?;
+        .try_next()
+        .await?
+        .ok_or_else(|| anyhow!("No netlink response"))?;
+
     for nla in msg.nlas {
         if let netlink_packet_route::link::nlas::Nla::IfName(name) = nla {
             return Ok(name);

From ceaa99766520bc05b4c0d9f21be5e20f2cc6e3cc Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 26 Apr 2024 09:37:05 +0200
Subject: [PATCH 22/25] update deps:

---
 Cargo.lock | 45 +++++++++++++++++++++++++++------------------
 1 file changed, 27 insertions(+), 18 deletions(-)

diff --git a/Cargo.lock b/Cargo.lock
index b445b64c5af..b16781dced4 100644
--- a/Cargo.lock
+++ b/Cargo.lock
@@ -2841,9 +2841,9 @@ checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c"
 
 [[package]]
 name = "lock_api"
-version = "0.4.11"
+version = "0.4.12"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45"
+checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17"
 dependencies = [
  "autocfg",
  "scopeguard",
@@ -3364,9 +3364,9 @@ checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae"
 
 [[package]]
 name = "parking_lot"
-version = "0.12.1"
+version = "0.12.2"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f"
+checksum = "7e4af0ca4f6caed20e900d564c242b8e5d4903fdacf31d3daf527b66fe6f42fb"
 dependencies = [
  "lock_api",
  "parking_lot_core",
@@ -3374,15 +3374,15 @@ dependencies = [
 
 [[package]]
 name = "parking_lot_core"
-version = "0.9.9"
+version = "0.9.10"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e"
+checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8"
 dependencies = [
  "cfg-if",
  "libc",
- "redox_syscall",
+ "redox_syscall 0.5.1",
  "smallvec",
- "windows-targets 0.48.5",
+ "windows-targets 0.52.5",
 ]
 
 [[package]]
@@ -4067,6 +4067,15 @@ dependencies = [
  "bitflags 1.3.2",
 ]
 
+[[package]]
+name = "redox_syscall"
+version = "0.5.1"
+source = "registry+https://github.com/rust-lang/crates.io-index"
+checksum = "469052894dcb553421e483e4209ee581a45100d31b4018de03e5a7ad86374a7e"
+dependencies = [
+ "bitflags 2.5.0",
+]
+
 [[package]]
 name = "redox_users"
 version = "0.4.5"
@@ -4615,11 +4624,11 @@ dependencies = [
 
 [[package]]
 name = "serde_with"
-version = "3.7.0"
+version = "3.8.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "ee80b0e361bbf88fd2f6e242ccd19cfda072cb0faa6ae694ecee08199938569a"
+checksum = "2c85f8e96d1d6857f13768fcbd895fcb06225510022a2774ed8b5150581847b0"
 dependencies = [
- "base64 0.21.7",
+ "base64 0.22.0",
  "chrono",
  "hex",
  "indexmap 1.9.3",
@@ -4633,9 +4642,9 @@ dependencies = [
 
 [[package]]
 name = "serde_with_macros"
-version = "3.7.0"
+version = "3.8.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "6561dc161a9224638a31d876ccdfefbc1df91d3f3a8342eddb35f055d48c7655"
+checksum = "c8b3a576c4eb2924262d5951a3b737ccaf16c931e39a2810c36f9a7e25575557"
 dependencies = [
  "darling",
  "proc-macro2",
@@ -5814,7 +5823,7 @@ version = "1.5.1"
 source = "registry+https://github.com/rust-lang/crates.io-index"
 checksum = "a44ab49fad634e88f55bf8f9bb3abd2f27d7204172a112c7c9987e01c1c94ea9"
 dependencies = [
- "redox_syscall",
+ "redox_syscall 0.4.1",
  "wasite",
  "web-sys",
 ]
@@ -5843,9 +5852,9 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6"
 
 [[package]]
 name = "winapi-util"
-version = "0.1.7"
+version = "0.1.8"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "134306a13c5647ad6453e8deaec55d3a44d6021970129e6188735e74bf546697"
+checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b"
 dependencies = [
  "windows-sys 0.52.0",
 ]
@@ -6253,6 +6262,6 @@ dependencies = [
 
 [[package]]
 name = "zeroize"
-version = "1.8.0"
+version = "1.7.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "63381fa6624bf92130a6b87c0d07380116f80b565c42cf0d754136f0238359ef"
+checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d"

From 0c3f2f24e479f474b817c8f2baf399097a2e1f27 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 26 Apr 2024 12:33:41 +0200
Subject: [PATCH 23/25] improve shutdown

---
 iroh-net/src/util.rs         | 50 +++++++++++++++++++++++++++++++++---
 iroh/src/node/builder.rs     |  3 +--
 iroh/src/sync_engine.rs      | 29 +++++++--------------
 iroh/src/sync_engine/live.rs | 17 +++++++++---
 4 files changed, 70 insertions(+), 29 deletions(-)

diff --git a/iroh-net/src/util.rs b/iroh-net/src/util.rs
index 0424fbf06a6..58165a9b8dc 100644
--- a/iroh-net/src/util.rs
+++ b/iroh-net/src/util.rs
@@ -3,16 +3,22 @@
 use std::{
     future::Future,
     pin::Pin,
+    sync::Arc,
     task::{Context, Poll},
 };
 
+use futures_lite::future::Boxed as BoxFuture;
+use futures_util::{future::Shared, FutureExt};
+
 /// A join handle that owns the task it is running, and aborts it when dropped.
 #[derive(Debug, derive_more::Deref)]
-pub struct AbortingJoinHandle<T>(pub tokio::task::JoinHandle<T>);
+pub struct AbortingJoinHandle<T> {
+    handle: tokio::task::JoinHandle<T>,
+}
 
 impl<T> From<tokio::task::JoinHandle<T>> for AbortingJoinHandle<T> {
     fn from(handle: tokio::task::JoinHandle<T>) -> Self {
-        Self(handle)
+        Self { handle }
     }
 }
 
@@ -20,13 +26,49 @@ impl<T> Future for AbortingJoinHandle<T> {
     type Output = std::result::Result<T, tokio::task::JoinError>;
 
     fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
-        Pin::new(&mut self.0).poll(cx)
+        Pin::new(&mut self.handle).poll(cx)
     }
 }
 
 impl<T> Drop for AbortingJoinHandle<T> {
     fn drop(&mut self) {
-        self.0.abort();
+        self.handle.abort();
+    }
+}
+
+/// A join handle that owns the task it is running, and aborts it when dropped.
+/// It is cloneable and will abort when the last instance is dropped.
+#[derive(Debug, Clone)]
+pub struct SharedAbortingJoinHandle<T: Clone + Send> {
+    fut: Shared<BoxFuture<std::result::Result<T, String>>>,
+    abort: Arc<tokio::task::AbortHandle>,
+}
+
+impl<T: Clone + Send + 'static> From<tokio::task::JoinHandle<T>> for SharedAbortingJoinHandle<T> {
+    fn from(handle: tokio::task::JoinHandle<T>) -> Self {
+        let abort = handle.abort_handle();
+        let fut: BoxFuture<std::result::Result<T, String>> =
+            Box::pin(async move { handle.await.map_err(|e| e.to_string()) });
+        Self {
+            fut: fut.shared(),
+            abort: Arc::new(abort),
+        }
+    }
+}
+
+impl<T: Clone + Send> Future for SharedAbortingJoinHandle<T> {
+    type Output = std::result::Result<T, String>;
+
+    fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
+        Pin::new(&mut self.fut).poll(cx)
+    }
+}
+
+impl<T: Clone + Send> Drop for SharedAbortingJoinHandle<T> {
+    fn drop(&mut self) {
+        if Arc::strong_count(&self.abort) == 1 {
+            self.abort.abort();
+        }
     }
 }
 
diff --git a/iroh/src/node/builder.rs b/iroh/src/node/builder.rs
index c906813b438..31a6bd631f2 100644
--- a/iroh/src/node/builder.rs
+++ b/iroh/src/node/builder.rs
@@ -19,7 +19,6 @@ use iroh_net::{
     discovery::{dns::DnsDiscovery, pkarr_publish::PkarrPublisher, ConcurrentDiscovery, Discovery},
     magic_endpoint::get_alpn,
     relay::RelayMode,
-    util::AbortingJoinHandle,
     MagicEndpoint,
 };
 use iroh_sync::net::SYNC_ALPN;
@@ -407,7 +406,7 @@ where
             let db = self.blobs_store.clone();
             let callbacks = callbacks.clone();
             let task = lp.spawn_pinned(move || Self::gc_loop(db, sync_db, gc_period, callbacks));
-            Some(AbortingJoinHandle(task))
+            Some(task.into())
         } else {
             None
         };
diff --git a/iroh/src/sync_engine.rs b/iroh/src/sync_engine.rs
index 8a9736e6ecb..69d426ac9ef 100644
--- a/iroh/src/sync_engine.rs
+++ b/iroh/src/sync_engine.rs
@@ -9,11 +9,11 @@ use futures_lite::{Stream, StreamExt};
 use iroh_bytes::downloader::Downloader;
 use iroh_bytes::{store::EntryStatus, Hash};
 use iroh_gossip::net::Gossip;
+use iroh_net::util::SharedAbortingJoinHandle;
 use iroh_net::{key::PublicKey, MagicEndpoint, NodeAddr};
 use iroh_sync::{actor::SyncHandle, ContentStatus, ContentStatusCallback, Entry, NamespaceId};
 use serde::{Deserialize, Serialize};
 use tokio::sync::{mpsc, oneshot};
-use tokio::task::JoinSet;
 use tracing::{error, error_span, Instrument};
 
 mod gossip;
@@ -43,7 +43,7 @@ pub struct SyncEngine {
     pub(crate) endpoint: MagicEndpoint,
     pub(crate) sync: SyncHandle,
     to_live_actor: mpsc::Sender<ToLiveActor>,
-    tasks: Arc<JoinSet<()>>,
+    actor_handle: SharedAbortingJoinHandle<()>,
     #[debug("ContentStatusCallback")]
     content_status_cb: ContentStatusCallback,
 }
@@ -80,34 +80,26 @@ impl SyncEngine {
             live_actor_tx.clone(),
             to_gossip_actor,
         );
-        let mut gossip_actor = GossipActor::new(
+        let gossip_actor = GossipActor::new(
             to_gossip_actor_recv,
             sync.clone(),
             gossip,
             live_actor_tx.clone(),
         );
-        let mut tasks = JoinSet::new();
-        tasks.spawn(
+        let actor_handle = tokio::task::spawn(
             async move {
-                if let Err(err) = actor.run().await {
+                if let Err(err) = actor.run(gossip_actor).await {
                     error!("sync actor failed: {err:?}");
                 }
             }
             .instrument(error_span!("sync", %me)),
         );
-        tasks.spawn(
-            async move {
-                if let Err(err) = gossip_actor.run().await {
-                    error!("gossip recv actor failed: {err:?}");
-                }
-            }
-            .instrument(error_span!("sync", %me)),
-        );
+
         Self {
             endpoint,
             sync,
             to_live_actor: live_actor_tx,
-            tasks: Arc::new(tasks),
+            actor_handle: actor_handle.into(),
             content_status_cb,
         }
     }
@@ -213,11 +205,8 @@ impl SyncEngine {
     /// Shutdown the sync engine.
     pub async fn shutdown(self) -> Result<()> {
         self.to_live_actor.send(ToLiveActor::Shutdown).await?;
-        if let Ok(mut tasks) = Arc::try_unwrap(self.tasks) {
-            while let Some(t) = tasks.join_next().await {
-                t?;
-            }
-        }
+
+        self.actor_handle.await.map_err(|e| anyhow::anyhow!(e))?;
         Ok(())
     }
 }
diff --git a/iroh/src/sync_engine/live.rs b/iroh/src/sync_engine/live.rs
index a25923a061c..db81473319c 100644
--- a/iroh/src/sync_engine/live.rs
+++ b/iroh/src/sync_engine/live.rs
@@ -25,9 +25,9 @@ use tokio::{
     sync::{self, mpsc, oneshot},
     task::JoinSet,
 };
-use tracing::{debug, error, info, instrument, trace, warn, Instrument, Span};
+use tracing::{debug, error, error_span, info, instrument, trace, warn, Instrument, Span};
 
-use super::gossip::ToGossipActor;
+use super::gossip::{GossipActor, ToGossipActor};
 use super::state::{NamespaceStates, Origin, SyncReason};
 
 /// An iroh-sync operation
@@ -205,11 +205,22 @@ impl<B: iroh_bytes::store::Store> LiveActor<B> {
     }
 
     /// Run the actor loop.
-    pub async fn run(&mut self) -> Result<()> {
+    pub async fn run(&mut self, mut gossip_actor: GossipActor) -> Result<()> {
+        let me = self.endpoint.node_id().fmt_short();
+        let gossip_handle = tokio::task::spawn(
+            async move {
+                if let Err(err) = gossip_actor.run().await {
+                    error!("gossip recv actor failed: {err:?}");
+                }
+            }
+            .instrument(error_span!("sync", %me)),
+        );
+
         let res = self.run_inner().await;
         if let Err(err) = self.shutdown().await {
             error!(?err, "Error during shutdown");
         }
+        gossip_handle.await?;
         res
     }
 

From f12562dd0f3d9189f2d5e85b67e9c8955bb7db2c Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Fri, 26 Apr 2024 13:05:52 +0200
Subject: [PATCH 24/25] fixup

---
 iroh-cli/src/commands/doctor.rs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/iroh-cli/src/commands/doctor.rs b/iroh-cli/src/commands/doctor.rs
index 92a1ee676d2..d4629c0073d 100644
--- a/iroh-cli/src/commands/doctor.rs
+++ b/iroh-cli/src/commands/doctor.rs
@@ -366,7 +366,7 @@ impl Gui {
             .template("{spinner:.green} [{bar:80.cyan/blue}] {msg} {bytes}/{total_bytes} ({bytes_per_sec})").unwrap()
             .progress_chars("█▉▊▋▌▍▎▏ "));
         let counters2 = counters.clone();
-        let counter_task = AbortingJoinHandle(tokio::spawn(async move {
+        let counter_task = AbortingJoinHandle::from(tokio::spawn(async move {
             loop {
                 Self::update_counters(&counters2);
                 Self::update_connection_info(&conn_info, &endpoint, &node_id);

From f6945a74a2b825a21ee7519cb6b9196fc3ea8fe8 Mon Sep 17 00:00:00 2001
From: dignifiedquire <me@dignifiedquire.com>
Date: Mon, 29 Apr 2024 09:36:50 +0200
Subject: [PATCH 25/25] fixup

---
 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 b275cdd5e28..9c5b5a2885d 100644
--- a/iroh-net/src/test_utils.rs
+++ b/iroh-net/src/test_utils.rs
@@ -388,7 +388,7 @@ pub(crate) mod pkarr_dns_state {
         ) -> impl Future<Output = Result<()>> + Send {
             const TTL: u32 = 30;
             let res = self.resolve_dns(query, reply, TTL);
-            futures::future::ready(res)
+            std::future::ready(res)
         }
     }
 }