Skip to content

Commit 0784403

Browse files
fix(iroh-blobs): do not hit the network when downloading blobs which are complete (#2586)
## Description Two changes to the downloader: * Never try to download from ourselves. If the only provider node added is our own node, fail with error "no providers". * The actual download request flow is turned into a generator (while keeping API compatibility for the existing `get_to_db` public function). A new `get_to_db_in_steps` function either runs to completion if the requested data is fully available locally, or yields a `NeedsConn` struct at the point where it needs a network connection to proceed. The `NeedsConn` has an `async proceed(self, conn: Connection)`, which must be called with a connection for the actual download to start. This two-step process allows the downloader to check if we should dial nodes at all, or are already done without doing anything, while emitting the exact same flow of events (because we run the same loop) to the client. To achieve this, `get_to_db` now uses a genawaiter generator internally. This means that the big loop that is the iroh-blobs protocol request flow does not have to be changed at all, only that instead of a closure we yield and resume, which makes this much easier to integrate into an external state machine like the downloader. The changes needed for this for the downloader are a bit verbose because the downloader itself is generic over a `Getter`, with impls for the actual impl and a test impl that does not use networking; therefore the new `NeedsConn` state has to be modeled with an additional associated type and trait here. This PR adds three tests: * Downloading a missing blob from the local node fails without trying to connect to ourselves * Downloading an existing blob succeeds without trying to download * Downloading an existing collection succeeds without trying to download Closes #2575 Replaced #2576 ## Notes and open questions ## Breaking changes None, only an API addition to the public API of iroh_blobs: `iroh_blobs::get::check_local_with_progress_if_complete` --------- Co-authored-by: dignifiedquire <me@dignifiedquire.com>
1 parent 43ef8b6 commit 0784403

File tree

11 files changed

+594
-163
lines changed

11 files changed

+594
-163
lines changed

iroh-blobs/src/downloader.rs

Lines changed: 134 additions & 57 deletions
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,12 @@
2727
//! requests to a single node is also limited.
2828
2929
use std::{
30-
collections::{hash_map::Entry, HashMap, HashSet},
30+
collections::{
31+
hash_map::{self, Entry},
32+
HashMap, HashSet,
33+
},
3134
fmt,
35+
future::Future,
3236
num::NonZeroUsize,
3337
sync::{
3438
atomic::{AtomicU64, Ordering},
@@ -46,7 +50,7 @@ use tokio::{
4650
sync::{mpsc, oneshot},
4751
task::JoinSet,
4852
};
49-
use tokio_util::{sync::CancellationToken, time::delay_queue};
53+
use tokio_util::{either::Either, sync::CancellationToken, time::delay_queue};
5054
use tracing::{debug, error_span, trace, warn, Instrument};
5155

5256
use crate::{
@@ -75,13 +79,15 @@ pub struct IntentId(pub u64);
7579
/// Trait modeling a dialer. This allows for IO-less testing.
7680
pub trait Dialer: Stream<Item = (NodeId, anyhow::Result<Self::Connection>)> + Unpin {
7781
/// Type of connections returned by the Dialer.
78-
type Connection: Clone;
82+
type Connection: Clone + 'static;
7983
/// Dial a node.
8084
fn queue_dial(&mut self, node_id: NodeId);
8185
/// Get the number of dialing nodes.
8286
fn pending_count(&self) -> usize;
8387
/// Check if a node is being dialed.
8488
fn is_pending(&self, node: NodeId) -> bool;
89+
/// Get the node id of our node.
90+
fn node_id(&self) -> NodeId;
8591
}
8692

8793
/// Signals what should be done with the request when it fails.
@@ -97,20 +103,39 @@ pub enum FailureAction {
97103
RetryLater(anyhow::Error),
98104
}
99105

100-
/// Future of a get request.
101-
type GetFut = BoxedLocal<InternalDownloadResult>;
106+
/// Future of a get request, for the checking stage.
107+
type GetStartFut<N> = BoxedLocal<Result<GetOutput<N>, FailureAction>>;
108+
/// Future of a get request, for the downloading stage.
109+
type GetProceedFut = BoxedLocal<InternalDownloadResult>;
102110

103111
/// Trait modelling performing a single request over a connection. This allows for IO-less testing.
104112
pub trait Getter {
105113
/// Type of connections the Getter requires to perform a download.
106-
type Connection;
107-
/// Return a future that performs the download using the given connection.
114+
type Connection: 'static;
115+
/// Type of the intermediary state returned from [`Self::get`] if a connection is needed.
116+
type NeedsConn: NeedsConn<Self::Connection>;
117+
/// Returns a future that checks the local store if the request is already complete, returning
118+
/// a struct implementing [`NeedsConn`] if we need a network connection to proceed.
108119
fn get(
109120
&mut self,
110121
kind: DownloadKind,
111-
conn: Self::Connection,
112122
progress_sender: BroadcastProgressSender,
113-
) -> GetFut;
123+
) -> GetStartFut<Self::NeedsConn>;
124+
}
125+
126+
/// Trait modelling the intermediary state when a connection is needed to proceed.
127+
pub trait NeedsConn<C>: std::fmt::Debug + 'static {
128+
/// Proceeds the download with the given connection.
129+
fn proceed(self, conn: C) -> GetProceedFut;
130+
}
131+
132+
/// Output returned from [`Getter::get`].
133+
#[derive(Debug)]
134+
pub enum GetOutput<N> {
135+
/// The request is already complete in the local store.
136+
Complete(Stats),
137+
/// The request needs a connection to continue.
138+
NeedsConn(N),
114139
}
115140

116141
/// Concurrency limits for the [`Downloader`].
@@ -280,7 +305,7 @@ pub struct DownloadHandle {
280305
receiver: oneshot::Receiver<ExternalDownloadResult>,
281306
}
282307

283-
impl std::future::Future for DownloadHandle {
308+
impl Future for DownloadHandle {
284309
type Output = ExternalDownloadResult;
285310

286311
fn poll(
@@ -424,10 +449,12 @@ struct IntentHandlers {
424449
}
425450

426451
/// Information about a request.
427-
#[derive(Debug, Default)]
428-
struct RequestInfo {
452+
#[derive(Debug)]
453+
struct RequestInfo<NC> {
429454
/// Registered intents with progress senders and result callbacks.
430455
intents: HashMap<IntentId, IntentHandlers>,
456+
progress_sender: BroadcastProgressSender,
457+
get_state: Option<NC>,
431458
}
432459

433460
/// Information about a request in progress.
@@ -529,7 +556,7 @@ struct Service<G: Getter, D: Dialer> {
529556
/// Queue of pending downloads.
530557
queue: Queue,
531558
/// Information about pending and active requests.
532-
requests: HashMap<DownloadKind, RequestInfo>,
559+
requests: HashMap<DownloadKind, RequestInfo<G::NeedsConn>>,
533560
/// State of running downloads.
534561
active_requests: HashMap<DownloadKind, ActiveRequestInfo>,
535562
/// Tasks for currently running downloads.
@@ -666,48 +693,85 @@ impl<G: Getter<Connection = D::Connection>, D: Dialer> Service<G, D> {
666693
on_progress: progress,
667694
};
668695

669-
// early exit if no providers.
670-
if nodes.is_empty() && self.providers.get_candidates(&kind.hash()).next().is_none() {
671-
self.finalize_download(
672-
kind,
673-
[(intent_id, intent_handlers)].into(),
674-
Err(DownloadError::NoProviders),
675-
);
676-
return;
677-
}
678-
679696
// add the nodes to the provider map
680-
let updated = self
681-
.providers
682-
.add_hash_with_nodes(kind.hash(), nodes.iter().map(|n| n.node_id));
697+
// (skip the node id of our own node - we should never attempt to download from ourselves)
698+
let node_ids = nodes
699+
.iter()
700+
.map(|n| n.node_id)
701+
.filter(|node_id| *node_id != self.dialer.node_id());
702+
let updated = self.providers.add_hash_with_nodes(kind.hash(), node_ids);
683703

684704
// queue the transfer (if not running) or attach to transfer progress (if already running)
685-
if self.active_requests.contains_key(&kind) {
686-
// the transfer is already running, so attach the progress sender
687-
if let Some(on_progress) = &intent_handlers.on_progress {
688-
// this is async because it sends the current state over the progress channel
689-
if let Err(err) = self
690-
.progress_tracker
691-
.subscribe(kind, on_progress.clone())
692-
.await
693-
{
694-
debug!(?err, %kind, "failed to subscribe progress sender to transfer");
705+
match self.requests.entry(kind) {
706+
hash_map::Entry::Occupied(mut entry) => {
707+
if let Some(on_progress) = &intent_handlers.on_progress {
708+
// this is async because it sends the current state over the progress channel
709+
if let Err(err) = self
710+
.progress_tracker
711+
.subscribe(kind, on_progress.clone())
712+
.await
713+
{
714+
debug!(?err, %kind, "failed to subscribe progress sender to transfer");
715+
}
695716
}
717+
entry.get_mut().intents.insert(intent_id, intent_handlers);
696718
}
697-
} else {
698-
// the transfer is not running.
699-
if updated && self.queue.is_parked(&kind) {
700-
// the transfer is on hold for pending retries, and we added new nodes, so move back to queue.
701-
self.queue.unpark(&kind);
702-
} else if !self.queue.contains(&kind) {
703-
// the transfer is not yet queued: add to queue.
719+
hash_map::Entry::Vacant(entry) => {
720+
tracing::warn!("is new, queue");
721+
let progress_sender = self.progress_tracker.track(
722+
kind,
723+
intent_handlers
724+
.on_progress
725+
.clone()
726+
.into_iter()
727+
.collect::<Vec<_>>(),
728+
);
729+
730+
let get_state = match self.getter.get(kind, progress_sender.clone()).await {
731+
Err(_err) => {
732+
self.finalize_download(
733+
kind,
734+
[(intent_id, intent_handlers)].into(),
735+
// TODO: add better error variant? this is only triggered if the local
736+
// store failed with local IO.
737+
Err(DownloadError::DownloadFailed),
738+
);
739+
return;
740+
}
741+
Ok(GetOutput::Complete(stats)) => {
742+
self.finalize_download(
743+
kind,
744+
[(intent_id, intent_handlers)].into(),
745+
Ok(stats),
746+
);
747+
return;
748+
}
749+
Ok(GetOutput::NeedsConn(state)) => {
750+
// early exit if no providers.
751+
if self.providers.get_candidates(&kind.hash()).next().is_none() {
752+
self.finalize_download(
753+
kind,
754+
[(intent_id, intent_handlers)].into(),
755+
Err(DownloadError::NoProviders),
756+
);
757+
return;
758+
}
759+
state
760+
}
761+
};
762+
entry.insert(RequestInfo {
763+
intents: [(intent_id, intent_handlers)].into_iter().collect(),
764+
progress_sender,
765+
get_state: Some(get_state),
766+
});
704767
self.queue.insert(kind);
705768
}
706769
}
707770

708-
// store the request info
709-
let request_info = self.requests.entry(kind).or_default();
710-
request_info.intents.insert(intent_id, intent_handlers);
771+
if updated && self.queue.is_parked(&kind) {
772+
// the transfer is on hold for pending retries, and we added new nodes, so move back to queue.
773+
self.queue.unpark(&kind);
774+
}
711775
}
712776

713777
/// Cancels a download intent.
@@ -860,7 +924,6 @@ impl<G: Getter<Connection = D::Connection>, D: Dialer> Service<G, D> {
860924
) {
861925
self.progress_tracker.remove(&kind);
862926
self.remove_hash_if_not_queued(&kind.hash());
863-
let result = result.map_err(|_| DownloadError::DownloadFailed);
864927
for (_id, handlers) in intents.into_iter() {
865928
handlers.on_finish.send(result.clone()).ok();
866929
}
@@ -1082,14 +1145,9 @@ impl<G: Getter<Connection = D::Connection>, D: Dialer> Service<G, D> {
10821145
/// Panics if hash is not in self.requests or node is not in self.nodes.
10831146
fn start_download(&mut self, kind: DownloadKind, node: NodeId) {
10841147
let node_info = self.connected_nodes.get_mut(&node).expect("node exists");
1085-
let request_info = self.requests.get(&kind).expect("hash exists");
1086-
1087-
// create a progress sender and subscribe all intents to the progress sender
1088-
let subscribers = request_info
1089-
.intents
1090-
.values()
1091-
.flat_map(|state| state.on_progress.clone());
1092-
let progress_sender = self.progress_tracker.track(kind, subscribers);
1148+
let request_info = self.requests.get_mut(&kind).expect("request exists");
1149+
let progress = request_info.progress_sender.clone();
1150+
// .expect("queued state exists");
10931151

10941152
// create the active request state
10951153
let cancellation = CancellationToken::new();
@@ -1098,17 +1156,32 @@ impl<G: Getter<Connection = D::Connection>, D: Dialer> Service<G, D> {
10981156
node,
10991157
};
11001158
let conn = node_info.conn.clone();
1101-
let get_fut = self.getter.get(kind, conn, progress_sender);
1159+
1160+
// If this is the first provider node we try, we have an initial state
1161+
// from starting the generator in Self::handle_queue_new_download.
1162+
// If this not the first provider node we try, we have to recreate the generator, because
1163+
// we can only resume it once.
1164+
let get_state = match request_info.get_state.take() {
1165+
Some(state) => Either::Left(async move { Ok(GetOutput::NeedsConn(state)) }),
1166+
None => Either::Right(self.getter.get(kind, progress)),
1167+
};
11021168
let fut = async move {
11031169
// NOTE: it's an open question if we should do timeouts at this point. Considerations from @Frando:
11041170
// > at this stage we do not know the size of the download, so the timeout would have
11051171
// > to be so large that it won't be useful for non-huge downloads. At the same time,
11061172
// > this means that a super slow node would block a download from succeeding for a long
11071173
// > time, while faster nodes could be readily available.
11081174
// As a conclusion, timeouts should be added only after downloads are known to be bounded
1175+
let fut = async move {
1176+
match get_state.await? {
1177+
GetOutput::Complete(stats) => Ok(stats),
1178+
GetOutput::NeedsConn(state) => state.proceed(conn).await,
1179+
}
1180+
};
1181+
tokio::pin!(fut);
11091182
let res = tokio::select! {
11101183
_ = cancellation.cancelled() => Err(FailureAction::AllIntentsDropped),
1111-
res = get_fut => res
1184+
res = &mut fut => res
11121185
};
11131186
trace!("transfer finished");
11141187

@@ -1433,4 +1506,8 @@ impl Dialer for iroh_net::dialer::Dialer {
14331506
fn is_pending(&self, node: NodeId) -> bool {
14341507
self.is_pending(node)
14351508
}
1509+
1510+
fn node_id(&self) -> NodeId {
1511+
self.endpoint().node_id()
1512+
}
14361513
}

0 commit comments

Comments
 (0)