iroh_blobs/api/
downloader.rs

1//! API for downloads from multiple nodes.
2use std::{
3    collections::{HashMap, HashSet},
4    fmt::Debug,
5    future::{Future, IntoFuture},
6    sync::Arc,
7};
8
9use anyhow::bail;
10use genawaiter::sync::Gen;
11use iroh::{Endpoint, EndpointId};
12use irpc::{channel::mpsc, rpc_requests};
13use n0_future::{future, stream, task::JoinSet, BufferedStreamExt, Stream, StreamExt};
14use rand::seq::SliceRandom;
15use serde::{de::Error, Deserialize, Serialize};
16use tracing::instrument::Instrument;
17
18use super::Store;
19use crate::{
20    protocol::{GetManyRequest, GetRequest},
21    util::{
22        connection_pool::ConnectionPool,
23        sink::{Drain, IrpcSenderRefSink, Sink, TokioMpscSenderSink},
24    },
25    BlobFormat, Hash, HashAndFormat,
26};
27
28#[derive(Debug, Clone)]
29pub struct Downloader {
30    client: irpc::Client<SwarmProtocol>,
31}
32
33#[rpc_requests(message = SwarmMsg, alias = "Msg", rpc_feature = "rpc")]
34#[derive(Debug, Serialize, Deserialize)]
35enum SwarmProtocol {
36    #[rpc(tx = mpsc::Sender<DownloadProgressItem>)]
37    Download(DownloadRequest),
38}
39
40struct DownloaderActor {
41    store: Store,
42    pool: ConnectionPool,
43    tasks: JoinSet<()>,
44    running: HashSet<n0_future::task::Id>,
45}
46
47#[derive(Debug, Serialize, Deserialize)]
48pub enum DownloadProgressItem {
49    #[serde(skip)]
50    Error(anyhow::Error),
51    TryProvider {
52        id: EndpointId,
53        request: Arc<GetRequest>,
54    },
55    ProviderFailed {
56        id: EndpointId,
57        request: Arc<GetRequest>,
58    },
59    PartComplete {
60        request: Arc<GetRequest>,
61    },
62    Progress(u64),
63    DownloadError,
64}
65
66impl DownloaderActor {
67    fn new(store: Store, endpoint: Endpoint) -> Self {
68        Self {
69            store,
70            pool: ConnectionPool::new(endpoint, crate::ALPN, Default::default()),
71            tasks: JoinSet::new(),
72            running: HashSet::new(),
73        }
74    }
75
76    async fn run(mut self, mut rx: tokio::sync::mpsc::Receiver<SwarmMsg>) {
77        while let Some(msg) = rx.recv().await {
78            match msg {
79                SwarmMsg::Download(request) => {
80                    self.spawn(handle_download(
81                        self.store.clone(),
82                        self.pool.clone(),
83                        request,
84                    ));
85                }
86            }
87        }
88    }
89
90    fn spawn(&mut self, fut: impl Future<Output = ()> + Send + 'static) {
91        let span = tracing::Span::current();
92        let id = self.tasks.spawn(fut.instrument(span)).id();
93        self.running.insert(id);
94    }
95}
96
97async fn handle_download(store: Store, pool: ConnectionPool, msg: DownloadMsg) {
98    let DownloadMsg { inner, mut tx, .. } = msg;
99    if let Err(cause) = handle_download_impl(store, pool, inner, &mut tx).await {
100        tx.send(DownloadProgressItem::Error(cause)).await.ok();
101    }
102}
103
104async fn handle_download_impl(
105    store: Store,
106    pool: ConnectionPool,
107    request: DownloadRequest,
108    tx: &mut mpsc::Sender<DownloadProgressItem>,
109) -> anyhow::Result<()> {
110    match request.strategy {
111        SplitStrategy::Split => handle_download_split_impl(store, pool, request, tx).await?,
112        SplitStrategy::None => match request.request {
113            FiniteRequest::Get(get) => {
114                let sink = IrpcSenderRefSink(tx);
115                execute_get(&pool, Arc::new(get), &request.providers, &store, sink).await?;
116            }
117            FiniteRequest::GetMany(_) => {
118                handle_download_split_impl(store, pool, request, tx).await?
119            }
120        },
121    }
122    Ok(())
123}
124
125async fn handle_download_split_impl(
126    store: Store,
127    pool: ConnectionPool,
128    request: DownloadRequest,
129    tx: &mut mpsc::Sender<DownloadProgressItem>,
130) -> anyhow::Result<()> {
131    let providers = request.providers;
132    let requests = split_request(&request.request, &providers, &pool, &store, Drain).await?;
133    let (progress_tx, progress_rx) = tokio::sync::mpsc::channel(32);
134    let mut futs = stream::iter(requests.into_iter().enumerate())
135        .map(|(id, request)| {
136            let pool = pool.clone();
137            let providers = providers.clone();
138            let store = store.clone();
139            let progress_tx = progress_tx.clone();
140            async move {
141                let hash = request.hash;
142                let (tx, rx) = tokio::sync::mpsc::channel::<(usize, DownloadProgressItem)>(16);
143                progress_tx.send(rx).await.ok();
144                let sink = TokioMpscSenderSink(tx).with_map(move |x| (id, x));
145                let res = execute_get(&pool, Arc::new(request), &providers, &store, sink).await;
146                (hash, res)
147            }
148        })
149        .buffered_unordered(32);
150    let mut progress_stream = {
151        let mut offsets = HashMap::new();
152        let mut total = 0;
153        into_stream(progress_rx)
154            .flat_map(into_stream)
155            .map(move |(id, item)| match item {
156                DownloadProgressItem::Progress(offset) => {
157                    total += offset;
158                    if let Some(prev) = offsets.insert(id, offset) {
159                        total -= prev;
160                    }
161                    DownloadProgressItem::Progress(total)
162                }
163                x => x,
164            })
165    };
166    loop {
167        tokio::select! {
168            Some(item) = progress_stream.next() => {
169                tx.send(item).await?;
170            },
171            res = futs.next() => {
172                match res {
173                    Some((_hash, Ok(()))) => {
174                    }
175                    Some((_hash, Err(_e))) => {
176                        tx.send(DownloadProgressItem::DownloadError).await?;
177                    }
178                    None => break,
179                }
180            }
181            _ = tx.closed() => {
182                // The sender has been closed, we should stop processing.
183                break;
184            }
185        }
186    }
187    Ok(())
188}
189
190fn into_stream<T>(mut recv: tokio::sync::mpsc::Receiver<T>) -> impl Stream<Item = T> {
191    Gen::new(|co| async move {
192        while let Some(item) = recv.recv().await {
193            co.yield_(item).await;
194        }
195    })
196}
197
198#[derive(Debug, Serialize, Deserialize, derive_more::From)]
199pub enum FiniteRequest {
200    Get(GetRequest),
201    GetMany(GetManyRequest),
202}
203
204pub trait SupportedRequest {
205    fn into_request(self) -> FiniteRequest;
206}
207
208impl<I: Into<Hash>, T: IntoIterator<Item = I>> SupportedRequest for T {
209    fn into_request(self) -> FiniteRequest {
210        let hashes = self.into_iter().map(Into::into).collect::<GetManyRequest>();
211        FiniteRequest::GetMany(hashes)
212    }
213}
214
215impl SupportedRequest for GetRequest {
216    fn into_request(self) -> FiniteRequest {
217        self.into()
218    }
219}
220
221impl SupportedRequest for GetManyRequest {
222    fn into_request(self) -> FiniteRequest {
223        self.into()
224    }
225}
226
227impl SupportedRequest for Hash {
228    fn into_request(self) -> FiniteRequest {
229        GetRequest::blob(self).into()
230    }
231}
232
233impl SupportedRequest for HashAndFormat {
234    fn into_request(self) -> FiniteRequest {
235        (match self.format {
236            BlobFormat::Raw => GetRequest::blob(self.hash),
237            BlobFormat::HashSeq => GetRequest::all(self.hash),
238        })
239        .into()
240    }
241}
242
243#[derive(Debug, Serialize, Deserialize)]
244pub struct AddProviderRequest {
245    pub hash: Hash,
246    pub providers: Vec<EndpointId>,
247}
248
249#[derive(Debug)]
250pub struct DownloadRequest {
251    pub request: FiniteRequest,
252    pub providers: Arc<dyn ContentDiscovery>,
253    pub strategy: SplitStrategy,
254}
255
256impl DownloadRequest {
257    pub fn new(
258        request: impl SupportedRequest,
259        providers: impl ContentDiscovery,
260        strategy: SplitStrategy,
261    ) -> Self {
262        Self {
263            request: request.into_request(),
264            providers: Arc::new(providers),
265            strategy,
266        }
267    }
268}
269
270#[derive(Debug, Serialize, Deserialize)]
271pub enum SplitStrategy {
272    None,
273    Split,
274}
275
276impl Serialize for DownloadRequest {
277    fn serialize<S>(&self, _serializer: S) -> Result<S::Ok, S::Error>
278    where
279        S: serde::Serializer,
280    {
281        Err(serde::ser::Error::custom(
282            "cannot serialize DownloadRequest",
283        ))
284    }
285}
286
287// Implement Deserialize to always fail
288impl<'de> Deserialize<'de> for DownloadRequest {
289    fn deserialize<D>(_deserializer: D) -> Result<Self, D::Error>
290    where
291        D: serde::Deserializer<'de>,
292    {
293        Err(D::Error::custom("cannot deserialize DownloadRequest"))
294    }
295}
296
297pub type DownloadOptions = DownloadRequest;
298
299pub struct DownloadProgress {
300    fut: future::Boxed<irpc::Result<mpsc::Receiver<DownloadProgressItem>>>,
301}
302
303impl DownloadProgress {
304    fn new(fut: future::Boxed<irpc::Result<mpsc::Receiver<DownloadProgressItem>>>) -> Self {
305        Self { fut }
306    }
307
308    pub async fn stream(self) -> irpc::Result<impl Stream<Item = DownloadProgressItem> + Unpin> {
309        let rx = self.fut.await?;
310        Ok(Box::pin(rx.into_stream().map(|item| match item {
311            Ok(item) => item,
312            Err(e) => DownloadProgressItem::Error(e.into()),
313        })))
314    }
315
316    async fn complete(self) -> anyhow::Result<()> {
317        let rx = self.fut.await?;
318        let stream = rx.into_stream();
319        tokio::pin!(stream);
320        while let Some(item) = stream.next().await {
321            match item? {
322                DownloadProgressItem::Error(e) => Err(e)?,
323                DownloadProgressItem::DownloadError => anyhow::bail!("Download error"),
324                _ => {}
325            }
326        }
327        Ok(())
328    }
329}
330
331impl IntoFuture for DownloadProgress {
332    type Output = anyhow::Result<()>;
333    type IntoFuture = future::Boxed<Self::Output>;
334
335    fn into_future(self) -> Self::IntoFuture {
336        Box::pin(self.complete())
337    }
338}
339
340impl Downloader {
341    pub fn new(store: &Store, endpoint: &Endpoint) -> Self {
342        let (tx, rx) = tokio::sync::mpsc::channel::<SwarmMsg>(32);
343        let actor = DownloaderActor::new(store.clone(), endpoint.clone());
344        n0_future::task::spawn(actor.run(rx));
345        Self { client: tx.into() }
346    }
347
348    pub fn download(
349        &self,
350        request: impl SupportedRequest,
351        providers: impl ContentDiscovery,
352    ) -> DownloadProgress {
353        let request = request.into_request();
354        let providers = Arc::new(providers);
355        self.download_with_opts(DownloadOptions {
356            request,
357            providers,
358            strategy: SplitStrategy::None,
359        })
360    }
361
362    pub fn download_with_opts(&self, options: DownloadOptions) -> DownloadProgress {
363        let fut = self.client.server_streaming(options, 32);
364        DownloadProgress::new(Box::pin(fut))
365    }
366}
367
368/// Split a request into multiple requests that can be run in parallel.
369async fn split_request<'a>(
370    request: &'a FiniteRequest,
371    providers: &Arc<dyn ContentDiscovery>,
372    pool: &ConnectionPool,
373    store: &Store,
374    progress: impl Sink<DownloadProgressItem, Error = irpc::channel::SendError>,
375) -> anyhow::Result<Box<dyn Iterator<Item = GetRequest> + Send + 'a>> {
376    Ok(match request {
377        FiniteRequest::Get(req) => {
378            let Some(_first) = req.ranges.iter_infinite().next() else {
379                return Ok(Box::new(std::iter::empty()));
380            };
381            let first = GetRequest::blob(req.hash);
382            execute_get(pool, Arc::new(first), providers, store, progress).await?;
383            let size = store.observe(req.hash).await?.size();
384            anyhow::ensure!(size % 32 == 0, "Size is not a multiple of 32");
385            let n = size / 32;
386            Box::new(
387                req.ranges
388                    .iter_infinite()
389                    .take(n as usize + 1)
390                    .enumerate()
391                    .filter_map(|(i, ranges)| {
392                        if i != 0 && !ranges.is_empty() {
393                            Some(
394                                GetRequest::builder()
395                                    .offset(i as u64, ranges.clone())
396                                    .build(req.hash),
397                            )
398                        } else {
399                            None
400                        }
401                    }),
402            )
403        }
404        FiniteRequest::GetMany(req) => Box::new(
405            req.hashes
406                .iter()
407                .enumerate()
408                .map(|(i, hash)| GetRequest::blob_ranges(*hash, req.ranges[i as u64].clone())),
409        ),
410    })
411}
412
413/// Execute a get request sequentially for multiple providers.
414///
415/// It will try each provider in order
416/// until it finds one that can fulfill the request. When trying a new provider,
417/// it takes the progress from the previous providers into account, so e.g.
418/// if the first provider had the first 10% of the data, it will only ask the next
419/// provider for the remaining 90%.
420///
421/// This is fully sequential, so there will only be one request in flight at a time.
422///
423/// If the request is not complete after trying all providers, it will return an error.
424/// If the provider stream never ends, it will try indefinitely.
425async fn execute_get(
426    pool: &ConnectionPool,
427    request: Arc<GetRequest>,
428    providers: &Arc<dyn ContentDiscovery>,
429    store: &Store,
430    mut progress: impl Sink<DownloadProgressItem, Error = irpc::channel::SendError>,
431) -> anyhow::Result<()> {
432    let remote = store.remote();
433    let mut providers = providers.find_providers(request.content());
434    while let Some(provider) = providers.next().await {
435        progress
436            .send(DownloadProgressItem::TryProvider {
437                id: provider,
438                request: request.clone(),
439            })
440            .await?;
441        let conn = pool.get_or_connect(provider);
442        let local = remote.local_for_request(request.clone()).await?;
443        if local.is_complete() {
444            return Ok(());
445        }
446        let local_bytes = local.local_bytes();
447        let Ok(conn) = conn.await else {
448            progress
449                .send(DownloadProgressItem::ProviderFailed {
450                    id: provider,
451                    request: request.clone(),
452                })
453                .await?;
454            continue;
455        };
456        match remote
457            .execute_get_sink(
458                conn.clone(),
459                local.missing(),
460                (&mut progress).with_map(move |x| DownloadProgressItem::Progress(x + local_bytes)),
461            )
462            .await
463        {
464            Ok(_stats) => {
465                progress
466                    .send(DownloadProgressItem::PartComplete {
467                        request: request.clone(),
468                    })
469                    .await?;
470                return Ok(());
471            }
472            Err(_cause) => {
473                progress
474                    .send(DownloadProgressItem::ProviderFailed {
475                        id: provider,
476                        request: request.clone(),
477                    })
478                    .await?;
479                continue;
480            }
481        }
482    }
483    bail!("Unable to download {}", request.hash);
484}
485
486/// Trait for pluggable content discovery strategies.
487pub trait ContentDiscovery: Debug + Send + Sync + 'static {
488    fn find_providers(&self, hash: HashAndFormat) -> n0_future::stream::Boxed<EndpointId>;
489}
490
491impl<C, I> ContentDiscovery for C
492where
493    C: Debug + Clone + IntoIterator<Item = I> + Send + Sync + 'static,
494    C::IntoIter: Send + Sync + 'static,
495    I: Into<EndpointId> + Send + Sync + 'static,
496{
497    fn find_providers(&self, _: HashAndFormat) -> n0_future::stream::Boxed<EndpointId> {
498        let providers = self.clone();
499        n0_future::stream::iter(providers.into_iter().map(Into::into)).boxed()
500    }
501}
502
503#[derive(derive_more::Debug)]
504pub struct Shuffled {
505    nodes: Vec<EndpointId>,
506}
507
508impl Shuffled {
509    pub fn new(nodes: Vec<EndpointId>) -> Self {
510        Self { nodes }
511    }
512}
513
514impl ContentDiscovery for Shuffled {
515    fn find_providers(&self, _: HashAndFormat) -> n0_future::stream::Boxed<EndpointId> {
516        let mut nodes = self.nodes.clone();
517        nodes.shuffle(&mut rand::rng());
518        n0_future::stream::iter(nodes).boxed()
519    }
520}
521
522#[cfg(test)]
523#[cfg(feature = "fs-store")]
524mod tests {
525    use std::ops::Deref;
526
527    use bao_tree::ChunkRanges;
528    use n0_future::StreamExt;
529    use testresult::TestResult;
530
531    use crate::{
532        api::{
533            blobs::AddBytesOptions,
534            downloader::{DownloadOptions, Downloader, Shuffled, SplitStrategy},
535        },
536        hashseq::HashSeq,
537        protocol::{GetManyRequest, GetRequest},
538        tests::node_test_setup_fs,
539    };
540
541    #[tokio::test]
542    #[ignore = "todo"]
543    async fn downloader_get_many_smoke() -> TestResult<()> {
544        let testdir = tempfile::tempdir()?;
545        let (r1, store1, _, _) = node_test_setup_fs(testdir.path().join("a")).await?;
546        let (r2, store2, _, _) = node_test_setup_fs(testdir.path().join("b")).await?;
547        let (r3, store3, _, sp3) = node_test_setup_fs(testdir.path().join("c")).await?;
548        let tt1 = store1.add_slice("hello world").await?;
549        let tt2 = store2.add_slice("hello world 2").await?;
550        let node1_addr = r1.endpoint().addr();
551        let node1_id = node1_addr.id;
552        let node2_addr = r2.endpoint().addr();
553        let node2_id = node2_addr.id;
554        let swarm = Downloader::new(&store3, r3.endpoint());
555        sp3.add_endpoint_info(node1_addr.clone());
556        sp3.add_endpoint_info(node2_addr.clone());
557        let request = GetManyRequest::builder()
558            .hash(tt1.hash, ChunkRanges::all())
559            .hash(tt2.hash, ChunkRanges::all())
560            .build();
561        let mut progress = swarm
562            .download(request, Shuffled::new(vec![node1_id, node2_id]))
563            .stream()
564            .await?;
565        while progress.next().await.is_some() {}
566        assert_eq!(store3.get_bytes(tt1.hash).await?.deref(), b"hello world");
567        assert_eq!(store3.get_bytes(tt2.hash).await?.deref(), b"hello world 2");
568        Ok(())
569    }
570
571    #[tokio::test]
572    async fn downloader_get_smoke() -> TestResult<()> {
573        // tracing_subscriber::fmt::try_init().ok();
574        let testdir = tempfile::tempdir()?;
575        let (r1, store1, _, _) = node_test_setup_fs(testdir.path().join("a")).await?;
576        let (r2, store2, _, _) = node_test_setup_fs(testdir.path().join("b")).await?;
577        let (r3, store3, _, sp3) = node_test_setup_fs(testdir.path().join("c")).await?;
578        let tt1 = store1.add_slice(vec![1; 10000000]).await?;
579        let tt2 = store2.add_slice(vec![2; 10000000]).await?;
580        let hs = [tt1.hash, tt2.hash].into_iter().collect::<HashSeq>();
581        let root = store1
582            .add_bytes_with_opts(AddBytesOptions {
583                data: hs.clone().into(),
584                format: crate::BlobFormat::HashSeq,
585            })
586            .await?;
587        let node1_addr = r1.endpoint().addr();
588        let node1_id = node1_addr.id;
589        let node2_addr = r2.endpoint().addr();
590        let node2_id = node2_addr.id;
591        let swarm = Downloader::new(&store3, r3.endpoint());
592        sp3.add_endpoint_info(node1_addr.clone());
593        sp3.add_endpoint_info(node2_addr.clone());
594        let request = GetRequest::builder()
595            .root(ChunkRanges::all())
596            .next(ChunkRanges::all())
597            .next(ChunkRanges::all())
598            .build(root.hash);
599        if true {
600            let mut progress = swarm
601                .download_with_opts(DownloadOptions::new(
602                    request,
603                    [node1_id, node2_id],
604                    SplitStrategy::Split,
605                ))
606                .stream()
607                .await?;
608            while progress.next().await.is_some() {}
609        }
610        if false {
611            let conn = r3.endpoint().connect(node1_addr, crate::ALPN).await?;
612            let remote = store3.remote();
613            let _rh = remote
614                .execute_get(
615                    conn.clone(),
616                    GetRequest::builder()
617                        .root(ChunkRanges::all())
618                        .build(root.hash),
619                )
620                .await?;
621            let h1 = remote.execute_get(
622                conn.clone(),
623                GetRequest::builder()
624                    .child(0, ChunkRanges::all())
625                    .build(root.hash),
626            );
627            let h2 = remote.execute_get(
628                conn.clone(),
629                GetRequest::builder()
630                    .child(1, ChunkRanges::all())
631                    .build(root.hash),
632            );
633            h1.await?;
634            h2.await?;
635        }
636        Ok(())
637    }
638
639    #[tokio::test]
640    async fn downloader_get_all() -> TestResult<()> {
641        let testdir = tempfile::tempdir()?;
642        let (r1, store1, _, _) = node_test_setup_fs(testdir.path().join("a")).await?;
643        let (r2, store2, _, _) = node_test_setup_fs(testdir.path().join("b")).await?;
644        let (r3, store3, _, sp3) = node_test_setup_fs(testdir.path().join("c")).await?;
645        let tt1 = store1.add_slice(vec![1; 10000000]).await?;
646        let tt2 = store2.add_slice(vec![2; 10000000]).await?;
647        let hs = [tt1.hash, tt2.hash].into_iter().collect::<HashSeq>();
648        let root = store1
649            .add_bytes_with_opts(AddBytesOptions {
650                data: hs.clone().into(),
651                format: crate::BlobFormat::HashSeq,
652            })
653            .await?;
654        let node1_addr = r1.endpoint().addr();
655        let node1_id = node1_addr.id;
656        let node2_addr = r2.endpoint().addr();
657        let node2_id = node2_addr.id;
658        let swarm = Downloader::new(&store3, r3.endpoint());
659        sp3.add_endpoint_info(node1_addr.clone());
660        sp3.add_endpoint_info(node2_addr.clone());
661        let request = GetRequest::all(root.hash);
662        let mut progress = swarm
663            .download_with_opts(DownloadOptions::new(
664                request,
665                [node1_id, node2_id],
666                SplitStrategy::Split,
667            ))
668            .stream()
669            .await?;
670        while progress.next().await.is_some() {}
671        Ok(())
672    }
673}