Skip to content

Commit 9ae2e52

Browse files
authoredDec 6, 2024··
fix: Fix the task leak with the lazy in-mem rpc client while still keeping it lazy (#31)
* Make blobs more cheaply cloneable by by giving it an Inner * Remove the lazy part. The lazy handler kept a reference to Blobs alive. This caused both the task and the blobs to never be dropped. To solve this you can just split the inner part in 2 parts, one that has the handle and one that has the logic. But that is not nice. I think it is best for the mem rpc handler to exist completely separately, especially given that rpc is a non-default feature. * spawn_rpc should make it sufficiently clear that this is a thing you need to put away somewhere. Or maybe spawn_client? * back to the lazy client * add comment about the purpose of the handler
1 parent dba7850 commit 9ae2e52

File tree

5 files changed

+276
-201
lines changed

5 files changed

+276
-201
lines changed
 

‎examples/custom-protocol.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -122,7 +122,7 @@ async fn main() -> Result<()> {
122122

123123
// Print out our query results.
124124
for hash in hashes {
125-
read_and_print(&blobs_client, hash).await?;
125+
read_and_print(blobs_client, hash).await?;
126126
}
127127
}
128128
}

‎src/net_protocol.rs

Lines changed: 47 additions & 179 deletions
Original file line numberDiff line numberDiff line change
@@ -5,28 +5,22 @@
55

66
use std::{collections::BTreeSet, fmt::Debug, ops::DerefMut, sync::Arc};
77

8-
use anyhow::{anyhow, bail, Result};
8+
use anyhow::{bail, Result};
99
use futures_lite::future::Boxed as BoxedFuture;
1010
use futures_util::future::BoxFuture;
1111
use iroh::{endpoint::Connecting, protocol::ProtocolHandler, Endpoint, NodeAddr};
1212
use iroh_base::hash::{BlobFormat, Hash};
1313
use serde::{Deserialize, Serialize};
14-
use tracing::{debug, warn};
14+
use tracing::debug;
1515

1616
use crate::{
17-
downloader::{DownloadRequest, Downloader},
18-
get::{
19-
db::{DownloadProgress, GetState},
20-
Stats,
21-
},
17+
downloader::Downloader,
2218
provider::EventSender,
2319
store::GcConfig,
2420
util::{
2521
local_pool::{self, LocalPoolHandle},
26-
progress::{AsyncChannelProgressSender, ProgressSender},
2722
SetTagOption,
2823
},
29-
HashAndFormat,
3024
};
3125

3226
/// A callback that blobs can ask about a set of hashes that should not be garbage collected.
@@ -47,16 +41,21 @@ impl Default for GcState {
4741
}
4842
}
4943

50-
#[derive(Debug, Clone)]
51-
pub struct Blobs<S> {
52-
rt: LocalPoolHandle,
44+
#[derive(Debug)]
45+
pub(crate) struct BlobsInner<S> {
46+
pub(crate) rt: LocalPoolHandle,
5347
pub(crate) store: S,
5448
events: EventSender,
55-
downloader: Downloader,
49+
pub(crate) downloader: Downloader,
50+
pub(crate) endpoint: Endpoint,
51+
gc_state: std::sync::Mutex<GcState>,
5652
#[cfg(feature = "rpc")]
57-
batches: Arc<tokio::sync::Mutex<BlobBatches>>,
58-
endpoint: Endpoint,
59-
gc_state: Arc<std::sync::Mutex<GcState>>,
53+
pub(crate) batches: tokio::sync::Mutex<BlobBatches>,
54+
}
55+
56+
#[derive(Debug, Clone)]
57+
pub struct Blobs<S> {
58+
pub(crate) inner: Arc<BlobsInner<S>>,
6059
#[cfg(feature = "rpc")]
6160
pub(crate) rpc_handler: Arc<std::sync::OnceLock<crate::rpc::RpcHandler>>,
6261
}
@@ -76,7 +75,7 @@ pub(crate) struct BlobBatches {
7675
#[derive(Debug, Default)]
7776
struct BlobBatch {
7877
/// The tags in this batch.
79-
tags: std::collections::BTreeMap<HashAndFormat, Vec<crate::TempTag>>,
78+
tags: std::collections::BTreeMap<iroh::hash::HashAndFormat, Vec<crate::TempTag>>,
8079
}
8180

8281
#[cfg(feature = "rpc")]
@@ -95,7 +94,11 @@ impl BlobBatches {
9594
}
9695

9796
/// Remove a tag from a batch.
98-
pub fn remove_one(&mut self, batch: BatchId, content: &HashAndFormat) -> Result<()> {
97+
pub fn remove_one(
98+
&mut self,
99+
batch: BatchId,
100+
content: &iroh::hash::HashAndFormat,
101+
) -> Result<()> {
99102
if let Some(batch) = self.batches.get_mut(&batch) {
100103
if let Some(tags) = batch.tags.get_mut(content) {
101104
tags.pop();
@@ -178,40 +181,46 @@ impl<S: crate::store::Store> Blobs<S> {
178181
endpoint: Endpoint,
179182
) -> Self {
180183
Self {
181-
rt,
182-
store,
183-
events,
184-
downloader,
185-
endpoint,
186-
#[cfg(feature = "rpc")]
187-
batches: Default::default(),
188-
gc_state: Default::default(),
184+
inner: Arc::new(BlobsInner {
185+
rt,
186+
store,
187+
events,
188+
downloader,
189+
endpoint,
190+
#[cfg(feature = "rpc")]
191+
batches: Default::default(),
192+
gc_state: Default::default(),
193+
}),
189194
#[cfg(feature = "rpc")]
190195
rpc_handler: Default::default(),
191196
}
192197
}
193198

194199
pub fn store(&self) -> &S {
195-
&self.store
200+
&self.inner.store
201+
}
202+
203+
pub fn events(&self) -> &EventSender {
204+
&self.inner.events
196205
}
197206

198207
pub fn rt(&self) -> &LocalPoolHandle {
199-
&self.rt
208+
&self.inner.rt
200209
}
201210

202211
pub fn downloader(&self) -> &Downloader {
203-
&self.downloader
212+
&self.inner.downloader
204213
}
205214

206215
pub fn endpoint(&self) -> &Endpoint {
207-
&self.endpoint
216+
&self.inner.endpoint
208217
}
209218

210219
/// Add a callback that will be called before the garbage collector runs.
211220
///
212221
/// This can only be called before the garbage collector has started, otherwise it will return an error.
213222
pub fn add_protected(&self, cb: ProtectCb) -> Result<()> {
214-
let mut state = self.gc_state.lock().unwrap();
223+
let mut state = self.inner.gc_state.lock().unwrap();
215224
match &mut *state {
216225
GcState::Initial(cbs) => {
217226
cbs.push(cb);
@@ -225,7 +234,7 @@ impl<S: crate::store::Store> Blobs<S> {
225234

226235
/// Start garbage collection with the given settings.
227236
pub fn start_gc(&self, config: GcConfig) -> Result<()> {
228-
let mut state = self.gc_state.lock().unwrap();
237+
let mut state = self.inner.gc_state.lock().unwrap();
229238
let protected = match state.deref_mut() {
230239
GcState::Initial(items) => std::mem::take(items),
231240
GcState::Started(_) => bail!("gc already started"),
@@ -241,161 +250,20 @@ impl<S: crate::store::Store> Blobs<S> {
241250
set
242251
}
243252
};
244-
let store = self.store.clone();
253+
let store = self.store().clone();
245254
let run = self
246-
.rt
255+
.rt()
247256
.spawn(move || async move { store.gc_run(config, protected_cb).await });
248257
*state = GcState::Started(Some(run));
249258
Ok(())
250259
}
251-
252-
#[cfg(feature = "rpc")]
253-
pub(crate) async fn batches(&self) -> tokio::sync::MutexGuard<'_, BlobBatches> {
254-
self.batches.lock().await
255-
}
256-
257-
pub(crate) async fn download(
258-
&self,
259-
endpoint: Endpoint,
260-
req: BlobDownloadRequest,
261-
progress: AsyncChannelProgressSender<DownloadProgress>,
262-
) -> Result<()> {
263-
let BlobDownloadRequest {
264-
hash,
265-
format,
266-
nodes,
267-
tag,
268-
mode,
269-
} = req;
270-
let hash_and_format = HashAndFormat { hash, format };
271-
let temp_tag = self.store.temp_tag(hash_and_format);
272-
let stats = match mode {
273-
DownloadMode::Queued => {
274-
self.download_queued(endpoint, hash_and_format, nodes, progress.clone())
275-
.await?
276-
}
277-
DownloadMode::Direct => {
278-
self.download_direct_from_nodes(endpoint, hash_and_format, nodes, progress.clone())
279-
.await?
280-
}
281-
};
282-
283-
progress.send(DownloadProgress::AllDone(stats)).await.ok();
284-
match tag {
285-
SetTagOption::Named(tag) => {
286-
self.store.set_tag(tag, Some(hash_and_format)).await?;
287-
}
288-
SetTagOption::Auto => {
289-
self.store.create_tag(hash_and_format).await?;
290-
}
291-
}
292-
drop(temp_tag);
293-
294-
Ok(())
295-
}
296-
297-
async fn download_queued(
298-
&self,
299-
endpoint: Endpoint,
300-
hash_and_format: HashAndFormat,
301-
nodes: Vec<NodeAddr>,
302-
progress: AsyncChannelProgressSender<DownloadProgress>,
303-
) -> Result<Stats> {
304-
/// Name used for logging when new node addresses are added from gossip.
305-
const BLOB_DOWNLOAD_SOURCE_NAME: &str = "blob_download";
306-
307-
let mut node_ids = Vec::with_capacity(nodes.len());
308-
let mut any_added = false;
309-
for node in nodes {
310-
node_ids.push(node.node_id);
311-
if !node.info.is_empty() {
312-
endpoint.add_node_addr_with_source(node, BLOB_DOWNLOAD_SOURCE_NAME)?;
313-
any_added = true;
314-
}
315-
}
316-
let can_download = !node_ids.is_empty() && (any_added || endpoint.discovery().is_some());
317-
anyhow::ensure!(can_download, "no way to reach a node for download");
318-
let req = DownloadRequest::new(hash_and_format, node_ids).progress_sender(progress);
319-
let handle = self.downloader.queue(req).await;
320-
let stats = handle.await?;
321-
Ok(stats)
322-
}
323-
324-
#[tracing::instrument("download_direct", skip_all, fields(hash=%hash_and_format.hash.fmt_short()))]
325-
async fn download_direct_from_nodes(
326-
&self,
327-
endpoint: Endpoint,
328-
hash_and_format: HashAndFormat,
329-
nodes: Vec<NodeAddr>,
330-
progress: AsyncChannelProgressSender<DownloadProgress>,
331-
) -> Result<Stats> {
332-
let mut last_err = None;
333-
let mut remaining_nodes = nodes.len();
334-
let mut nodes_iter = nodes.into_iter();
335-
'outer: loop {
336-
match crate::get::db::get_to_db_in_steps(
337-
self.store.clone(),
338-
hash_and_format,
339-
progress.clone(),
340-
)
341-
.await?
342-
{
343-
GetState::Complete(stats) => return Ok(stats),
344-
GetState::NeedsConn(needs_conn) => {
345-
let (conn, node_id) = 'inner: loop {
346-
match nodes_iter.next() {
347-
None => break 'outer,
348-
Some(node) => {
349-
remaining_nodes -= 1;
350-
let node_id = node.node_id;
351-
if node_id == endpoint.node_id() {
352-
debug!(
353-
?remaining_nodes,
354-
"skip node {} (it is the node id of ourselves)",
355-
node_id.fmt_short()
356-
);
357-
continue 'inner;
358-
}
359-
match endpoint.connect(node, crate::protocol::ALPN).await {
360-
Ok(conn) => break 'inner (conn, node_id),
361-
Err(err) => {
362-
debug!(
363-
?remaining_nodes,
364-
"failed to connect to {}: {err}",
365-
node_id.fmt_short()
366-
);
367-
continue 'inner;
368-
}
369-
}
370-
}
371-
}
372-
};
373-
match needs_conn.proceed(conn).await {
374-
Ok(stats) => return Ok(stats),
375-
Err(err) => {
376-
warn!(
377-
?remaining_nodes,
378-
"failed to download from {}: {err}",
379-
node_id.fmt_short()
380-
);
381-
last_err = Some(err);
382-
}
383-
}
384-
}
385-
}
386-
}
387-
match last_err {
388-
Some(err) => Err(err.into()),
389-
None => Err(anyhow!("No nodes to download from provided")),
390-
}
391-
}
392260
}
393261

394262
impl<S: crate::store::Store> ProtocolHandler for Blobs<S> {
395263
fn accept(&self, conn: Connecting) -> BoxedFuture<Result<()>> {
396-
let db = self.store.clone();
397-
let events = self.events.clone();
398-
let rt = self.rt.clone();
264+
let db = self.store().clone();
265+
let events = self.events().clone();
266+
let rt = self.rt().clone();
399267

400268
Box::pin(async move {
401269
crate::provider::handle_connection(conn.await?, db, events, rt).await;
@@ -404,7 +272,7 @@ impl<S: crate::store::Store> ProtocolHandler for Blobs<S> {
404272
}
405273

406274
fn shutdown(&self) -> BoxedFuture<()> {
407-
let store = self.store.clone();
275+
let store = self.store().clone();
408276
Box::pin(async move {
409277
store.shutdown().await;
410278
})

‎src/rpc.rs

Lines changed: 224 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -2,19 +2,21 @@
22
33
use std::{
44
io,
5+
ops::Deref,
56
sync::{Arc, Mutex},
67
};
78

89
use anyhow::anyhow;
910
use client::{
10-
blobs::{self, BlobInfo, BlobStatus, IncompleteBlobInfo, WrapOption},
11+
blobs::{self, BlobInfo, BlobStatus, DownloadMode, IncompleteBlobInfo, MemClient, WrapOption},
1112
tags::TagInfo,
1213
MemConnector,
1314
};
1415
use futures_buffered::BufferedStreamExt;
1516
use futures_lite::StreamExt;
1617
use futures_util::{FutureExt, Stream};
1718
use genawaiter::sync::{Co, Gen};
19+
use iroh::{Endpoint, NodeAddr};
1820
use iroh_base::hash::{BlobFormat, HashAndFormat};
1921
use iroh_io::AsyncSliceReader;
2022
use proto::{
@@ -38,15 +40,21 @@ use quic_rpc::{
3840
RpcClient, RpcServer,
3941
};
4042
use tokio_util::task::AbortOnDropHandle;
43+
use tracing::{debug, warn};
4144

4245
use crate::{
46+
downloader::{DownloadRequest, Downloader},
4347
export::ExportProgress,
4448
format::collection::Collection,
45-
get::db::DownloadProgress,
46-
net_protocol::{BlobDownloadRequest, Blobs},
49+
get::{
50+
db::{DownloadProgress, GetState},
51+
Stats,
52+
},
53+
net_protocol::{BlobDownloadRequest, Blobs, BlobsInner},
4754
provider::{AddProgress, BatchAddPathProgress},
4855
store::{ConsistencyCheckProgress, ImportProgress, MapEntry, ValidateProgress},
4956
util::{
57+
local_pool::LocalPoolHandle,
5058
progress::{AsyncChannelProgressSender, ProgressSender},
5159
SetTagOption,
5260
},
@@ -62,13 +70,63 @@ const RPC_BLOB_GET_CHANNEL_CAP: usize = 2;
6270

6371
impl<D: crate::store::Store> Blobs<D> {
6472
/// Get a client for the blobs protocol
65-
pub fn client(&self) -> blobs::MemClient {
66-
let client = self
73+
pub fn client(&self) -> &blobs::MemClient {
74+
&self
6775
.rpc_handler
68-
.get_or_init(|| RpcHandler::new(self))
76+
.get_or_init(|| RpcHandler::new(&self.inner))
6977
.client
70-
.clone();
71-
blobs::Client::new(client)
78+
}
79+
80+
/// Handle an RPC request
81+
pub async fn handle_rpc_request<C>(
82+
self,
83+
msg: Request,
84+
chan: RpcChannel<RpcService, C>,
85+
) -> std::result::Result<(), RpcServerError<C>>
86+
where
87+
C: ChannelTypes<RpcService>,
88+
{
89+
Handler(self.inner.clone())
90+
.handle_rpc_request(msg, chan)
91+
.await
92+
}
93+
}
94+
95+
/// This is just an internal helper so I don't have to
96+
/// define all the rpc methods on `self: Arc<BlobsInner<S>>`
97+
#[derive(Clone)]
98+
struct Handler<S>(Arc<BlobsInner<S>>);
99+
100+
impl<S> Deref for Handler<S> {
101+
type Target = BlobsInner<S>;
102+
103+
fn deref(&self) -> &Self::Target {
104+
&self.0
105+
}
106+
}
107+
108+
impl<D: crate::store::Store> Handler<D> {
109+
fn store(&self) -> &D {
110+
&self.0.store
111+
}
112+
113+
fn rt(&self) -> &LocalPoolHandle {
114+
&self.0.rt
115+
}
116+
117+
fn endpoint(&self) -> &Endpoint {
118+
&self.0.endpoint
119+
}
120+
121+
fn downloader(&self) -> &Downloader {
122+
&self.0.downloader
123+
}
124+
125+
#[cfg(feature = "rpc")]
126+
pub(crate) async fn batches(
127+
&self,
128+
) -> tokio::sync::MutexGuard<'_, crate::net_protocol::BlobBatches> {
129+
self.0.batches.lock().await
72130
}
73131

74132
/// Handle an RPC request
@@ -872,24 +930,178 @@ impl<D: crate::store::Store> Blobs<D> {
872930

873931
Ok(CreateCollectionResponse { hash, tag })
874932
}
933+
934+
pub(crate) async fn download(
935+
&self,
936+
endpoint: Endpoint,
937+
req: BlobDownloadRequest,
938+
progress: AsyncChannelProgressSender<DownloadProgress>,
939+
) -> anyhow::Result<()> {
940+
let BlobDownloadRequest {
941+
hash,
942+
format,
943+
nodes,
944+
tag,
945+
mode,
946+
} = req;
947+
let hash_and_format = HashAndFormat { hash, format };
948+
let temp_tag = self.store().temp_tag(hash_and_format);
949+
let stats = match mode {
950+
DownloadMode::Queued => {
951+
self.download_queued(endpoint, hash_and_format, nodes, progress.clone())
952+
.await?
953+
}
954+
DownloadMode::Direct => {
955+
self.download_direct_from_nodes(endpoint, hash_and_format, nodes, progress.clone())
956+
.await?
957+
}
958+
};
959+
960+
progress.send(DownloadProgress::AllDone(stats)).await.ok();
961+
match tag {
962+
SetTagOption::Named(tag) => {
963+
self.store().set_tag(tag, Some(hash_and_format)).await?;
964+
}
965+
SetTagOption::Auto => {
966+
self.store().create_tag(hash_and_format).await?;
967+
}
968+
}
969+
drop(temp_tag);
970+
971+
Ok(())
972+
}
973+
974+
async fn download_queued(
975+
&self,
976+
endpoint: Endpoint,
977+
hash_and_format: HashAndFormat,
978+
nodes: Vec<NodeAddr>,
979+
progress: AsyncChannelProgressSender<DownloadProgress>,
980+
) -> anyhow::Result<Stats> {
981+
/// Name used for logging when new node addresses are added from gossip.
982+
const BLOB_DOWNLOAD_SOURCE_NAME: &str = "blob_download";
983+
984+
let mut node_ids = Vec::with_capacity(nodes.len());
985+
let mut any_added = false;
986+
for node in nodes {
987+
node_ids.push(node.node_id);
988+
if !node.info.is_empty() {
989+
endpoint.add_node_addr_with_source(node, BLOB_DOWNLOAD_SOURCE_NAME)?;
990+
any_added = true;
991+
}
992+
}
993+
let can_download = !node_ids.is_empty() && (any_added || endpoint.discovery().is_some());
994+
anyhow::ensure!(can_download, "no way to reach a node for download");
995+
let req = DownloadRequest::new(hash_and_format, node_ids).progress_sender(progress);
996+
let handle = self.downloader().queue(req).await;
997+
let stats = handle.await?;
998+
Ok(stats)
999+
}
1000+
1001+
#[tracing::instrument("download_direct", skip_all, fields(hash=%hash_and_format.hash.fmt_short()))]
1002+
async fn download_direct_from_nodes(
1003+
&self,
1004+
endpoint: Endpoint,
1005+
hash_and_format: HashAndFormat,
1006+
nodes: Vec<NodeAddr>,
1007+
progress: AsyncChannelProgressSender<DownloadProgress>,
1008+
) -> anyhow::Result<Stats> {
1009+
let mut last_err = None;
1010+
let mut remaining_nodes = nodes.len();
1011+
let mut nodes_iter = nodes.into_iter();
1012+
'outer: loop {
1013+
match crate::get::db::get_to_db_in_steps(
1014+
self.store().clone(),
1015+
hash_and_format,
1016+
progress.clone(),
1017+
)
1018+
.await?
1019+
{
1020+
GetState::Complete(stats) => return Ok(stats),
1021+
GetState::NeedsConn(needs_conn) => {
1022+
let (conn, node_id) = 'inner: loop {
1023+
match nodes_iter.next() {
1024+
None => break 'outer,
1025+
Some(node) => {
1026+
remaining_nodes -= 1;
1027+
let node_id = node.node_id;
1028+
if node_id == endpoint.node_id() {
1029+
debug!(
1030+
?remaining_nodes,
1031+
"skip node {} (it is the node id of ourselves)",
1032+
node_id.fmt_short()
1033+
);
1034+
continue 'inner;
1035+
}
1036+
match endpoint.connect(node, crate::protocol::ALPN).await {
1037+
Ok(conn) => break 'inner (conn, node_id),
1038+
Err(err) => {
1039+
debug!(
1040+
?remaining_nodes,
1041+
"failed to connect to {}: {err}",
1042+
node_id.fmt_short()
1043+
);
1044+
continue 'inner;
1045+
}
1046+
}
1047+
}
1048+
}
1049+
};
1050+
match needs_conn.proceed(conn).await {
1051+
Ok(stats) => return Ok(stats),
1052+
Err(err) => {
1053+
warn!(
1054+
?remaining_nodes,
1055+
"failed to download from {}: {err}",
1056+
node_id.fmt_short()
1057+
);
1058+
last_err = Some(err);
1059+
}
1060+
}
1061+
}
1062+
}
1063+
}
1064+
match last_err {
1065+
Some(err) => Err(err.into()),
1066+
None => Err(anyhow!("No nodes to download from provided")),
1067+
}
1068+
}
8751069
}
8761070

1071+
/// An in memory rpc handler for the blobs rpc protocol
1072+
///
1073+
/// This struct contains both a task that handles rpc requests and a client
1074+
/// that can be used to send rpc requests.
1075+
///
1076+
/// Dropping it will stop the handler task, so you need to put it somewhere
1077+
/// where it will be kept alive. This struct will capture a copy of
1078+
/// [`crate::net_protocol::Blobs`] and keep it alive.
8771079
#[derive(Debug)]
8781080
pub(crate) struct RpcHandler {
8791081
/// Client to hand out
880-
client: RpcClient<RpcService, MemConnector>,
1082+
client: MemClient,
8811083
/// Handler task
8821084
_handler: AbortOnDropHandle<()>,
8831085
}
8841086

1087+
impl Deref for RpcHandler {
1088+
type Target = MemClient;
1089+
1090+
fn deref(&self) -> &Self::Target {
1091+
&self.client
1092+
}
1093+
}
1094+
8851095
impl RpcHandler {
886-
fn new<D: crate::store::Store>(blobs: &Blobs<D>) -> Self {
1096+
fn new<D: crate::store::Store>(blobs: &Arc<BlobsInner<D>>) -> Self {
8871097
let blobs = blobs.clone();
8881098
let (listener, connector) = quic_rpc::transport::flume::channel(1);
8891099
let listener = RpcServer::new(listener);
8901100
let client = RpcClient::new(connector);
891-
let _handler = listener
892-
.spawn_accept_loop(move |req, chan| blobs.clone().handle_rpc_request(req, chan));
1101+
let client = MemClient::new(client);
1102+
let _handler = listener.spawn_accept_loop(move |req, chan| {
1103+
Handler(blobs.clone()).handle_rpc_request(req, chan)
1104+
});
8931105
Self { client, _handler }
8941106
}
8951107
}

‎tests/blobs.rs

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@ async fn blobs_gc_smoke() -> TestResult<()> {
1313
let pool = LocalPool::default();
1414
let endpoint = Endpoint::builder().bind().await?;
1515
let blobs = Blobs::memory().build(pool.handle(), &endpoint);
16-
let client = blobs.clone().client();
16+
let client = blobs.client();
1717
blobs.start_gc(GcConfig {
1818
period: Duration::from_millis(1),
1919
done_callback: None,
@@ -32,12 +32,7 @@ async fn blobs_gc_protected() -> TestResult<()> {
3232
let pool = LocalPool::default();
3333
let endpoint = Endpoint::builder().bind().await?;
3434
let blobs = Blobs::memory().build(pool.handle(), &endpoint);
35-
let client: iroh_blobs::rpc::client::blobs::Client<
36-
quic_rpc::transport::flume::FlumeConnector<
37-
iroh_blobs::rpc::proto::Response,
38-
iroh_blobs::rpc::proto::Request,
39-
>,
40-
> = blobs.clone().client();
35+
let client = blobs.client();
4136
let h1 = client.add_bytes(b"test".to_vec()).await?;
4237
let protected = Arc::new(Mutex::new(Vec::new()));
4338
blobs.add_protected(Box::new({

‎tests/gc.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -66,8 +66,8 @@ impl<S: Store> Node<S> {
6666
}
6767

6868
/// Returns an in-memory blobs client
69-
pub fn blobs(&self) -> blobs::MemClient {
70-
self.blobs.clone().client()
69+
pub fn blobs(&self) -> &blobs::MemClient {
70+
self.blobs.client()
7171
}
7272

7373
/// Returns an in-memory tags client

0 commit comments

Comments
 (0)
Please sign in to comment.