Skip to content

Commit 3d75ded

Browse files
refactor!(iroh): move protocol specific fields in NodeInner into protocols (#2723)
- refactor(iroh): store gossip in protocolmap - refactor(iroh): avoid duplicate SecretKey storage - refactor(iroh): store downloader as part of the blobs - refactor(iroh): store DocsEngine in protocols - refactor(iroh): move blobs store into BlobsProtocol - refactor(iroh): move BlobsBatches into BlobsProtocol ## Breaking Changes - removed - `iroh::node::ProtocolBuilder::downloader` - `iroh::node::ProtocolBuilder::blobs_db` - `iroh::node::ProtocolBuilder::gossip`
1 parent 8a4bb09 commit 3d75ded

File tree

5 files changed

+413
-368
lines changed

5 files changed

+413
-368
lines changed

iroh/src/node.rs

Lines changed: 27 additions & 91 deletions
Original file line numberDiff line numberDiff line change
@@ -35,8 +35,9 @@
3535
//! well, without going through [`client`](crate::client::Iroh))
3636
//!
3737
//! To shut down the node, call [`Node::shutdown`].
38-
use std::collections::{BTreeMap, BTreeSet};
38+
use std::collections::BTreeSet;
3939
use std::fmt::Debug;
40+
use std::marker::PhantomData;
4041
use std::net::SocketAddr;
4142
use std::path::{Path, PathBuf};
4243
use std::sync::Arc;
@@ -47,14 +48,14 @@ use futures_lite::StreamExt;
4748
use futures_util::future::MapErr;
4849
use futures_util::future::Shared;
4950
use iroh_base::key::PublicKey;
51+
use iroh_blobs::protocol::Closed;
5052
use iroh_blobs::store::Store as BaoStore;
5153
use iroh_blobs::util::local_pool::{LocalPool, LocalPoolHandle};
52-
use iroh_blobs::{downloader::Downloader, protocol::Closed};
53-
use iroh_blobs::{HashAndFormat, TempTag};
54-
use iroh_gossip::net::Gossip;
54+
use iroh_docs::net::DOCS_ALPN;
55+
use iroh_gossip::net::{Gossip, GOSSIP_ALPN};
5556
use iroh_net::endpoint::{DirectAddrsStream, RemoteInfo};
56-
use iroh_net::key::SecretKey;
5757
use iroh_net::{AddrInfo, Endpoint, NodeAddr};
58+
use protocol::BlobsProtocol;
5859
use quic_rpc::transport::ServerEndpoint as _;
5960
use quic_rpc::RpcServer;
6061
use tokio::task::{JoinError, JoinSet};
@@ -64,7 +65,6 @@ use tracing::{debug, error, info, info_span, trace, warn, Instrument};
6465

6566
use crate::node::nodes_storage::store_node_addrs;
6667
use crate::node::{docs::DocsEngine, protocol::ProtocolMap};
67-
use crate::rpc_protocol::blobs::BatchId;
6868

6969
mod builder;
7070
mod docs;
@@ -118,70 +118,14 @@ pub(crate) type JoinErrToStr = Box<dyn Fn(JoinError) -> String + Send + Sync + '
118118

119119
#[derive(derive_more::Debug)]
120120
struct NodeInner<D> {
121-
db: D,
121+
db: PhantomData<D>,
122122
rpc_addr: Option<SocketAddr>,
123-
docs: Option<DocsEngine>,
124123
endpoint: Endpoint,
125-
gossip: Gossip,
126-
secret_key: SecretKey,
127124
cancel_token: CancellationToken,
128125
client: crate::client::Iroh,
129-
downloader: Downloader,
130-
blob_batches: tokio::sync::Mutex<BlobBatches>,
131126
local_pool_handle: LocalPoolHandle,
132127
}
133128

134-
/// Keeps track of all the currently active batch operations of the blobs api.
135-
#[derive(Debug, Default)]
136-
struct BlobBatches {
137-
/// Currently active batches
138-
batches: BTreeMap<BatchId, BlobBatch>,
139-
/// Used to generate new batch ids.
140-
max: u64,
141-
}
142-
143-
/// A single batch of blob operations
144-
#[derive(Debug, Default)]
145-
struct BlobBatch {
146-
/// The tags in this batch.
147-
tags: BTreeMap<HashAndFormat, Vec<TempTag>>,
148-
}
149-
150-
impl BlobBatches {
151-
/// Create a new unique batch id.
152-
fn create(&mut self) -> BatchId {
153-
let id = self.max;
154-
self.max += 1;
155-
BatchId(id)
156-
}
157-
158-
/// Store a temp tag in a batch identified by a batch id.
159-
fn store(&mut self, batch: BatchId, tt: TempTag) {
160-
let entry = self.batches.entry(batch).or_default();
161-
entry.tags.entry(tt.hash_and_format()).or_default().push(tt);
162-
}
163-
164-
/// Remove a tag from a batch.
165-
fn remove_one(&mut self, batch: BatchId, content: &HashAndFormat) -> Result<()> {
166-
if let Some(batch) = self.batches.get_mut(&batch) {
167-
if let Some(tags) = batch.tags.get_mut(content) {
168-
tags.pop();
169-
if tags.is_empty() {
170-
batch.tags.remove(content);
171-
}
172-
return Ok(());
173-
}
174-
}
175-
// this can happen if we try to upgrade a tag from an expired batch
176-
anyhow::bail!("tag not found in batch");
177-
}
178-
179-
/// Remove an entire batch.
180-
fn remove(&mut self, batch: BatchId) {
181-
self.batches.remove(&batch);
182-
}
183-
}
184-
185129
/// In memory node.
186130
pub type MemNode = Node<iroh_blobs::store::mem::Store>;
187131

@@ -245,7 +189,7 @@ impl<D: BaoStore> Node<D> {
245189

246190
/// Returns the [`PublicKey`] of the node.
247191
pub fn node_id(&self) -> PublicKey {
248-
self.inner.secret_key.public()
192+
self.inner.endpoint.secret_key().public()
249193
}
250194

251195
/// Return a client to control this node over an in-memory channel.
@@ -344,32 +288,40 @@ impl<D: iroh_blobs::store::Store> NodeInner<D> {
344288
let external_rpc = RpcServer::new(external_rpc);
345289
let internal_rpc = RpcServer::new(internal_rpc);
346290

291+
let gossip = protocols
292+
.get_typed::<Gossip>(GOSSIP_ALPN)
293+
.expect("missing gossip");
294+
347295
// TODO(frando): I think this is not needed as we do the same in a task just below.
348296
// forward the initial endpoints to the gossip protocol.
349297
// it may happen the the first endpoint update callback is missed because the gossip cell
350298
// is only initialized once the endpoint is fully bound
351299
if let Some(direct_addresses) = self.endpoint.direct_addresses().next().await {
352300
debug!(me = ?self.endpoint.node_id(), "gossip initial update: {direct_addresses:?}");
353-
self.gossip.update_direct_addresses(&direct_addresses).ok();
301+
gossip.update_direct_addresses(&direct_addresses).ok();
354302
}
355303

356304
// Spawn a task for the garbage collection.
357305
if let GcPolicy::Interval(gc_period) = gc_policy {
358-
let inner = self.clone();
306+
let protocols = protocols.clone();
359307
let handle = local_pool.spawn(move || async move {
360-
let inner2 = inner.clone();
361-
inner
362-
.db
308+
let docs_engine = protocols.get_typed::<DocsEngine>(DOCS_ALPN);
309+
let blobs = protocols
310+
.get_typed::<BlobsProtocol<D>>(iroh_blobs::protocol::ALPN)
311+
.expect("missing blobs");
312+
313+
blobs
314+
.store()
363315
.gc_run(
364316
iroh_blobs::store::GcConfig {
365317
period: gc_period,
366318
done_callback: gc_done_callback,
367319
},
368320
move || {
369-
let inner2 = inner2.clone();
321+
let docs_engine = docs_engine.clone();
370322
async move {
371323
let mut live = BTreeSet::default();
372-
if let Some(docs) = &inner2.docs {
324+
if let Some(docs) = docs_engine {
373325
let doc_hashes = match docs.sync.content_hashes().await {
374326
Ok(hashes) => hashes,
375327
Err(err) => {
@@ -449,7 +401,7 @@ impl<D: iroh_blobs::store::Store> NodeInner<D> {
449401
join_set.spawn(async move {
450402
let mut stream = inner.endpoint.direct_addresses();
451403
while let Some(eps) = stream.next().await {
452-
if let Err(err) = inner.gossip.update_direct_addresses(&eps) {
404+
if let Err(err) = gossip.update_direct_addresses(&eps) {
453405
warn!("Failed to update direct addresses for gossip: {err:?}");
454406
}
455407
}
@@ -468,7 +420,7 @@ impl<D: iroh_blobs::store::Store> NodeInner<D> {
468420
request = external_rpc.accept() => {
469421
match request {
470422
Ok(accepting) => {
471-
rpc::Handler::spawn_rpc_request(self.clone(), &mut join_set, accepting);
423+
rpc::Handler::spawn_rpc_request(self.clone(), &mut join_set, accepting, protocols.clone());
472424
}
473425
Err(e) => {
474426
info!("rpc request error: {:?}", e);
@@ -479,7 +431,7 @@ impl<D: iroh_blobs::store::Store> NodeInner<D> {
479431
request = internal_rpc.accept() => {
480432
match request {
481433
Ok(accepting) => {
482-
rpc::Handler::spawn_rpc_request(self.clone(), &mut join_set, accepting);
434+
rpc::Handler::spawn_rpc_request(self.clone(), &mut join_set, accepting, protocols.clone());
483435
}
484436
Err(e) => {
485437
info!("internal rpc request error: {:?}", e);
@@ -533,18 +485,6 @@ impl<D: iroh_blobs::store::Store> NodeInner<D> {
533485
async fn shutdown(&self, protocols: Arc<ProtocolMap>) {
534486
let error_code = Closed::ProviderTerminating;
535487

536-
// Shutdown future for the docs engine, if enabled.
537-
let docs_shutdown = {
538-
let docs = self.docs.clone();
539-
async move {
540-
if let Some(docs) = docs {
541-
docs.shutdown().await
542-
} else {
543-
Ok(())
544-
}
545-
}
546-
};
547-
548488
// We ignore all errors during shutdown.
549489
let _ = tokio::join!(
550490
// Close the endpoint.
@@ -554,10 +494,6 @@ impl<D: iroh_blobs::store::Store> NodeInner<D> {
554494
self.endpoint
555495
.clone()
556496
.close(error_code.into(), error_code.reason()),
557-
// Shutdown docs engine.
558-
docs_shutdown,
559-
// Shutdown blobs store engine.
560-
self.db.shutdown(),
561497
// Shutdown protocol handlers.
562498
protocols.shutdown(),
563499
);
@@ -636,7 +572,7 @@ mod tests {
636572
use bytes::Bytes;
637573
use iroh_base::node_addr::AddrInfoOptions;
638574
use iroh_blobs::{provider::AddProgress, util::SetTagOption, BlobFormat};
639-
use iroh_net::{relay::RelayMode, test_utils::DnsPkarrServer, NodeAddr};
575+
use iroh_net::{key::SecretKey, relay::RelayMode, test_utils::DnsPkarrServer, NodeAddr};
640576

641577
use crate::client::blobs::{AddOutcome, WrapOption};
642578

iroh/src/node/builder.rs

Lines changed: 19 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -647,16 +647,11 @@ where
647647

648648
let inner = Arc::new(NodeInner {
649649
rpc_addr: self.rpc_addr,
650-
db: self.blobs_store,
651-
docs,
650+
db: Default::default(),
652651
endpoint,
653-
secret_key: self.secret_key,
654652
client,
655653
cancel_token: CancellationToken::new(),
656-
downloader,
657-
gossip,
658654
local_pool_handle: lp.handle().clone(),
659-
blob_batches: Default::default(),
660655
});
661656

662657
let protocol_builder = ProtocolBuilder {
@@ -670,7 +665,13 @@ where
670665
local_pool: lp,
671666
};
672667

673-
let protocol_builder = protocol_builder.register_iroh_protocols(self.blob_events);
668+
let protocol_builder = protocol_builder.register_iroh_protocols(
669+
self.blob_events,
670+
self.blobs_store,
671+
gossip,
672+
downloader,
673+
docs,
674+
);
674675

675676
Ok(protocol_builder)
676677
}
@@ -764,26 +765,11 @@ impl<D: iroh_blobs::store::Store> ProtocolBuilder<D> {
764765
&self.inner.endpoint
765766
}
766767

767-
/// Returns the [`crate::blobs::store::Store`] used by the node.
768-
pub fn blobs_db(&self) -> &D {
769-
&self.inner.db
770-
}
771-
772768
/// Returns a reference to the used [`LocalPoolHandle`].
773769
pub fn local_pool_handle(&self) -> &LocalPoolHandle {
774770
self.local_pool.handle()
775771
}
776772

777-
/// Returns a reference to the [`Downloader`] used by the node.
778-
pub fn downloader(&self) -> &Downloader {
779-
&self.inner.downloader
780-
}
781-
782-
/// Returns a reference to the [`Gossip`] handle used by the node.
783-
pub fn gossip(&self) -> &Gossip {
784-
&self.inner.gossip
785-
}
786-
787773
/// Returns a protocol handler for an ALPN.
788774
///
789775
/// This downcasts to the concrete type and returns `None` if the handler registered for `alpn`
@@ -793,21 +779,28 @@ impl<D: iroh_blobs::store::Store> ProtocolBuilder<D> {
793779
}
794780

795781
/// Registers the core iroh protocols (blobs, gossip, docs).
796-
fn register_iroh_protocols(mut self, blob_events: EventSender) -> Self {
782+
fn register_iroh_protocols(
783+
mut self,
784+
blob_events: EventSender,
785+
store: D,
786+
gossip: Gossip,
787+
downloader: Downloader,
788+
docs: Option<DocsEngine>,
789+
) -> Self {
797790
// Register blobs.
798791
let blobs_proto = BlobsProtocol::new_with_events(
799-
self.blobs_db().clone(),
792+
store,
800793
self.local_pool_handle().clone(),
801794
blob_events,
795+
downloader,
802796
);
803797
self = self.accept(iroh_blobs::protocol::ALPN.to_vec(), Arc::new(blobs_proto));
804798

805799
// Register gossip.
806-
let gossip = self.gossip().clone();
807800
self = self.accept(GOSSIP_ALPN.to_vec(), Arc::new(gossip));
808801

809802
// Register docs, if enabled.
810-
if let Some(docs) = self.inner.docs.clone() {
803+
if let Some(docs) = docs {
811804
self = self.accept(DOCS_ALPN.to_vec(), Arc::new(docs));
812805
}
813806

iroh/src/node/docs.rs

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,4 +52,13 @@ impl ProtocolHandler for DocsEngine {
5252
fn accept(self: Arc<Self>, conn: Connecting) -> BoxedFuture<Result<()>> {
5353
Box::pin(async move { self.handle_connection(conn).await })
5454
}
55+
56+
fn shutdown(self: Arc<Self>) -> BoxedFuture<()> {
57+
Box::pin(async move {
58+
let this: &Self = &self;
59+
if let Err(err) = this.shutdown().await {
60+
tracing::warn!("shutdown error: {:?}", err);
61+
}
62+
})
63+
}
5564
}

0 commit comments

Comments
 (0)