From 176715a567c7f5c660a980dcc334b5d5892d2cb1 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Wed, 30 Oct 2024 18:48:30 +0100 Subject: [PATCH 01/26] refactor: extract RPC definitions into here --- Cargo.lock | 113 ++-- Cargo.toml | 19 +- src/engine.rs | 11 +- src/lib.rs | 3 + src/rpc.rs | 56 ++ src/rpc/client.rs | 913 +++++++++++++++++++++++++++++++++ src/rpc/docs_handle_request.rs | 593 +++++++++++++++++++++ src/rpc/proto.rs | 435 ++++++++++++++++ 8 files changed, 2107 insertions(+), 36 deletions(-) create mode 100644 src/rpc.rs create mode 100644 src/rpc/client.rs create mode 100644 src/rpc/docs_handle_request.rs create mode 100644 src/rpc/proto.rs diff --git a/Cargo.lock b/Cargo.lock index 10ac6b9..6be02e7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -758,6 +758,18 @@ dependencies = [ "zeroize", ] +[[package]] +name = "educe" +version = "0.4.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f0042ff8246a363dbe77d2ceedb073339e85a804b9a47636c6e016a9a32c05f" +dependencies = [ + "enum-ordinalize", + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "elliptic-curve" version = "0.13.8" @@ -795,7 +807,20 @@ version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a1e6a265c649f3f5979b601d26f1d05ada116434c87741c9493cb56218f76cbc" dependencies = [ - "heck 0.5.0", + "heck", + "proc-macro2", + "quote", + "syn 2.0.82", +] + +[[package]] +name = "enum-ordinalize" +version = "3.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bf1fa3f06bbff1ea5b1a9c7b14aa992a39657db60a2759457328d7e058f49ee" +dependencies = [ + "num-bigint", + "num-traits", "proc-macro2", "quote", "syn 2.0.82", @@ -1269,12 +1294,6 @@ dependencies = [ "hashbrown 0.14.5", ] -[[package]] -name = "heck" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" - [[package]] name = "heck" version = "0.5.0" @@ -1712,9 +1731,13 @@ dependencies = [ "iroh-router", "iroh-test", "lru", + "nested_enum_utils", "num_enum", + "portable-atomic", "postcard", "proptest", + "quic-rpc", + "quic-rpc-derive", "rand", "rand_chacha", "rand_core", @@ -1722,7 +1745,8 @@ dependencies = [ "redb 2.1.4", "self_cell", "serde", - "strum 0.25.0", + "serde-error", + "strum", "tempfile", "test-strategy", "thiserror", @@ -1847,7 +1871,7 @@ dependencies = [ "serde", "smallvec", "socket2", - "strum 0.26.3", + "strum", "stun-rs", "surge-ping", "thiserror", @@ -2161,6 +2185,18 @@ dependencies = [ "getrandom", ] +[[package]] +name = "nested_enum_utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f256ef99e7ac37428ef98c89bef9d84b590172de4bbfbe81b68a4cd3abadb32" +dependencies = [ + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "netdev" version = "0.30.0" @@ -2925,6 +2961,39 @@ dependencies = [ "winapi", ] +[[package]] +name = "quic-rpc" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b0ea1bd0b3124538bb71ed8cedbe92608fd1cf227e4f5ff53fb28746737b794" +dependencies = [ + "anyhow", + "derive_more", + "educe", + "flume", + "futures-lite 2.3.0", + "futures-sink", + "futures-util", + "hex", + "pin-project", + "serde", + "slab", + "tokio", + "tracing", +] + +[[package]] +name = "quic-rpc-derive" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94b91a3f7a42657cbfbd0c2499c1f037738eff45bb7f59c6ce3d3d9e890d141c" +dependencies = [ + "proc-macro2", + "quic-rpc", + "quote", + "syn 1.0.109", +] + [[package]] name = "quick-error" version = "1.2.3" @@ -3850,35 +3919,13 @@ dependencies = [ "syn 2.0.82", ] -[[package]] -name = "strum" -version = "0.25.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" -dependencies = [ - "strum_macros 0.25.3", -] - [[package]] name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" dependencies = [ - "strum_macros 0.26.4", -] - -[[package]] -name = "strum_macros" -version = "0.25.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" -dependencies = [ - "heck 0.4.1", - "proc-macro2", - "quote", - "rustversion", - "syn 2.0.82", + "strum_macros", ] [[package]] @@ -3887,7 +3934,7 @@ version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" dependencies = [ - "heck 0.5.0", + "heck", "proc-macro2", "quote", "rustversion", diff --git a/Cargo.toml b/Cargo.toml index b6e8315..0981d13 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -50,7 +50,7 @@ redb = { version = "2.0.0" } redb_v1 = { package = "redb", version = "1.5.1" } self_cell = "1.0.3" serde = { version = "1.0.164", features = ["derive"] } -strum = { version = "0.25", features = ["derive"] } +strum = { version = "0.26", features = ["derive"] } tempfile = { version = "3.4" } thiserror = "1" tokio = { version = "1", features = ["sync", "rt", "time", "macros"] } @@ -58,6 +58,13 @@ tokio-stream = { version = "0.1", optional = true, features = ["sync"]} tokio-util = { version = "0.7.12", optional = true, features = ["codec", "io-util", "io", "rt"] } tracing = "0.1" +# rpc +nested_enum_utils = { version = "0.1.0", optional = true } +quic-rpc = { version = "0.13", optional = true } +quic-rpc-derive = { version = "0.13", optional = true } +serde-error = { version = "0.1.3", optional = true } +portable-atomic = { version = "1.9.0", optional = true } + [dev-dependencies] iroh-test = "0.27.0" rand_chacha = "0.3.1" @@ -67,10 +74,18 @@ tempfile = "3.4" test-strategy = "0.3.1" [features] -default = ["net", "metrics", "engine"] +default = ["net", "metrics", "engine", "rpc"] net = ["dep:iroh-net", "tokio/io-util", "dep:tokio-stream", "dep:tokio-util"] metrics = ["iroh-metrics/metrics"] engine = ["net", "dep:iroh-gossip", "dep:iroh-blobs", "dep:iroh-router"] +rpc = [ + "engine", + "dep:nested_enum_utils", + "dep:quic-rpc", + "dep:quic-rpc-derive", + "dep:serde-error", + "dep:portable-atomic", +] [package.metadata.docs.rs] all-features = true diff --git a/src/engine.rs b/src/engine.rs index ed8bb5f..36f6dcc 100644 --- a/src/engine.rs +++ b/src/engine.rs @@ -11,7 +11,9 @@ use std::{ use anyhow::{bail, Context, Result}; use futures_lite::{Stream, StreamExt}; -use iroh_blobs::{downloader::Downloader, store::EntryStatus, Hash}; +use iroh_blobs::{ + downloader::Downloader, store::EntryStatus, util::local_pool::LocalPoolHandle, Hash, +}; use iroh_gossip::net::Gossip; use iroh_net::{key::PublicKey, Endpoint, NodeAddr}; use serde::{Deserialize, Serialize}; @@ -52,6 +54,7 @@ pub struct Engine { actor_handle: Arc>, #[debug("ContentStatusCallback")] content_status_cb: ContentStatusCallback, + local_pool_handle: LocalPoolHandle, } impl Engine { @@ -66,6 +69,7 @@ impl Engine { bao_store: B, downloader: Downloader, default_author_storage: DefaultAuthorStorage, + local_pool_handle: LocalPoolHandle, ) -> anyhow::Result { let (live_actor_tx, to_live_actor_recv) = mpsc::channel(ACTOR_CHANNEL_CAP); let me = endpoint.node_id().fmt_short(); @@ -111,6 +115,7 @@ impl Engine { actor_handle: Arc::new(AbortOnDropHandle::new(actor_handle)), content_status_cb, default_author: Arc::new(default_author), + local_pool_handle, }) } @@ -205,6 +210,10 @@ impl Engine { reply_rx.await?; Ok(()) } + + pub(crate) fn local_pool_handle(&self) -> &LocalPoolHandle { + &self.local_pool_handle + } } /// Converts an [`EntryStatus`] into a ['ContentStatus']. diff --git a/src/lib.rs b/src/lib.rs index 63702b3..9ebbd23 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -48,6 +48,9 @@ mod ticket; #[cfg(feature = "engine")] #[cfg_attr(iroh_docsrs, doc(cfg(feature = "engine")))] pub mod engine; +#[cfg(feature = "engine")] +#[cfg_attr(iroh_docsrs, doc(cfg(feature = "engine")))] +pub mod rpc; pub mod actor; pub mod store; diff --git a/src/rpc.rs b/src/rpc.rs new file mode 100644 index 0000000..3103777 --- /dev/null +++ b/src/rpc.rs @@ -0,0 +1,56 @@ +//! Quic RPC implemenation for docs. + +use crate::engine::Engine; + +pub mod client; +pub mod proto; + +mod docs_handle_request; + +type RpcError = serde_error::Error; +type RpcResult = std::result::Result; + +impl Engine { + /// Handle a docs request from the RPC server. + pub async fn handle_rpc_request>( + &self, + msg: crate::rpc::proto::Request, + chan: quic_rpc::server::RpcChannel, + ) -> Result<(), quic_rpc::server::RpcServerError> { + use crate::rpc::proto::Request::*; + + let this = self.clone(); + match msg { + Open(msg) => chan.rpc(msg, this, Self::doc_open).await, + Close(msg) => chan.rpc(msg, this, Self::doc_close).await, + Status(msg) => chan.rpc(msg, this, Self::doc_status).await, + List(msg) => chan.server_streaming(msg, this, Self::doc_list).await, + Create(msg) => chan.rpc(msg, this, Self::doc_create).await, + Drop(msg) => chan.rpc(msg, this, Self::doc_drop).await, + Import(msg) => chan.rpc(msg, this, Self::doc_import).await, + Set(msg) => chan.rpc(msg, this, Self::doc_set).await, + ImportFile(msg) => { + chan.server_streaming(msg, this, Self::doc_import_file) + .await + } + ExportFile(msg) => { + chan.server_streaming(msg, this, Self::doc_export_file) + .await + } + Del(msg) => chan.rpc(msg, this, Self::doc_del).await, + SetHash(msg) => chan.rpc(msg, this, Self::doc_set_hash).await, + Get(msg) => chan.server_streaming(msg, this, Self::doc_get_many).await, + GetExact(msg) => chan.rpc(msg, this, Self::doc_get_exact).await, + StartSync(msg) => chan.rpc(msg, this, Self::doc_start_sync).await, + Leave(msg) => chan.rpc(msg, this, Self::doc_leave).await, + Share(msg) => chan.rpc(msg, this, Self::doc_share).await, + Subscribe(msg) => { + chan.try_server_streaming(msg, this, Self::doc_subscribe) + .await + } + SetDownloadPolicy(msg) => chan.rpc(msg, this, Self::doc_set_download_policy).await, + GetDownloadPolicy(msg) => chan.rpc(msg, this, Self::doc_get_download_policy).await, + GetSyncPeers(msg) => chan.rpc(msg, this, Self::doc_get_sync_peers).await, + } + } +} diff --git a/src/rpc/client.rs b/src/rpc/client.rs new file mode 100644 index 0000000..9590437 --- /dev/null +++ b/src/rpc/client.rs @@ -0,0 +1,913 @@ +//! API for document management. +//! +//! The main entry point is the [`Client`]. +//! +//! You obtain a [`Client`] via [`Iroh::docs()`](crate::client::Iroh::docs). + +use std::{ + path::{Path, PathBuf}, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +use anyhow::{anyhow, Context as _, Result}; +use bytes::Bytes; +use derive_more::{Display, FromStr}; +use futures_lite::{Stream, StreamExt}; +use iroh_base::{key::PublicKey, node_addr::AddrInfoOptions}; +use iroh_blobs::{export::ExportProgress, store::ExportMode, Hash}; +use iroh_net::NodeAddr; +use portable_atomic::{AtomicBool, Ordering}; +use quic_rpc::{client::BoxedServiceConnection, message::RpcMsg}; +use serde::{Deserialize, Serialize}; + +use super::proto::{ + CloseRequest, CreateRequest, DelRequest, DelResponse, DocListRequest, DocSubscribeRequest, + DropRequest, ExportFileRequest, GetDownloadPolicyRequest, GetExactRequest, GetManyRequest, + GetSyncPeersRequest, ImportFileRequest, ImportRequest, LeaveRequest, OpenRequest, RpcService, + SetDownloadPolicyRequest, SetHashRequest, SetRequest, ShareRequest, StartSyncRequest, + StatusRequest, +}; +#[doc(inline)] +pub use crate::engine::{Origin, SyncEvent, SyncReason}; +use crate::{ + actor::OpenState, + store::{DownloadPolicy, Query}, + AuthorId, Capability, CapabilityKind, ContentStatus, DocTicket, NamespaceId, PeerIdBytes, + RecordIdentifier, +}; + +/// Iroh docs client. +#[derive(Debug, Clone)] +pub struct Client> { + pub(super) rpc: quic_rpc::RpcClient, +} + +impl Client +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection, +{ + /// Creates a new document. + pub async fn create(&self) -> Result> { + let res = self.rpc.rpc(CreateRequest {}).await??; + let doc = Doc::new(self.rpc.clone(), res.id); + Ok(doc) + } + + /// Deletes a document from the local node. + /// + /// This is a destructive operation. Both the document secret key and all entries in the + /// document will be permanently deleted from the node's storage. Content blobs will be deleted + /// through garbage collection unless they are referenced from another document or tag. + pub async fn drop_doc(&self, doc_id: NamespaceId) -> Result<()> { + self.rpc.rpc(DropRequest { doc_id }).await??; + Ok(()) + } + + /// Imports a document from a namespace capability. + /// + /// This does not start sync automatically. Use [`Doc::start_sync`] to start sync. + pub async fn import_namespace(&self, capability: Capability) -> Result> { + let res = self.rpc.rpc(ImportRequest { capability }).await??; + let doc = Doc::new(self.rpc.clone(), res.doc_id); + Ok(doc) + } + + /// Imports a document from a ticket and joins all peers in the ticket. + pub async fn import(&self, ticket: DocTicket) -> Result> { + let DocTicket { capability, nodes } = ticket; + let doc = self.import_namespace(capability).await?; + doc.start_sync(nodes).await?; + Ok(doc) + } + + /// Imports a document from a ticket, creates a subscription stream and joins all peers in the ticket. + /// + /// Returns the [`Doc`] and a [`Stream`] of [`LiveEvent`]s. + /// + /// The subscription stream is created before the sync is started, so the first call to this + /// method after starting the node is guaranteed to not miss any sync events. + pub async fn import_and_subscribe( + &self, + ticket: DocTicket, + ) -> Result<(Doc, impl Stream>)> { + let DocTicket { capability, nodes } = ticket; + let res = self.rpc.rpc(ImportRequest { capability }).await??; + let doc = Doc::new(self.rpc.clone(), res.doc_id); + let events = doc.subscribe().await?; + doc.start_sync(nodes).await?; + Ok((doc, events)) + } + + /// Lists all documents. + pub async fn list(&self) -> Result>> { + let stream = self.rpc.server_streaming(DocListRequest {}).await?; + Ok(flatten(stream).map(|res| res.map(|res| (res.id, res.capability)))) + } + + /// Returns a [`Doc`] client for a single document. + /// + /// Returns None if the document cannot be found. + pub async fn open(&self, id: NamespaceId) -> Result>> { + self.rpc.rpc(OpenRequest { doc_id: id }).await??; + let doc = Doc::new(self.rpc.clone(), id); + Ok(Some(doc)) + } +} + +/// Document handle +#[derive(Debug, Clone)] +pub struct Doc>(Arc>) +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection; + +impl PartialEq for Doc +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection, +{ + fn eq(&self, other: &Self) -> bool { + self.0.id == other.0.id + } +} + +impl Eq for Doc +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection, +{ +} + +#[derive(Debug)] +struct DocInner> +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection, +{ + id: NamespaceId, + rpc: quic_rpc::RpcClient, + closed: AtomicBool, + rt: tokio::runtime::Handle, +} + +impl Drop for DocInner +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection, +{ + fn drop(&mut self) { + let doc_id = self.id; + let rpc = self.rpc.clone(); + if !self.closed.swap(true, Ordering::Relaxed) { + self.rt.spawn(async move { + rpc.rpc(CloseRequest { doc_id }).await.ok(); + }); + } + } +} + +impl Doc +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection, +{ + fn new(rpc: quic_rpc::RpcClient, id: NamespaceId) -> Self { + Self(Arc::new(DocInner { + rpc, + id, + closed: AtomicBool::new(false), + rt: tokio::runtime::Handle::current(), + })) + } + + async fn rpc(&self, msg: M) -> Result + where + M: RpcMsg, + { + let res = self.0.rpc.rpc(msg).await?; + Ok(res) + } + + /// Returns the document id of this doc. + pub fn id(&self) -> NamespaceId { + self.0.id + } + + /// Closes the document. + pub async fn close(&self) -> Result<()> { + if !self.0.closed.swap(true, Ordering::Relaxed) { + self.rpc(CloseRequest { doc_id: self.id() }).await??; + } + Ok(()) + } + + fn ensure_open(&self) -> Result<()> { + if self.0.closed.load(Ordering::Relaxed) { + Err(anyhow!("document is closed")) + } else { + Ok(()) + } + } + + /// Sets the content of a key to a byte array. + pub async fn set_bytes( + &self, + author_id: AuthorId, + key: impl Into, + value: impl Into, + ) -> Result { + self.ensure_open()?; + let res = self + .rpc(SetRequest { + doc_id: self.id(), + author_id, + key: key.into(), + value: value.into(), + }) + .await??; + Ok(res.entry.content_hash()) + } + + /// Sets an entries on the doc via its key, hash, and size. + pub async fn set_hash( + &self, + author_id: AuthorId, + key: impl Into, + hash: Hash, + size: u64, + ) -> Result<()> { + self.ensure_open()?; + self.rpc(SetHashRequest { + doc_id: self.id(), + author_id, + key: key.into(), + hash, + size, + }) + .await??; + Ok(()) + } + + /// Adds an entry from an absolute file path + pub async fn import_file( + &self, + author: AuthorId, + key: Bytes, + path: impl AsRef, + in_place: bool, + ) -> Result { + self.ensure_open()?; + let stream = self + .0 + .rpc + .server_streaming(ImportFileRequest { + doc_id: self.id(), + author_id: author, + path: path.as_ref().into(), + key, + in_place, + }) + .await?; + Ok(ImportFileProgress::new(stream)) + } + + /// Exports an entry as a file to a given absolute path. + pub async fn export_file( + &self, + entry: Entry, + path: impl AsRef, + mode: ExportMode, + ) -> Result { + self.ensure_open()?; + let stream = self + .0 + .rpc + .server_streaming(ExportFileRequest { + entry: entry.0, + path: path.as_ref().into(), + mode, + }) + .await?; + Ok(ExportFileProgress::new(stream)) + } + + /// Deletes entries that match the given `author` and key `prefix`. + /// + /// This inserts an empty entry with the key set to `prefix`, effectively clearing all other + /// entries whose key starts with or is equal to the given `prefix`. + /// + /// Returns the number of entries deleted. + pub async fn del(&self, author_id: AuthorId, prefix: impl Into) -> Result { + self.ensure_open()?; + let res = self + .rpc(DelRequest { + doc_id: self.id(), + author_id, + prefix: prefix.into(), + }) + .await??; + let DelResponse { removed } = res; + Ok(removed) + } + + /// Returns an entry for a key and author. + /// + /// Optionally also returns the entry unless it is empty (i.e. a deletion marker). + pub async fn get_exact( + &self, + author: AuthorId, + key: impl AsRef<[u8]>, + include_empty: bool, + ) -> Result> { + self.ensure_open()?; + let res = self + .rpc(GetExactRequest { + author, + key: key.as_ref().to_vec().into(), + doc_id: self.id(), + include_empty, + }) + .await??; + Ok(res.entry.map(|entry| entry.into())) + } + + /// Returns all entries matching the query. + pub async fn get_many( + &self, + query: impl Into, + ) -> Result>> { + self.ensure_open()?; + let stream = self + .0 + .rpc + .server_streaming(GetManyRequest { + doc_id: self.id(), + query: query.into(), + }) + .await?; + Ok(flatten(stream).map(|res| res.map(|res| res.entry.into()))) + } + + /// Returns a single entry. + pub async fn get_one(&self, query: impl Into) -> Result> { + self.get_many(query).await?.next().await.transpose() + } + + /// Shares this document with peers over a ticket. + pub async fn share( + &self, + mode: ShareMode, + addr_options: AddrInfoOptions, + ) -> anyhow::Result { + self.ensure_open()?; + let res = self + .rpc(ShareRequest { + doc_id: self.id(), + mode, + addr_options, + }) + .await??; + Ok(res.0) + } + + /// Starts to sync this document with a list of peers. + pub async fn start_sync(&self, peers: Vec) -> Result<()> { + self.ensure_open()?; + let _res = self + .rpc(StartSyncRequest { + doc_id: self.id(), + peers, + }) + .await??; + Ok(()) + } + + /// Stops the live sync for this document. + pub async fn leave(&self) -> Result<()> { + self.ensure_open()?; + let _res = self.rpc(LeaveRequest { doc_id: self.id() }).await??; + Ok(()) + } + + /// Subscribes to events for this document. + pub async fn subscribe(&self) -> anyhow::Result>> { + self.ensure_open()?; + let stream = self + .0 + .rpc + .try_server_streaming(DocSubscribeRequest { doc_id: self.id() }) + .await?; + Ok(stream.map(|res| match res { + Ok(res) => Ok(res.event.into()), + Err(err) => Err(err.into()), + })) + } + + /// Returns status info for this document + pub async fn status(&self) -> anyhow::Result { + self.ensure_open()?; + let res = self.rpc(StatusRequest { doc_id: self.id() }).await??; + Ok(res.status) + } + + /// Sets the download policy for this document + pub async fn set_download_policy(&self, policy: DownloadPolicy) -> Result<()> { + self.rpc(SetDownloadPolicyRequest { + doc_id: self.id(), + policy, + }) + .await??; + Ok(()) + } + + /// Returns the download policy for this document + pub async fn get_download_policy(&self) -> Result { + let res = self + .rpc(GetDownloadPolicyRequest { doc_id: self.id() }) + .await??; + Ok(res.policy) + } + + /// Returns sync peers for this document + pub async fn get_sync_peers(&self) -> Result>> { + let res = self + .rpc(GetSyncPeersRequest { doc_id: self.id() }) + .await??; + Ok(res.peers) + } +} + +impl<'a, S, C> From<&'a Doc> for &'a quic_rpc::RpcClient +where + S: quic_rpc::Service, + C: quic_rpc::ServiceConnection, +{ + fn from(doc: &'a Doc) -> &'a quic_rpc::RpcClient { + &doc.0.rpc + } +} + +/// A single entry in a [`Doc`]. +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq)] +pub struct Entry(crate::Entry); + +impl From for Entry { + fn from(value: crate::Entry) -> Self { + Self(value) + } +} + +impl From for Entry { + fn from(value: crate::SignedEntry) -> Self { + Self(value.into()) + } +} + +impl Entry { + /// Returns the [`RecordIdentifier`] for this entry. + pub fn id(&self) -> &RecordIdentifier { + self.0.id() + } + + /// Returns the [`AuthorId`] of this entry. + pub fn author(&self) -> AuthorId { + self.0.author() + } + + /// Returns the [`struct@Hash`] of the content data of this record. + pub fn content_hash(&self) -> Hash { + self.0.content_hash() + } + + /// Returns the length of the data addressed by this record's content hash. + pub fn content_len(&self) -> u64 { + self.0.content_len() + } + + /// Returns the key of this entry. + pub fn key(&self) -> &[u8] { + self.0.key() + } + + /// Returns the timestamp of this entry. + pub fn timestamp(&self) -> u64 { + self.0.timestamp() + } + + // TODO: depends on an rpc client in iroh-blobs + // /// Reads the content of an [`Entry`] as a streaming [`blobs::Reader`]. + // /// + // /// You can pass either a [`Doc`] or the `Iroh` client by reference as `client`. + // pub async fn content_reader(&self, client: impl Into<&Client>) -> Result { + // blobs::Reader::from_rpc_read(client.into(), self.content_hash()).await + // } + + // /// Reads all content of an [`Entry`] into a buffer. + // /// + // /// You can pass either a [`Doc`] or the `Iroh` client by reference as `client`. + // pub async fn content_bytes(&self, client: impl Into<&Client>) -> Result { + // blobs::Reader::from_rpc_read(client.into(), self.content_hash()) + // .await? + // .read_to_bytes() + // .await + // } +} + +/// Progress messages for an doc import operation +/// +/// An import operation involves computing the outboard of a file, and then +/// either copying or moving the file into the database, then setting the author, hash, size, and tag of that +/// file as an entry in the doc. +#[derive(Debug, Serialize, Deserialize)] +pub enum ImportProgress { + /// An item was found with name `name`, from now on referred to via `id`. + Found { + /// A new unique id for this entry. + id: u64, + /// The name of the entry. + name: String, + /// The size of the entry in bytes. + size: u64, + }, + /// We got progress ingesting item `id`. + Progress { + /// The unique id of the entry. + id: u64, + /// The offset of the progress, in bytes. + offset: u64, + }, + /// We are done adding `id` to the data store and the hash is `hash`. + IngestDone { + /// The unique id of the entry. + id: u64, + /// The hash of the entry. + hash: Hash, + }, + /// We are done setting the entry to the doc. + AllDone { + /// The key of the entry + key: Bytes, + }, + /// We got an error and need to abort. + /// + /// This will be the last message in the stream. + Abort(serde_error::Error), +} + +/// Intended capability for document share tickets +#[derive(Serialize, Deserialize, Debug, Clone, Display, FromStr)] +pub enum ShareMode { + /// Read-only access + Read, + /// Write access + Write, +} + +/// Events informing about actions of the live sync progress. +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, strum::Display)] +pub enum LiveEvent { + /// A local insertion. + InsertLocal { + /// The inserted entry. + entry: Entry, + }, + /// Received a remote insert. + InsertRemote { + /// The peer that sent us the entry. + from: PublicKey, + /// The inserted entry. + entry: Entry, + /// If the content is available at the local node + content_status: ContentStatus, + }, + /// The content of an entry was downloaded and is now available at the local node + ContentReady { + /// The content hash of the newly available entry content + hash: Hash, + }, + /// We have a new neighbor in the swarm. + NeighborUp(PublicKey), + /// We lost a neighbor in the swarm. + NeighborDown(PublicKey), + /// A set-reconciliation sync finished. + SyncFinished(SyncEvent), + /// All pending content is now ready. + /// + /// This event signals that all queued content downloads from the last sync run have either + /// completed or failed. + /// + /// It will only be emitted after a [`Self::SyncFinished`] event, never before. + /// + /// Receiving this event does not guarantee that all content in the document is available. If + /// blobs failed to download, this event will still be emitted after all operations completed. + PendingContentReady, +} + +impl From for LiveEvent { + fn from(event: crate::engine::LiveEvent) -> LiveEvent { + match event { + crate::engine::LiveEvent::InsertLocal { entry } => Self::InsertLocal { + entry: entry.into(), + }, + crate::engine::LiveEvent::InsertRemote { + from, + entry, + content_status, + } => Self::InsertRemote { + from, + content_status, + entry: entry.into(), + }, + crate::engine::LiveEvent::ContentReady { hash } => Self::ContentReady { hash }, + crate::engine::LiveEvent::NeighborUp(node) => Self::NeighborUp(node), + crate::engine::LiveEvent::NeighborDown(node) => Self::NeighborDown(node), + crate::engine::LiveEvent::SyncFinished(details) => Self::SyncFinished(details), + crate::engine::LiveEvent::PendingContentReady => Self::PendingContentReady, + } + } +} + +/// Progress stream for [`Doc::import_file`]. +#[derive(derive_more::Debug)] +#[must_use = "streams do nothing unless polled"] +pub struct ImportFileProgress { + #[debug(skip)] + stream: Pin> + Send + Unpin + 'static>>, +} + +impl ImportFileProgress { + fn new( + stream: (impl Stream, impl Into>> + + Send + + Unpin + + 'static), + ) -> Self { + let stream = stream.map(|item| match item { + Ok(item) => Ok(item.into()), + Err(err) => Err(err.into()), + }); + Self { + stream: Box::pin(stream), + } + } + + /// Finishes writing the stream, ignoring all intermediate progress events. + /// + /// Returns a [`ImportFileOutcome`] which contains a tag, key, and hash and the size of the + /// content. + pub async fn finish(mut self) -> Result { + let mut entry_size = 0; + let mut entry_hash = None; + while let Some(msg) = self.next().await { + match msg? { + ImportProgress::Found { size, .. } => { + entry_size = size; + } + ImportProgress::AllDone { key } => { + let hash = entry_hash + .context("expected DocImportProgress::IngestDone event to occur")?; + let outcome = ImportFileOutcome { + hash, + key, + size: entry_size, + }; + return Ok(outcome); + } + ImportProgress::Abort(err) => return Err(err.into()), + ImportProgress::Progress { .. } => {} + ImportProgress::IngestDone { hash, .. } => { + entry_hash = Some(hash); + } + } + } + Err(anyhow!("Response stream ended prematurely")) + } +} + +/// Outcome of a [`Doc::import_file`] operation +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ImportFileOutcome { + /// The hash of the entry's content + pub hash: Hash, + /// The size of the entry + pub size: u64, + /// The key of the entry + pub key: Bytes, +} + +impl Stream for ImportFileProgress { + type Item = Result; + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut self.stream).poll_next(cx) + } +} + +/// Progress stream for [`Doc::export_file`]. +#[derive(derive_more::Debug)] +pub struct ExportFileProgress { + #[debug(skip)] + stream: Pin> + Send + Unpin + 'static>>, +} +impl ExportFileProgress { + fn new( + stream: (impl Stream, impl Into>> + + Send + + Unpin + + 'static), + ) -> Self { + let stream = stream.map(|item| match item { + Ok(item) => Ok(item.into()), + Err(err) => Err(err.into()), + }); + Self { + stream: Box::pin(stream), + } + } + + /// Iterates through the export progress stream, returning when the stream has completed. + /// + /// Returns a [`ExportFileOutcome`] which contains a file path the data was written to and the size of the content. + pub async fn finish(mut self) -> Result { + let mut total_size = 0; + let mut path = None; + while let Some(msg) = self.next().await { + match msg? { + ExportProgress::Found { size, outpath, .. } => { + total_size = size.value(); + path = Some(outpath); + } + ExportProgress::AllDone => { + let path = path.context("expected ExportProgress::Found event to occur")?; + let outcome = ExportFileOutcome { + size: total_size, + path, + }; + return Ok(outcome); + } + ExportProgress::Done { .. } => {} + ExportProgress::Abort(err) => return Err(anyhow!(err)), + ExportProgress::Progress { .. } => {} + } + } + Err(anyhow!("Response stream ended prematurely")) + } +} + +/// Outcome of a [`Doc::export_file`] operation +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ExportFileOutcome { + /// The size of the entry + size: u64, + /// The path to which the entry was saved + path: PathBuf, +} + +impl Stream for ExportFileProgress { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut self.stream).poll_next(cx) + } +} + +fn flatten( + s: impl Stream, E2>>, +) -> impl Stream> +where + E1: std::error::Error + Send + Sync + 'static, + E2: std::error::Error + Send + Sync + 'static, +{ + s.map(|res| match res { + Ok(Ok(res)) => Ok(res), + Ok(Err(err)) => Err(err.into()), + Err(err) => Err(err.into()), + }) +} + +// TODO +// #[cfg(test)] +// mod tests { +// use rand::RngCore; +// use tokio::io::AsyncWriteExt; + +// use super::*; + +// #[tokio::test] +// async fn test_drop_doc_client_sync() -> Result<()> { +// let _guard = iroh_test::logging::setup(); + +// let node = crate::node::Node::memory().enable_docs().spawn().await?; + +// let client = node.client(); +// let doc = client.docs().create().await?; + +// let res = std::thread::spawn(move || { +// drop(doc); +// drop(node); +// }); + +// tokio::task::spawn_blocking(move || res.join().map_err(|e| anyhow::anyhow!("{:?}", e))) +// .await??; + +// Ok(()) +// } + +// /// Test that closing a doc does not close other instances. +// #[tokio::test] +// async fn test_doc_close() -> Result<()> { +// let _guard = iroh_test::logging::setup(); + +// let node = crate::node::Node::memory().enable_docs().spawn().await?; +// let author = node.authors().default().await?; +// // open doc two times +// let doc1 = node.docs().create().await?; +// let doc2 = node.docs().open(doc1.id()).await?.expect("doc to exist"); +// // close doc1 instance +// doc1.close().await?; +// // operations on doc1 now fail. +// assert!(doc1.set_bytes(author, "foo", "bar").await.is_err()); +// // dropping doc1 will close the doc if not already closed +// // wait a bit because the close-on-drop spawns a task for which we cannot track completion. +// drop(doc1); +// tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + +// // operations on doc2 still succeed +// doc2.set_bytes(author, "foo", "bar").await?; +// Ok(()) +// } + +// #[tokio::test] +// async fn test_doc_import_export() -> Result<()> { +// let _guard = iroh_test::logging::setup(); + +// let node = crate::node::Node::memory().enable_docs().spawn().await?; + +// // create temp file +// let temp_dir = tempfile::tempdir().context("tempdir")?; + +// let in_root = temp_dir.path().join("in"); +// tokio::fs::create_dir_all(in_root.clone()) +// .await +// .context("create dir all")?; +// let out_root = temp_dir.path().join("out"); + +// let path = in_root.join("test"); + +// let size = 100; +// let mut buf = vec![0u8; size]; +// rand::thread_rng().fill_bytes(&mut buf); +// let mut file = tokio::fs::File::create(path.clone()) +// .await +// .context("create file")?; +// file.write_all(&buf.clone()).await.context("write_all")?; +// file.flush().await.context("flush")?; + +// // create doc & author +// let client = node.client(); +// let doc = client.docs().create().await.context("doc create")?; +// let author = client.authors().create().await.context("author create")?; + +// // import file +// let import_outcome = doc +// .import_file( +// author, +// crate::util::fs::path_to_key(path.clone(), None, Some(in_root))?, +// path, +// true, +// ) +// .await +// .context("import file")? +// .finish() +// .await +// .context("import finish")?; + +// // export file +// let entry = doc +// .get_one(Query::author(author).key_exact(import_outcome.key)) +// .await +// .context("get one")? +// .unwrap(); +// let key = entry.key().to_vec(); +// let export_outcome = doc +// .export_file( +// entry, +// crate::util::fs::key_to_path(key, None, Some(out_root))?, +// ExportMode::Copy, +// ) +// .await +// .context("export file")? +// .finish() +// .await +// .context("export finish")?; + +// let got_bytes = tokio::fs::read(export_outcome.path) +// .await +// .context("tokio read")?; +// assert_eq!(buf, got_bytes); + +// Ok(()) +// } +// } diff --git a/src/rpc/docs_handle_request.rs b/src/rpc/docs_handle_request.rs new file mode 100644 index 0000000..889263c --- /dev/null +++ b/src/rpc/docs_handle_request.rs @@ -0,0 +1,593 @@ +use std::sync::{Arc, Mutex}; + +use anyhow::anyhow; +use futures_lite::{Stream, StreamExt}; +use iroh_blobs::{ + export::ExportProgress, + store::{ExportFormat, ImportProgress, Store as BaoStore}, + util::progress::{AsyncChannelProgressSender, ProgressSender}, + BlobFormat, HashAndFormat, +}; + +use super::{ + client::ShareMode, + proto::{ + // authors::{ + // CreateRequest, CreateResponse, DeleteRequest, DeleteResponse, ExportRequest, + // ExportResponse, GetDefaultRequest, GetDefaultResponse, ImportRequest, ImportResponse, + // ListRequest as AuthorListRequest, ListResponse as AuthorListResponse, + // SetDefaultRequest, SetDefaultResponse, + // }, + CloseRequest, + CloseResponse, + CreateRequest as DocCreateRequest, + CreateResponse as DocCreateResponse, + DelRequest, + DelResponse, + DocListRequest, + DocSubscribeRequest, + DocSubscribeResponse, + DropRequest, + DropResponse, + ExportFileRequest, + ExportFileResponse, + GetDownloadPolicyRequest, + GetDownloadPolicyResponse, + GetExactRequest, + GetExactResponse, + GetManyRequest, + GetManyResponse, + GetSyncPeersRequest, + GetSyncPeersResponse, + ImportFileRequest, + ImportFileResponse, + ImportRequest as DocImportRequest, + ImportResponse as DocImportResponse, + LeaveRequest, + LeaveResponse, + ListResponse as DocListResponse, + OpenRequest, + OpenResponse, + SetDownloadPolicyRequest, + SetDownloadPolicyResponse, + SetHashRequest, + SetHashResponse, + SetRequest, + SetResponse, + ShareRequest, + ShareResponse, + StartSyncRequest, + StartSyncResponse, + StatusRequest, + StatusResponse, + }, + RpcError, RpcResult, +}; +use crate::{engine::Engine, DocTicket, NamespaceSecret}; + +/// Capacity for the flume channels to forward sync store iterators to async RPC streams. +const ITER_CHANNEL_CAP: usize = 64; + +impl Engine { + // pub(super) async fn author_create(self, _req: CreateRequest) -> RpcResult { + // self.with_docs(|docs| async move { + // // TODO: pass rng + // let author = Author::new(&mut rand::rngs::OsRng {}); + // docs.sync + // .import_author(author.clone()) + // .await + // .map_err(|e| RpcError::new(&*e))?; + // Ok(CreateResponse { + // author_id: author.id(), + // }) + // }) + // .await + // } + + // pub(super) async fn author_default( + // self, + // _req: GetDefaultRequest, + // ) -> RpcResult { + // self.with_docs(|docs| async move { + // let author_id = docs.default_author.get(); + // Ok(GetDefaultResponse { author_id }) + // }) + // .await + // } + + // pub(super) async fn author_set_default( + // self, + // req: SetDefaultRequest, + // ) -> RpcResult { + // self.with_docs(|docs| async move { + // docs.default_author + // .set(req.author_id, &docs.sync) + // .await + // .map_err(|e| RpcError::new(&*e))?; + // Ok(SetDefaultResponse) + // }) + // .await + // } + + // pub(super) fn author_list( + // self, + // _req: AuthorListRequest, + // ) -> impl Stream> + Unpin { + // self.with_docs_stream(|docs| { + // let (tx, rx) = async_channel::bounded(ITER_CHANNEL_CAP); + // let sync = docs.sync.clone(); + // // we need to spawn a task to send our request to the sync handle, because the method + // // itself must be sync. + // tokio::task::spawn(async move { + // let tx2 = tx.clone(); + // if let Err(err) = sync.list_authors(tx).await { + // tx2.send(Err(err)).await.ok(); + // } + // }); + // rx.boxed().map(|r| { + // r.map(|author_id| AuthorListResponse { author_id }) + // .map_err(|e| RpcError::new(&*e)) + // }) + // }) + // } + + // pub(super) async fn author_import(self, req: ImportRequest) -> RpcResult { + // self.with_docs(|docs| async move { + // let author_id = docs + // .sync + // .import_author(req.author) + // .await + // .map_err(|e| RpcError::new(&*e))?; + // Ok(ImportResponse { author_id }) + // }) + // .await + // } + + // pub(super) async fn author_export(self, req: ExportRequest) -> RpcResult { + // self.with_docs(|docs| async move { + // let author = docs + // .sync + // .export_author(req.author) + // .await + // .map_err(|e| RpcError::new(&*e))?; + + // Ok(ExportResponse { author }) + // }) + // .await + // } + + // pub(super) async fn author_delete(self, req: DeleteRequest) -> RpcResult { + // self.with_docs(|docs| async move { + // if req.author == docs.default_author.get() { + // return Err(RpcError::new(&*anyhow!( + // "Deleting the default author is not supported" + // ))); + // } + // docs.sync + // .delete_author(req.author) + // .await + // .map_err(|e| RpcError::new(&*e))?; + // Ok(DeleteResponse) + // }) + // .await + // } + + pub(super) async fn doc_create(self, _req: DocCreateRequest) -> RpcResult { + let namespace = NamespaceSecret::new(&mut rand::rngs::OsRng {}); + let id = namespace.id(); + self.sync + .import_namespace(namespace.into()) + .await + .map_err(|e| RpcError::new(&*e))?; + self.sync + .open(id, Default::default()) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(DocCreateResponse { id }) + } + + pub(super) async fn doc_drop(self, req: DropRequest) -> RpcResult { + let DropRequest { doc_id } = req; + self.leave(doc_id, true) + .await + .map_err(|e| RpcError::new(&*e))?; + self.sync + .drop_replica(doc_id) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(DropResponse {}) + } + + pub(super) fn doc_list( + self, + _req: DocListRequest, + ) -> impl Stream> + Unpin { + let (tx, rx) = async_channel::bounded(ITER_CHANNEL_CAP); + let sync = self.sync.clone(); + // we need to spawn a task to send our request to the sync handle, because the method + // itself must be sync. + tokio::task::spawn(async move { + let tx2 = tx.clone(); + if let Err(err) = sync.list_replicas(tx).await { + tx2.send(Err(err)).await.ok(); + } + }); + rx.boxed().map(|r| { + r.map(|(id, capability)| DocListResponse { id, capability }) + .map_err(|e| RpcError::new(&*e)) + }) + } + + pub(super) async fn doc_open(self, req: OpenRequest) -> RpcResult { + self.sync + .open(req.doc_id, Default::default()) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(OpenResponse {}) + } + + pub(super) async fn doc_close(self, req: CloseRequest) -> RpcResult { + self.sync + .close(req.doc_id) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(CloseResponse {}) + } + + pub(super) async fn doc_status(self, req: StatusRequest) -> RpcResult { + let status = self + .sync + .get_state(req.doc_id) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(StatusResponse { status }) + } + + pub(super) async fn doc_share(self, req: ShareRequest) -> RpcResult { + let ShareRequest { + doc_id, + mode, + addr_options, + } = req; + let mut me = self + .endpoint + .node_addr() + .await + .map_err(|e| RpcError::new(&*e))?; + me.apply_options(addr_options); + + let capability = match mode { + ShareMode::Read => crate::Capability::Read(doc_id), + ShareMode::Write => { + let secret = self + .sync + .export_secret_key(doc_id) + .await + .map_err(|e| RpcError::new(&*e))?; + crate::Capability::Write(secret) + } + }; + self.start_sync(doc_id, vec![]) + .await + .map_err(|e| RpcError::new(&*e))?; + + Ok(ShareResponse(DocTicket { + capability, + nodes: vec![me], + })) + } + + pub(super) async fn doc_subscribe( + self, + req: DocSubscribeRequest, + ) -> RpcResult>> { + let stream = self + .subscribe(req.doc_id) + .await + .map_err(|e| RpcError::new(&*e))?; + + Ok(stream.map(|el| { + el.map(|event| DocSubscribeResponse { event }) + .map_err(|e| RpcError::new(&*e)) + })) + } + + pub(super) async fn doc_import(self, req: DocImportRequest) -> RpcResult { + let DocImportRequest { capability } = req; + let doc_id = self + .sync + .import_namespace(capability) + .await + .map_err(|e| RpcError::new(&*e))?; + self.sync + .open(doc_id, Default::default()) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(DocImportResponse { doc_id }) + } + + pub(super) async fn doc_start_sync( + self, + req: StartSyncRequest, + ) -> RpcResult { + let StartSyncRequest { doc_id, peers } = req; + self.start_sync(doc_id, peers) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(StartSyncResponse {}) + } + + pub(super) async fn doc_leave(self, req: LeaveRequest) -> RpcResult { + let LeaveRequest { doc_id } = req; + self.leave(doc_id, false) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(LeaveResponse {}) + } + + pub(super) async fn doc_set(self, req: SetRequest) -> RpcResult { + todo!() + // let blobs_store = self.blobs_store(); + // let SetRequest { + // doc_id, + // author_id, + // key, + // value, + // } = req; + // let len = value.len(); + // let tag = blobs_store + // .import_bytes(value, BlobFormat::Raw) + // .await + // .map_err(|e| RpcError::new(&e))?; + // self.sync + // .insert_local(doc_id, author_id, key.clone(), *tag.hash(), len as u64) + // .await + // .map_err(|e| RpcError::new(&*e))?; + // let entry = self + // .sync + // .get_exact(doc_id, author_id, key, false) + // .await + // .map_err(|e| RpcError::new(&*e))? + // .ok_or_else(|| RpcError::new(&*anyhow!("failed to get entry after insertion")))?; + // Ok(SetResponse { entry }) + } + + pub(super) async fn doc_del(self, req: DelRequest) -> RpcResult { + let DelRequest { + doc_id, + author_id, + prefix, + } = req; + let removed = self + .sync + .delete_prefix(doc_id, author_id, prefix) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(DelResponse { removed }) + } + + pub(super) async fn doc_set_hash(self, req: SetHashRequest) -> RpcResult { + let SetHashRequest { + doc_id, + author_id, + key, + hash, + size, + } = req; + self.sync + .insert_local(doc_id, author_id, key.clone(), hash, size) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(SetHashResponse {}) + } + + pub(super) fn doc_get_many( + self, + req: GetManyRequest, + ) -> impl Stream> + Unpin { + let GetManyRequest { doc_id, query } = req; + let (tx, rx) = async_channel::bounded(ITER_CHANNEL_CAP); + let sync = self.sync.clone(); + // we need to spawn a task to send our request to the sync handle, because the method + // itself must be sync. + tokio::task::spawn(async move { + let tx2 = tx.clone(); + if let Err(err) = sync.get_many(doc_id, query, tx).await { + tx2.send(Err(err)).await.ok(); + } + }); + rx.boxed().map(|r| { + r.map(|entry| GetManyResponse { entry }) + .map_err(|e| RpcError::new(&*e)) + }) + } + + pub(super) async fn doc_get_exact(self, req: GetExactRequest) -> RpcResult { + let GetExactRequest { + doc_id, + author, + key, + include_empty, + } = req; + let entry = self + .sync + .get_exact(doc_id, author, key, include_empty) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(GetExactResponse { entry }) + } + + pub(super) async fn doc_set_download_policy( + self, + req: SetDownloadPolicyRequest, + ) -> RpcResult { + self.sync + .set_download_policy(req.doc_id, req.policy) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(SetDownloadPolicyResponse {}) + } + + pub(super) async fn doc_get_download_policy( + self, + req: GetDownloadPolicyRequest, + ) -> RpcResult { + let policy = self + .sync + .get_download_policy(req.doc_id) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(GetDownloadPolicyResponse { policy }) + } + + pub(super) async fn doc_get_sync_peers( + self, + req: GetSyncPeersRequest, + ) -> RpcResult { + let peers = self + .sync + .get_sync_peers(req.doc_id) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(GetSyncPeersResponse { peers }) + } + + pub(super) fn doc_import_file( + self, + msg: ImportFileRequest, + ) -> impl Stream { + // provide a little buffer so that we don't slow down the sender + let (tx, rx) = async_channel::bounded(32); + let tx2 = tx.clone(); + let this = self.clone(); + self.local_pool_handle().spawn_detached(|| async move { + if let Err(e) = this.doc_import_file0(msg, tx).await { + tx2.send(super::client::ImportProgress::Abort(RpcError::new(&*e))) + .await + .ok(); + } + }); + rx.map(ImportFileResponse) + } + + async fn doc_import_file0( + self, + msg: ImportFileRequest, + progress: async_channel::Sender, + ) -> anyhow::Result<()> { + use std::collections::BTreeMap; + + use iroh_blobs::store::ImportMode; + + use super::client::ImportProgress as DocImportProgress; + + let progress = AsyncChannelProgressSender::new(progress); + let names = Arc::new(Mutex::new(BTreeMap::new())); + // convert import progress to provide progress + let import_progress = progress.clone().with_filter_map(move |x| match x { + ImportProgress::Found { id, name } => { + names.lock().unwrap().insert(id, name); + None + } + ImportProgress::Size { id, size } => { + let name = names.lock().unwrap().remove(&id)?; + Some(DocImportProgress::Found { id, name, size }) + } + ImportProgress::OutboardProgress { id, offset } => { + Some(DocImportProgress::Progress { id, offset }) + } + ImportProgress::OutboardDone { hash, id } => { + Some(DocImportProgress::IngestDone { hash, id }) + } + _ => None, + }); + let ImportFileRequest { + doc_id, + author_id, + key, + path: root, + in_place, + } = msg; + // Check that the path is absolute and exists. + anyhow::ensure!(root.is_absolute(), "path must be absolute"); + anyhow::ensure!( + root.exists(), + "trying to add missing path: {}", + root.display() + ); + + let import_mode = match in_place { + true => ImportMode::TryReference, + false => ImportMode::Copy, + }; + + todo!() + // let blobs = self.blobs(); + // let (temp_tag, size) = blobs + // .store() + // .import_file(root, import_mode, BlobFormat::Raw, import_progress) + // .await?; + + // let hash_and_format = temp_tag.inner(); + // let HashAndFormat { hash, .. } = *hash_and_format; + // self.doc_set_hash(SetHashRequest { + // doc_id, + // author_id, + // key: key.clone(), + // hash, + // size, + // }) + // .await?; + // drop(temp_tag); + // progress.send(DocImportProgress::AllDone { key }).await?; + // Ok(()) + } + + pub(super) fn doc_export_file( + self, + msg: ExportFileRequest, + ) -> impl Stream { + let (tx, rx) = async_channel::bounded(1024); + let tx2 = tx.clone(); + let this = self.clone(); + self.local_pool_handle().spawn_detached(|| async move { + if let Err(e) = this.doc_export_file0(msg, tx).await { + tx2.send(ExportProgress::Abort(RpcError::new(&*e))) + .await + .ok(); + } + }); + rx.map(ExportFileResponse) + } + + async fn doc_export_file0( + self, + msg: ExportFileRequest, + progress: async_channel::Sender, + ) -> anyhow::Result<()> { + let progress = AsyncChannelProgressSender::new(progress); + let ExportFileRequest { entry, path, mode } = msg; + let key = bytes::Bytes::from(entry.key().to_vec()); + let export_progress = progress.clone().with_map(move |mut x| { + // assign the doc key to the `meta` field of the initial progress event + if let ExportProgress::Found { meta, .. } = &mut x { + *meta = Some(key.clone()) + } + x + }); + + todo!() + // let blobs = self.blobs(); + // iroh_blobs::export::export( + // blobs.store(), + // entry.content_hash(), + // path, + // ExportFormat::Blob, + // mode, + // export_progress, + // ) + // .await?; + // progress.send(ExportProgress::AllDone).await?; + // Ok(()) + } +} diff --git a/src/rpc/proto.rs b/src/rpc/proto.rs new file mode 100644 index 0000000..d646115 --- /dev/null +++ b/src/rpc/proto.rs @@ -0,0 +1,435 @@ +//! Protocol definitions for RPC. + +use std::path::PathBuf; + +use bytes::Bytes; +use iroh_base::node_addr::AddrInfoOptions; +use iroh_blobs::{export::ExportProgress, store::ExportMode, Hash}; +use iroh_net::NodeAddr; +use nested_enum_utils::enum_conversions; +use quic_rpc::pattern::try_server_streaming::StreamCreated; +use quic_rpc_derive::rpc_requests; +use serde::{Deserialize, Serialize}; + +use super::{ + client::{ImportProgress, ShareMode}, + RpcError, RpcResult, +}; +use crate::{ + actor::OpenState, + engine::LiveEvent, + store::{DownloadPolicy, Query}, + AuthorId, Capability, CapabilityKind, DocTicket, Entry, NamespaceId, PeerIdBytes, SignedEntry, +}; + +/// The RPC service type for the docs protocol. +#[derive(Debug, Clone)] +pub struct RpcService; + +impl quic_rpc::Service for RpcService { + type Req = Request; + type Res = Response; +} + +#[allow(missing_docs)] +#[derive(strum::Display, Debug, Serialize, Deserialize)] +#[enum_conversions] +#[rpc_requests(RpcService)] +pub enum Request { + #[rpc(response = RpcResult)] + Open(OpenRequest), + #[rpc(response = RpcResult)] + Close(CloseRequest), + #[rpc(response = RpcResult)] + Status(StatusRequest), + #[server_streaming(response = RpcResult)] + List(DocListRequest), + #[rpc(response = RpcResult)] + Create(CreateRequest), + #[rpc(response = RpcResult)] + Drop(DropRequest), + #[rpc(response = RpcResult)] + Import(ImportRequest), + #[rpc(response = RpcResult)] + Set(SetRequest), + #[rpc(response = RpcResult)] + SetHash(SetHashRequest), + #[server_streaming(response = RpcResult)] + Get(GetManyRequest), + #[rpc(response = RpcResult)] + GetExact(GetExactRequest), + #[server_streaming(response = ImportFileResponse)] + ImportFile(ImportFileRequest), + #[server_streaming(response = ExportFileResponse)] + ExportFile(ExportFileRequest), + #[rpc(response = RpcResult)] + Del(DelRequest), + #[rpc(response = RpcResult)] + StartSync(StartSyncRequest), + #[rpc(response = RpcResult)] + Leave(LeaveRequest), + #[rpc(response = RpcResult)] + Share(ShareRequest), + #[try_server_streaming(create_error = RpcError, item_error = RpcError, item = DocSubscribeResponse)] + Subscribe(DocSubscribeRequest), + #[rpc(response = RpcResult)] + GetDownloadPolicy(GetDownloadPolicyRequest), + #[rpc(response = RpcResult)] + SetDownloadPolicy(SetDownloadPolicyRequest), + #[rpc(response = RpcResult)] + GetSyncPeers(GetSyncPeersRequest), +} + +#[allow(missing_docs)] +#[derive(strum::Display, Debug, Serialize, Deserialize)] +#[enum_conversions] +pub enum Response { + Open(RpcResult), + Close(RpcResult), + Status(RpcResult), + List(RpcResult), + Create(RpcResult), + Drop(RpcResult), + Import(RpcResult), + Set(RpcResult), + SetHash(RpcResult), + Get(RpcResult), + GetExact(RpcResult), + ImportFile(ImportFileResponse), + ExportFile(ExportFileResponse), + Del(RpcResult), + Share(RpcResult), + StartSync(RpcResult), + Leave(RpcResult), + Subscribe(RpcResult), + GetDownloadPolicy(RpcResult), + SetDownloadPolicy(RpcResult), + GetSyncPeers(RpcResult), + StreamCreated(RpcResult), +} + +/// Subscribe to events for a document. +#[derive(Serialize, Deserialize, Debug)] +pub struct DocSubscribeRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`DocSubscribeRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct DocSubscribeResponse { + /// The event that occurred on the document + pub event: LiveEvent, +} + +/// List all documents +#[derive(Serialize, Deserialize, Debug)] +pub struct DocListRequest {} + +/// Response to [`DocListRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct ListResponse { + /// The document id + pub id: NamespaceId, + /// The capability over the document. + pub capability: CapabilityKind, +} + +/// Create a new document +#[derive(Serialize, Deserialize, Debug)] +pub struct CreateRequest {} + +/// Response to [`CreateRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct CreateResponse { + /// The document id + pub id: NamespaceId, +} + +/// Import a document from a capability. +#[derive(Serialize, Deserialize, Debug)] +pub struct ImportRequest { + /// The namespace capability. + pub capability: Capability, +} + +/// Response to [`ImportRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct ImportResponse { + /// the document id + pub doc_id: NamespaceId, +} + +/// Share a document with peers over a ticket. +#[derive(Serialize, Deserialize, Debug)] +pub struct ShareRequest { + /// The document id + pub doc_id: NamespaceId, + /// Whether to share read or write access to the document + pub mode: ShareMode, + /// Configuration of the addresses in the ticket. + pub addr_options: AddrInfoOptions, +} + +/// The response to [`ShareRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct ShareResponse(pub DocTicket); + +/// Get info on a document +#[derive(Serialize, Deserialize, Debug)] +pub struct StatusRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`StatusRequest`] +// TODO: actually provide info +#[derive(Serialize, Deserialize, Debug)] +pub struct StatusResponse { + /// Live sync status + pub status: OpenState, +} + +/// Open a document +#[derive(Serialize, Deserialize, Debug)] +pub struct OpenRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`OpenRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct OpenResponse {} + +/// Open a document +#[derive(Serialize, Deserialize, Debug)] +pub struct CloseRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`CloseRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct CloseResponse {} + +/// Start to sync a doc with peers. +#[derive(Serialize, Deserialize, Debug)] +pub struct StartSyncRequest { + /// The document id + pub doc_id: NamespaceId, + /// List of peers to join + pub peers: Vec, +} + +/// Response to [`StartSyncRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct StartSyncResponse {} + +/// Stop the live sync for a doc, and optionally delete the document. +#[derive(Serialize, Deserialize, Debug)] +pub struct LeaveRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`LeaveRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct LeaveResponse {} + +/// Stop the live sync for a doc, and optionally delete the document. +#[derive(Serialize, Deserialize, Debug)] +pub struct DropRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`DropRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct DropResponse {} + +/// Set an entry in a document +#[derive(Serialize, Deserialize, Debug)] +pub struct SetRequest { + /// The document id + pub doc_id: NamespaceId, + /// Author of this entry. + pub author_id: AuthorId, + /// Key of this entry. + pub key: Bytes, + /// Value of this entry. + // TODO: Allow to provide the hash directly + // TODO: Add a way to provide content as stream + pub value: Bytes, +} + +/// Response to [`SetRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct SetResponse { + /// The newly-created entry. + pub entry: SignedEntry, +} + +/// A request to the node to add the data at the given filepath as an entry to the document +/// +/// Will produce a stream of [`ImportProgress`] messages. +#[derive(Debug, Serialize, Deserialize)] +pub struct ImportFileRequest { + /// The document id + pub doc_id: NamespaceId, + /// Author of this entry. + pub author_id: AuthorId, + /// Key of this entry. + pub key: Bytes, + /// The filepath to the data + /// + /// This should be an absolute path valid for the file system on which + /// the node runs. Usually the cli will run on the same machine as the + /// node, so this should be an absolute path on the cli machine. + pub path: PathBuf, + /// True if the provider can assume that the data will not change, so it + /// can be shared in place. + pub in_place: bool, +} + +/// Wrapper around [`ImportProgress`]. +#[derive(Debug, Serialize, Deserialize, derive_more::Into)] +pub struct ImportFileResponse(pub ImportProgress); + +/// A request to the node to save the data of the entry to the given filepath +/// +/// Will produce a stream of [`ExportFileResponse`] messages. +#[derive(Debug, Serialize, Deserialize)] +pub struct ExportFileRequest { + /// The entry you want to export + pub entry: Entry, + /// The filepath to where the data should be saved + /// + /// This should be an absolute path valid for the file system on which + /// the node runs. Usually the cli will run on the same machine as the + /// node, so this should be an absolute path on the cli machine. + pub path: PathBuf, + /// The mode of exporting. Setting to `ExportMode::TryReference` means attempting + /// to use references for keeping file + pub mode: ExportMode, +} + +/// Progress messages for an doc export operation +/// +/// An export operation involves reading the entry from the database ans saving the entry to the +/// given `outpath` +#[derive(Debug, Serialize, Deserialize, derive_more::Into)] +pub struct ExportFileResponse(pub ExportProgress); + +/// Delete entries in a document +#[derive(Serialize, Deserialize, Debug)] +pub struct DelRequest { + /// The document id. + pub doc_id: NamespaceId, + /// Author of this entry. + pub author_id: AuthorId, + /// Prefix to delete. + pub prefix: Bytes, +} + +/// Response to [`DelRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct DelResponse { + /// The number of entries that were removed. + pub removed: usize, +} + +/// Set an entry in a document via its hash +#[derive(Serialize, Deserialize, Debug)] +pub struct SetHashRequest { + /// The document id + pub doc_id: NamespaceId, + /// Author of this entry. + pub author_id: AuthorId, + /// Key of this entry. + pub key: Bytes, + /// Hash of this entry. + pub hash: Hash, + /// Size of this entry. + pub size: u64, +} + +/// Response to [`SetHashRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct SetHashResponse {} + +/// Get entries from a document +#[derive(Serialize, Deserialize, Debug)] +pub struct GetManyRequest { + /// The document id + pub doc_id: NamespaceId, + /// Query to run + pub query: Query, +} + +/// Response to [`GetManyRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct GetManyResponse { + /// The document entry + pub entry: SignedEntry, +} + +/// Get entries from a document +#[derive(Serialize, Deserialize, Debug)] +pub struct GetExactRequest { + /// The document id + pub doc_id: NamespaceId, + /// Key matcher + pub key: Bytes, + /// Author matcher + pub author: AuthorId, + /// Whether to include empty entries (prefix deletion markers) + pub include_empty: bool, +} + +/// Response to [`GetExactRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct GetExactResponse { + /// The document entry + pub entry: Option, +} + +/// Set a download policy +#[derive(Serialize, Deserialize, Debug)] +pub struct SetDownloadPolicyRequest { + /// The document id + pub doc_id: NamespaceId, + /// Download policy + pub policy: DownloadPolicy, +} + +/// Response to [`SetDownloadPolicyRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct SetDownloadPolicyResponse {} + +/// Get a download policy +#[derive(Serialize, Deserialize, Debug)] +pub struct GetDownloadPolicyRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`GetDownloadPolicyRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct GetDownloadPolicyResponse { + /// The download policy + pub policy: DownloadPolicy, +} + +/// Get peers for document +#[derive(Serialize, Deserialize, Debug)] +pub struct GetSyncPeersRequest { + /// The document id + pub doc_id: NamespaceId, +} + +/// Response to [`GetSyncPeersRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct GetSyncPeersResponse { + /// List of peers ids + pub peers: Option>, +} From a761c374de391da3181f27398f2c97da30c1469f Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Wed, 30 Oct 2024 19:13:39 +0100 Subject: [PATCH 02/26] add new method --- src/rpc/client.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/rpc/client.rs b/src/rpc/client.rs index 9590437..b4a80ca 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -49,7 +49,12 @@ where S: quic_rpc::Service, C: quic_rpc::ServiceConnection, { - /// Creates a new document. + /// Creates a new docs client. + pub fn new(rpc: quic_rpc::RpcClient) -> Self { + Self { rpc } + } + + /// Creates a client. pub async fn create(&self) -> Result> { let res = self.rpc.rpc(CreateRequest {}).await??; let doc = Doc::new(self.rpc.clone(), res.id); From 70c565f03cb85f5a581e3f53e6366ce3b67b14ee Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 31 Oct 2024 11:48:36 +0100 Subject: [PATCH 03/26] minimal test working --- Cargo.lock | 9 +- Cargo.toml | 1 + src/rpc/client.rs | 356 ++++++++++++++++++++++++++++------------------ 3 files changed, 222 insertions(+), 144 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6be02e7..566116a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1759,8 +1759,7 @@ dependencies = [ [[package]] name = "iroh-gossip" version = "0.27.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b434d455389493ff2b2ecbab035c12eb3762f24d04080855ecd4956bf7739448" +source = "git+https://github.com/n0-computer/iroh-gossip?branch=main#77d392cc68a06024addb4b4162d18f0709bab3ab" dependencies = [ "anyhow", "async-channel", @@ -1775,10 +1774,16 @@ dependencies = [ "iroh-blake3", "iroh-metrics", "iroh-net", + "iroh-router", + "nested_enum_utils", "postcard", + "quic-rpc", + "quic-rpc-derive", "rand", "rand_core", "serde", + "serde-error", + "strum", "tokio", "tokio-util", "tracing", diff --git a/Cargo.toml b/Cargo.toml index 0981d13..dde40d7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -97,3 +97,4 @@ iroh-net = { git = "https://github.com/n0-computer/iroh", branch = "main" } iroh-metrics = { git = "https://github.com/n0-computer/iroh", branch = "main" } iroh-base = { git = "https://github.com/n0-computer/iroh", branch = "main" } iroh-blobs = { git = "https://github.com/n0-computer/iroh-blobs", branch = "main" } +iroh-gossip = { git = "https://github.com/n0-computer/iroh-gossip", branch = "main" } diff --git a/src/rpc/client.rs b/src/rpc/client.rs index b4a80ca..22a687b 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -501,24 +501,6 @@ impl Entry { pub fn timestamp(&self) -> u64 { self.0.timestamp() } - - // TODO: depends on an rpc client in iroh-blobs - // /// Reads the content of an [`Entry`] as a streaming [`blobs::Reader`]. - // /// - // /// You can pass either a [`Doc`] or the `Iroh` client by reference as `client`. - // pub async fn content_reader(&self, client: impl Into<&Client>) -> Result { - // blobs::Reader::from_rpc_read(client.into(), self.content_hash()).await - // } - - // /// Reads all content of an [`Entry`] into a buffer. - // /// - // /// You can pass either a [`Doc`] or the `Iroh` client by reference as `client`. - // pub async fn content_bytes(&self, client: impl Into<&Client>) -> Result { - // blobs::Reader::from_rpc_read(client.into(), self.content_hash()) - // .await? - // .read_to_bytes() - // .await - // } } /// Progress messages for an doc import operation @@ -792,127 +774,217 @@ where }) } -// TODO -// #[cfg(test)] -// mod tests { -// use rand::RngCore; -// use tokio::io::AsyncWriteExt; - -// use super::*; - -// #[tokio::test] -// async fn test_drop_doc_client_sync() -> Result<()> { -// let _guard = iroh_test::logging::setup(); - -// let node = crate::node::Node::memory().enable_docs().spawn().await?; - -// let client = node.client(); -// let doc = client.docs().create().await?; - -// let res = std::thread::spawn(move || { -// drop(doc); -// drop(node); -// }); - -// tokio::task::spawn_blocking(move || res.join().map_err(|e| anyhow::anyhow!("{:?}", e))) -// .await??; - -// Ok(()) -// } - -// /// Test that closing a doc does not close other instances. -// #[tokio::test] -// async fn test_doc_close() -> Result<()> { -// let _guard = iroh_test::logging::setup(); - -// let node = crate::node::Node::memory().enable_docs().spawn().await?; -// let author = node.authors().default().await?; -// // open doc two times -// let doc1 = node.docs().create().await?; -// let doc2 = node.docs().open(doc1.id()).await?.expect("doc to exist"); -// // close doc1 instance -// doc1.close().await?; -// // operations on doc1 now fail. -// assert!(doc1.set_bytes(author, "foo", "bar").await.is_err()); -// // dropping doc1 will close the doc if not already closed -// // wait a bit because the close-on-drop spawns a task for which we cannot track completion. -// drop(doc1); -// tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; - -// // operations on doc2 still succeed -// doc2.set_bytes(author, "foo", "bar").await?; -// Ok(()) -// } - -// #[tokio::test] -// async fn test_doc_import_export() -> Result<()> { -// let _guard = iroh_test::logging::setup(); - -// let node = crate::node::Node::memory().enable_docs().spawn().await?; - -// // create temp file -// let temp_dir = tempfile::tempdir().context("tempdir")?; - -// let in_root = temp_dir.path().join("in"); -// tokio::fs::create_dir_all(in_root.clone()) -// .await -// .context("create dir all")?; -// let out_root = temp_dir.path().join("out"); - -// let path = in_root.join("test"); - -// let size = 100; -// let mut buf = vec![0u8; size]; -// rand::thread_rng().fill_bytes(&mut buf); -// let mut file = tokio::fs::File::create(path.clone()) -// .await -// .context("create file")?; -// file.write_all(&buf.clone()).await.context("write_all")?; -// file.flush().await.context("flush")?; - -// // create doc & author -// let client = node.client(); -// let doc = client.docs().create().await.context("doc create")?; -// let author = client.authors().create().await.context("author create")?; - -// // import file -// let import_outcome = doc -// .import_file( -// author, -// crate::util::fs::path_to_key(path.clone(), None, Some(in_root))?, -// path, -// true, -// ) -// .await -// .context("import file")? -// .finish() -// .await -// .context("import finish")?; - -// // export file -// let entry = doc -// .get_one(Query::author(author).key_exact(import_outcome.key)) -// .await -// .context("get one")? -// .unwrap(); -// let key = entry.key().to_vec(); -// let export_outcome = doc -// .export_file( -// entry, -// crate::util::fs::key_to_path(key, None, Some(out_root))?, -// ExportMode::Copy, -// ) -// .await -// .context("export file")? -// .finish() -// .await -// .context("export finish")?; - -// let got_bytes = tokio::fs::read(export_outcome.path) -// .await -// .context("tokio read")?; -// assert_eq!(buf, got_bytes); - -// Ok(()) -// } -// } +#[cfg(test)] +mod tests { + use iroh_blobs::util::local_pool::LocalPool; + use iroh_gossip::net::GOSSIP_ALPN; + use rand::RngCore; + use tokio::io::AsyncWriteExt; + use tracing::warn; + + use crate::{ + engine::{DefaultAuthorStorage, Engine}, + net::DOCS_ALPN, + }; + + use super::*; + + async fn setup_router() -> Result<(Client, iroh_router::Router, tokio::task::JoinHandle>)> + { + let endpoint = iroh_net::Endpoint::builder().bind().await?; + let local_pool = LocalPool::single(); + let mut router = iroh_router::Router::builder(endpoint.clone()); + + // Setup gossip + let my_addr = endpoint.node_addr().await?; + let gossip = iroh_gossip::net::Gossip::from_endpoint( + endpoint.clone(), + Default::default(), + &my_addr.info, + ); + router = router.accept(GOSSIP_ALPN.to_vec(), Arc::new(gossip.clone())); + + // Setup blobs + + let bao_store = iroh_blobs::store::mem::Store::new(); + let downloader = iroh_blobs::downloader::Downloader::new( + bao_store.clone(), + endpoint.clone(), + local_pool.handle().clone(), + ); + let blobs = iroh_blobs::net_protocol::Blobs::new_with_events( + bao_store.clone(), + local_pool.handle().clone(), + Default::default(), + downloader.clone(), + ); + router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), Arc::new(blobs)); + + // Setup docs + + let replica_store = crate::store::Store::memory(); + let engine = Engine::spawn( + endpoint.clone(), + gossip, + replica_store, + bao_store, + downloader, + DefaultAuthorStorage::Mem, + local_pool.handle().clone(), + ) + .await?; + + router = router.accept(DOCS_ALPN.to_vec(), Arc::new(engine.clone())); + + // Build the router + + let router = router.spawn().await?; + + // Setup RPC + let (internal_rpc, controller) = + quic_rpc::transport::flume::service_connection::(32); + let controller = quic_rpc::transport::boxed::Connection::new(controller); + let internal_rpc = quic_rpc::transport::boxed::ServerEndpoint::new(internal_rpc); + let internal_rpc = quic_rpc::RpcServer::new(internal_rpc); + + let rpc_server_task = tokio::task::spawn(async move { + loop { + let request = internal_rpc.accept().await; + match request { + Ok(accepting) => { + let engine = engine.clone(); + tokio::task::spawn(async move { + let (msg, chan) = accepting.read_first().await.unwrap(); + engine.handle_rpc_request(msg, chan).await.unwrap(); + }); + } + Err(err) => { + warn!("rpc error: {:?}", err); + } + } + } + }); + + let docs_client = Client::new(quic_rpc::RpcClient::new(controller.clone())); + + Ok((docs_client, router, rpc_server_task)) + } + + #[tokio::test] + async fn test_drop_doc_client_sync() -> Result<()> { + let _guard = iroh_test::logging::setup(); + + let (docs_client, router, rpc_server_task) = setup_router().await?; + + let doc = docs_client.create().await?; + + let res = std::thread::spawn(move || { + drop(doc); + }); + + tokio::task::spawn_blocking(move || res.join().map_err(|e| anyhow::anyhow!("{:?}", e))) + .await??; + + rpc_server_task.abort(); + router.shutdown().await?; + + Ok(()) + } + + // /// Test that closing a doc does not close other instances. + // #[tokio::test] + // async fn test_doc_close() -> Result<()> { + // let _guard = iroh_test::logging::setup(); + + // let node = iroh::node::Node::memory().enable_docs().spawn().await?; + // let author = node.authors().default().await?; + // // open doc two times + // let doc1 = node.docs().create().await?; + // let doc2 = node.docs().open(doc1.id()).await?.expect("doc to exist"); + // // close doc1 instance + // doc1.close().await?; + // // operations on doc1 now fail. + // assert!(doc1.set_bytes(author, "foo", "bar").await.is_err()); + // // dropping doc1 will close the doc if not already closed + // // wait a bit because the close-on-drop spawns a task for which we cannot track completion. + // drop(doc1); + // tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + // // operations on doc2 still succeed + // doc2.set_bytes(author, "foo", "bar").await?; + // Ok(()) + // } + + // #[tokio::test] + // async fn test_doc_import_export() -> Result<()> { + // let _guard = iroh_test::logging::setup(); + + // let node = iroh::node::Node::memory().enable_docs().spawn().await?; + + // // create temp file + // let temp_dir = tempfile::tempdir().context("tempdir")?; + + // let in_root = temp_dir.path().join("in"); + // tokio::fs::create_dir_all(in_root.clone()) + // .await + // .context("create dir all")?; + // let out_root = temp_dir.path().join("out"); + + // let path = in_root.join("test"); + + // let size = 100; + // let mut buf = vec![0u8; size]; + // rand::thread_rng().fill_bytes(&mut buf); + // let mut file = tokio::fs::File::create(path.clone()) + // .await + // .context("create file")?; + // file.write_all(&buf.clone()).await.context("write_all")?; + // file.flush().await.context("flush")?; + + // // create doc & author + // let client = node.rpc_client().clone(); + // let docs_client = Client::from_service(client); + // let doc = docs_client.create().await.context("doc create")?; + // let author = client.authors().create().await.context("author create")?; + + // // import file + // let import_outcome = doc + // .import_file( + // author, + // iroh::util::fs::path_to_key(path.clone(), None, Some(in_root))?, + // path, + // true, + // ) + // .await + // .context("import file")? + // .finish() + // .await + // .context("import finish")?; + + // // export file + // let entry = doc + // .get_one(Query::author(author).key_exact(import_outcome.key)) + // .await + // .context("get one")? + // .unwrap(); + // let key = entry.key().to_vec(); + // let export_outcome = doc + // .export_file( + // entry, + // iroh::util::fs::key_to_path(key, None, Some(out_root))?, + // ExportMode::Copy, + // ) + // .await + // .context("export file")? + // .finish() + // .await + // .context("export finish")?; + + // let got_bytes = tokio::fs::read(export_outcome.path) + // .await + // .context("tokio read")?; + // assert_eq!(buf, got_bytes); + + // Ok(()) + // } +} From 558ca8ab39446d1f628b32f33c67058fbfb63e65 Mon Sep 17 00:00:00 2001 From: dignifiedquire Date: Thu, 31 Oct 2024 12:55:04 +0100 Subject: [PATCH 04/26] extract authors --- src/rpc.rs | 8 + src/rpc/client.rs | 265 +++++- src/rpc/docs_handle_request.rs | 263 +++--- src/rpc/proto.rs | 107 ++- tests/gc.rs | 40 + tests/sync.rs | 1405 ++++++++++++++++++++++++++++++++ 6 files changed, 1925 insertions(+), 163 deletions(-) create mode 100644 tests/gc.rs create mode 100644 tests/sync.rs diff --git a/src/rpc.rs b/src/rpc.rs index 3103777..9f1d9d2 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -51,6 +51,14 @@ impl Engine { SetDownloadPolicy(msg) => chan.rpc(msg, this, Self::doc_set_download_policy).await, GetDownloadPolicy(msg) => chan.rpc(msg, this, Self::doc_get_download_policy).await, GetSyncPeers(msg) => chan.rpc(msg, this, Self::doc_get_sync_peers).await, + + AuthorList(msg) => chan.server_streaming(msg, this, Self::author_list).await, + AuthorCreate(msg) => chan.rpc(msg, this, Self::author_create).await, + AuthorImport(msg) => chan.rpc(msg, this, Self::author_import).await, + AuthorExport(msg) => chan.rpc(msg, this, Self::author_export).await, + AuthorDelete(msg) => chan.rpc(msg, this, Self::author_delete).await, + AuthorGetDefault(msg) => chan.rpc(msg, this, Self::author_default).await, + AuthorSetDefault(msg) => chan.rpc(msg, this, Self::author_set_default).await, } } } diff --git a/src/rpc/client.rs b/src/rpc/client.rs index 22a687b..f15e375 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -23,9 +23,11 @@ use quic_rpc::{client::BoxedServiceConnection, message::RpcMsg}; use serde::{Deserialize, Serialize}; use super::proto::{ - CloseRequest, CreateRequest, DelRequest, DelResponse, DocListRequest, DocSubscribeRequest, - DropRequest, ExportFileRequest, GetDownloadPolicyRequest, GetExactRequest, GetManyRequest, - GetSyncPeersRequest, ImportFileRequest, ImportRequest, LeaveRequest, OpenRequest, RpcService, + AuthorCreateRequest, AuthorDeleteRequest, AuthorExportRequest, AuthorGetDefaultRequest, + AuthorImportRequest, AuthorListRequest, AuthorSetDefaultRequest, CloseRequest, CreateRequest, + DelRequest, DelResponse, DocListRequest, DocSubscribeRequest, DropRequest, ExportFileRequest, + GetDownloadPolicyRequest, GetExactRequest, GetManyRequest, GetSyncPeersRequest, + ImportFileRequest, ImportRequest, LeaveRequest, OpenRequest, RpcService, SetDownloadPolicyRequest, SetHashRequest, SetRequest, ShareRequest, StartSyncRequest, StatusRequest, }; @@ -34,8 +36,8 @@ pub use crate::engine::{Origin, SyncEvent, SyncReason}; use crate::{ actor::OpenState, store::{DownloadPolicy, Query}, - AuthorId, Capability, CapabilityKind, ContentStatus, DocTicket, NamespaceId, PeerIdBytes, - RecordIdentifier, + Author, AuthorId, Capability, CapabilityKind, ContentStatus, DocTicket, NamespaceId, + PeerIdBytes, RecordIdentifier, }; /// Iroh docs client. @@ -120,6 +122,75 @@ where let doc = Doc::new(self.rpc.clone(), id); Ok(Some(doc)) } + + /// Creates a new document author. + /// + /// You likely want to save the returned [`AuthorId`] somewhere so that you can use this author + /// again. + /// + /// If you need only a single author, use [`Self::default`]. + pub async fn author_create(&self) -> Result { + let res = self.rpc.rpc(AuthorCreateRequest).await??; + Ok(res.author_id) + } + + /// Returns the default document author of this node. + /// + /// On persistent nodes, the author is created on first start and its public key is saved + /// in the data directory. + /// + /// The default author can be set with [`Self::set_default`]. + pub async fn author_default(&self) -> Result { + let res = self.rpc.rpc(AuthorGetDefaultRequest).await??; + Ok(res.author_id) + } + + /// Sets the node-wide default author. + /// + /// If the author does not exist, an error is returned. + /// + /// On a persistent node, the author id will be saved to a file in the data directory and + /// reloaded after a restart. + pub async fn author_set_default(&self, author_id: AuthorId) -> Result<()> { + self.rpc + .rpc(AuthorSetDefaultRequest { author_id }) + .await??; + Ok(()) + } + + /// Lists document authors for which we have a secret key. + /// + /// It's only possible to create writes from authors that we have the secret key of. + pub async fn author_list(&self) -> Result>> { + let stream = self.rpc.server_streaming(AuthorListRequest {}).await?; + Ok(flatten(stream).map(|res| res.map(|res| res.author_id))) + } + + /// Exports the given author. + /// + /// Warning: The [`Author`] struct contains sensitive data. + pub async fn author_export(&self, author: AuthorId) -> Result> { + let res = self.rpc.rpc(AuthorExportRequest { author }).await??; + Ok(res.author) + } + + /// Imports the given author. + /// + /// Warning: The [`Author`] struct contains sensitive data. + pub async fn author_import(&self, author: Author) -> Result<()> { + self.rpc.rpc(AuthorImportRequest { author }).await??; + Ok(()) + } + + /// Deletes the given author by id. + /// + /// Warning: This permanently removes this author. + /// + /// Returns an error if attempting to delete the default author. + pub async fn author_delete(&self, author: AuthorId) -> Result<()> { + self.rpc.rpc(AuthorDeleteRequest { author }).await??; + Ok(()) + } } /// Document handle @@ -782,15 +853,17 @@ mod tests { use tokio::io::AsyncWriteExt; use tracing::warn; + use super::*; use crate::{ engine::{DefaultAuthorStorage, Engine}, net::DOCS_ALPN, }; - use super::*; - - async fn setup_router() -> Result<(Client, iroh_router::Router, tokio::task::JoinHandle>)> - { + async fn setup_router() -> Result<( + Client, + iroh_router::Router, + tokio::task::JoinHandle>, + )> { let endpoint = iroh_net::Endpoint::builder().bind().await?; let local_pool = LocalPool::single(); let mut router = iroh_router::Router::builder(endpoint.clone()); @@ -987,4 +1060,178 @@ mod tests { // Ok(()) // } + + // #[tokio::test] + // async fn test_authors() -> Result<()> { + // let node = Node::memory().enable_docs().spawn().await?; + + // // default author always exists + // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; + // assert_eq!(authors.len(), 1); + // let default_author = node.authors().default().await?; + // assert_eq!(authors, vec![default_author]); + + // let author_id = node.authors().create().await?; + + // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; + // assert_eq!(authors.len(), 2); + + // let author = node + // .authors() + // .export(author_id) + // .await? + // .expect("should have author"); + // node.authors().delete(author_id).await?; + // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; + // assert_eq!(authors.len(), 1); + + // node.authors().import(author).await?; + + // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; + // assert_eq!(authors.len(), 2); + + // assert!(node.authors().default().await? != author_id); + // node.authors().set_default(author_id).await?; + // assert_eq!(node.authors().default().await?, author_id); + + // Ok(()) + // } + + // #[tokio::test] + // async fn test_default_author_memory() -> Result<()> { + // let iroh = Node::memory().enable_docs().spawn().await?; + // let author = iroh.authors().default().await?; + // assert!(iroh.authors().export(author).await?.is_some()); + // assert!(iroh.authors().delete(author).await.is_err()); + // Ok(()) + // } + + // #[cfg(feature = "fs-store")] + // #[tokio::test] + // async fn test_default_author_persist() -> Result<()> { + // use crate::util::path::IrohPaths; + + // let _guard = iroh_test::logging::setup(); + + // let iroh_root_dir = tempfile::TempDir::new().unwrap(); + // let iroh_root = iroh_root_dir.path(); + + // // check that the default author exists and cannot be deleted. + // let default_author = { + // let iroh = Node::persistent(iroh_root) + // .await + // .unwrap() + // .enable_docs() + // .spawn() + // .await + // .unwrap(); + // let author = iroh.authors().default().await.unwrap(); + // assert!(iroh.authors().export(author).await.unwrap().is_some()); + // assert!(iroh.authors().delete(author).await.is_err()); + // iroh.shutdown().await.unwrap(); + // author + // }; + + // // check that the default author is persisted across restarts. + // { + // let iroh = Node::persistent(iroh_root) + // .await + // .unwrap() + // .enable_docs() + // .spawn() + // .await + // .unwrap(); + // let author = iroh.authors().default().await.unwrap(); + // assert_eq!(author, default_author); + // assert!(iroh.authors().export(author).await.unwrap().is_some()); + // assert!(iroh.authors().delete(author).await.is_err()); + // iroh.shutdown().await.unwrap(); + // }; + + // // check that a new default author is created if the default author file is deleted + // // manually. + // let default_author = { + // tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) + // .await + // .unwrap(); + // let iroh = Node::persistent(iroh_root) + // .await + // .unwrap() + // .enable_docs() + // .spawn() + // .await + // .unwrap(); + // let author = iroh.authors().default().await.unwrap(); + // assert!(author != default_author); + // assert!(iroh.authors().export(author).await.unwrap().is_some()); + // assert!(iroh.authors().delete(author).await.is_err()); + // iroh.shutdown().await.unwrap(); + // author + // }; + + // // check that the node fails to start if the default author is missing from the docs store. + // { + // let mut docs_store = iroh_docs::store::fs::Store::persistent( + // IrohPaths::DocsDatabase.with_root(iroh_root), + // ) + // .unwrap(); + // docs_store.delete_author(default_author).unwrap(); + // docs_store.flush().unwrap(); + // drop(docs_store); + // let iroh = Node::persistent(iroh_root) + // .await + // .unwrap() + // .enable_docs() + // .spawn() + // .await; + // assert!(iroh.is_err()); + + // // somehow the blob store is not shutdown correctly (yet?) on macos. + // // so we give it some time until we find a proper fix. + // #[cfg(target_os = "macos")] + // tokio::time::sleep(Duration::from_secs(1)).await; + + // tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) + // .await + // .unwrap(); + // drop(iroh); + // let iroh = Node::persistent(iroh_root) + // .await + // .unwrap() + // .enable_docs() + // .spawn() + // .await; + // assert!(iroh.is_ok()); + // iroh.unwrap().shutdown().await.unwrap(); + // } + + // // check that the default author can be set manually and is persisted. + // let default_author = { + // let iroh = Node::persistent(iroh_root) + // .await + // .unwrap() + // .enable_docs() + // .spawn() + // .await + // .unwrap(); + // let author = iroh.authors().create().await.unwrap(); + // iroh.authors().set_default(author).await.unwrap(); + // assert_eq!(iroh.authors().default().await.unwrap(), author); + // iroh.shutdown().await.unwrap(); + // author + // }; + // { + // let iroh = Node::persistent(iroh_root) + // .await + // .unwrap() + // .enable_docs() + // .spawn() + // .await + // .unwrap(); + // assert_eq!(iroh.authors().default().await.unwrap(), default_author); + // iroh.shutdown().await.unwrap(); + // } + + // Ok(()) + // } } diff --git a/src/rpc/docs_handle_request.rs b/src/rpc/docs_handle_request.rs index 889263c..44cb14f 100644 --- a/src/rpc/docs_handle_request.rs +++ b/src/rpc/docs_handle_request.rs @@ -4,173 +4,130 @@ use anyhow::anyhow; use futures_lite::{Stream, StreamExt}; use iroh_blobs::{ export::ExportProgress, - store::{ExportFormat, ImportProgress, Store as BaoStore}, + store::ImportProgress, util::progress::{AsyncChannelProgressSender, ProgressSender}, - BlobFormat, HashAndFormat, }; use super::{ client::ShareMode, proto::{ - // authors::{ - // CreateRequest, CreateResponse, DeleteRequest, DeleteResponse, ExportRequest, - // ExportResponse, GetDefaultRequest, GetDefaultResponse, ImportRequest, ImportResponse, - // ListRequest as AuthorListRequest, ListResponse as AuthorListResponse, - // SetDefaultRequest, SetDefaultResponse, - // }, - CloseRequest, - CloseResponse, - CreateRequest as DocCreateRequest, - CreateResponse as DocCreateResponse, - DelRequest, - DelResponse, - DocListRequest, - DocSubscribeRequest, - DocSubscribeResponse, - DropRequest, - DropResponse, - ExportFileRequest, - ExportFileResponse, - GetDownloadPolicyRequest, - GetDownloadPolicyResponse, - GetExactRequest, - GetExactResponse, - GetManyRequest, - GetManyResponse, - GetSyncPeersRequest, - GetSyncPeersResponse, - ImportFileRequest, - ImportFileResponse, - ImportRequest as DocImportRequest, - ImportResponse as DocImportResponse, - LeaveRequest, - LeaveResponse, - ListResponse as DocListResponse, - OpenRequest, - OpenResponse, - SetDownloadPolicyRequest, - SetDownloadPolicyResponse, - SetHashRequest, - SetHashResponse, - SetRequest, - SetResponse, - ShareRequest, - ShareResponse, - StartSyncRequest, - StartSyncResponse, - StatusRequest, - StatusResponse, + AuthorCreateRequest, AuthorCreateResponse, AuthorDeleteRequest, AuthorDeleteResponse, + AuthorExportRequest, AuthorExportResponse, AuthorGetDefaultRequest, + AuthorGetDefaultResponse, AuthorImportRequest, AuthorImportResponse, AuthorListRequest, + AuthorListResponse, AuthorSetDefaultRequest, AuthorSetDefaultResponse, CloseRequest, + CloseResponse, CreateRequest as DocCreateRequest, CreateResponse as DocCreateResponse, + DelRequest, DelResponse, DocListRequest, DocSubscribeRequest, DocSubscribeResponse, + DropRequest, DropResponse, ExportFileRequest, ExportFileResponse, GetDownloadPolicyRequest, + GetDownloadPolicyResponse, GetExactRequest, GetExactResponse, GetManyRequest, + GetManyResponse, GetSyncPeersRequest, GetSyncPeersResponse, ImportFileRequest, + ImportFileResponse, ImportRequest as DocImportRequest, ImportResponse as DocImportResponse, + LeaveRequest, LeaveResponse, ListResponse as DocListResponse, OpenRequest, OpenResponse, + SetDownloadPolicyRequest, SetDownloadPolicyResponse, SetHashRequest, SetHashResponse, + SetRequest, SetResponse, ShareRequest, ShareResponse, StartSyncRequest, StartSyncResponse, + StatusRequest, StatusResponse, }, RpcError, RpcResult, }; -use crate::{engine::Engine, DocTicket, NamespaceSecret}; +use crate::{engine::Engine, Author, DocTicket, NamespaceSecret}; /// Capacity for the flume channels to forward sync store iterators to async RPC streams. const ITER_CHANNEL_CAP: usize = 64; impl Engine { - // pub(super) async fn author_create(self, _req: CreateRequest) -> RpcResult { - // self.with_docs(|docs| async move { - // // TODO: pass rng - // let author = Author::new(&mut rand::rngs::OsRng {}); - // docs.sync - // .import_author(author.clone()) - // .await - // .map_err(|e| RpcError::new(&*e))?; - // Ok(CreateResponse { - // author_id: author.id(), - // }) - // }) - // .await - // } - - // pub(super) async fn author_default( - // self, - // _req: GetDefaultRequest, - // ) -> RpcResult { - // self.with_docs(|docs| async move { - // let author_id = docs.default_author.get(); - // Ok(GetDefaultResponse { author_id }) - // }) - // .await - // } - - // pub(super) async fn author_set_default( - // self, - // req: SetDefaultRequest, - // ) -> RpcResult { - // self.with_docs(|docs| async move { - // docs.default_author - // .set(req.author_id, &docs.sync) - // .await - // .map_err(|e| RpcError::new(&*e))?; - // Ok(SetDefaultResponse) - // }) - // .await - // } - - // pub(super) fn author_list( - // self, - // _req: AuthorListRequest, - // ) -> impl Stream> + Unpin { - // self.with_docs_stream(|docs| { - // let (tx, rx) = async_channel::bounded(ITER_CHANNEL_CAP); - // let sync = docs.sync.clone(); - // // we need to spawn a task to send our request to the sync handle, because the method - // // itself must be sync. - // tokio::task::spawn(async move { - // let tx2 = tx.clone(); - // if let Err(err) = sync.list_authors(tx).await { - // tx2.send(Err(err)).await.ok(); - // } - // }); - // rx.boxed().map(|r| { - // r.map(|author_id| AuthorListResponse { author_id }) - // .map_err(|e| RpcError::new(&*e)) - // }) - // }) - // } - - // pub(super) async fn author_import(self, req: ImportRequest) -> RpcResult { - // self.with_docs(|docs| async move { - // let author_id = docs - // .sync - // .import_author(req.author) - // .await - // .map_err(|e| RpcError::new(&*e))?; - // Ok(ImportResponse { author_id }) - // }) - // .await - // } - - // pub(super) async fn author_export(self, req: ExportRequest) -> RpcResult { - // self.with_docs(|docs| async move { - // let author = docs - // .sync - // .export_author(req.author) - // .await - // .map_err(|e| RpcError::new(&*e))?; - - // Ok(ExportResponse { author }) - // }) - // .await - // } - - // pub(super) async fn author_delete(self, req: DeleteRequest) -> RpcResult { - // self.with_docs(|docs| async move { - // if req.author == docs.default_author.get() { - // return Err(RpcError::new(&*anyhow!( - // "Deleting the default author is not supported" - // ))); - // } - // docs.sync - // .delete_author(req.author) - // .await - // .map_err(|e| RpcError::new(&*e))?; - // Ok(DeleteResponse) - // }) - // .await - // } + pub(super) async fn author_create( + self, + _req: AuthorCreateRequest, + ) -> RpcResult { + // TODO: pass rng + let author = Author::new(&mut rand::rngs::OsRng {}); + self.sync + .import_author(author.clone()) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(AuthorCreateResponse { + author_id: author.id(), + }) + } + + pub(super) async fn author_default( + self, + _req: AuthorGetDefaultRequest, + ) -> RpcResult { + let author_id = self.default_author.get(); + Ok(AuthorGetDefaultResponse { author_id }) + } + + pub(super) async fn author_set_default( + self, + req: AuthorSetDefaultRequest, + ) -> RpcResult { + self.default_author + .set(req.author_id, &self.sync) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(AuthorSetDefaultResponse) + } + + pub(super) fn author_list( + self, + _req: AuthorListRequest, + ) -> impl Stream> + Unpin { + let (tx, rx) = async_channel::bounded(ITER_CHANNEL_CAP); + let sync = self.sync.clone(); + // we need to spawn a task to send our request to the sync handle, because the method + // itself must be sync. + tokio::task::spawn(async move { + let tx2 = tx.clone(); + if let Err(err) = sync.list_authors(tx).await { + tx2.send(Err(err)).await.ok(); + } + }); + rx.boxed().map(|r| { + r.map(|author_id| AuthorListResponse { author_id }) + .map_err(|e| RpcError::new(&*e)) + }) + } + + pub(super) async fn author_import( + self, + req: AuthorImportRequest, + ) -> RpcResult { + let author_id = self + .sync + .import_author(req.author) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(AuthorImportResponse { author_id }) + } + + pub(super) async fn author_export( + self, + req: AuthorExportRequest, + ) -> RpcResult { + let author = self + .sync + .export_author(req.author) + .await + .map_err(|e| RpcError::new(&*e))?; + + Ok(AuthorExportResponse { author }) + } + + pub(super) async fn author_delete( + self, + req: AuthorDeleteRequest, + ) -> RpcResult { + if req.author == self.default_author.get() { + return Err(RpcError::new(&*anyhow!( + "Deleting the default author is not supported" + ))); + } + self.sync + .delete_author(req.author) + .await + .map_err(|e| RpcError::new(&*e))?; + Ok(AuthorDeleteResponse) + } pub(super) async fn doc_create(self, _req: DocCreateRequest) -> RpcResult { let namespace = NamespaceSecret::new(&mut rand::rngs::OsRng {}); diff --git a/src/rpc/proto.rs b/src/rpc/proto.rs index d646115..62cff1d 100644 --- a/src/rpc/proto.rs +++ b/src/rpc/proto.rs @@ -19,7 +19,8 @@ use crate::{ actor::OpenState, engine::LiveEvent, store::{DownloadPolicy, Query}, - AuthorId, Capability, CapabilityKind, DocTicket, Entry, NamespaceId, PeerIdBytes, SignedEntry, + Author, AuthorId, Capability, CapabilityKind, DocTicket, Entry, NamespaceId, PeerIdBytes, + SignedEntry, }; /// The RPC service type for the docs protocol. @@ -78,6 +79,20 @@ pub enum Request { SetDownloadPolicy(SetDownloadPolicyRequest), #[rpc(response = RpcResult)] GetSyncPeers(GetSyncPeersRequest), + #[server_streaming(response = RpcResult)] + AuthorList(AuthorListRequest), + #[rpc(response = RpcResult)] + AuthorCreate(AuthorCreateRequest), + #[rpc(response = RpcResult)] + AuthorGetDefault(AuthorGetDefaultRequest), + #[rpc(response = RpcResult)] + AuthorSetDefault(AuthorSetDefaultRequest), + #[rpc(response = RpcResult)] + AuthorImport(AuthorImportRequest), + #[rpc(response = RpcResult)] + AuthorExport(AuthorExportRequest), + #[rpc(response = RpcResult)] + AuthorDelete(AuthorDeleteRequest), } #[allow(missing_docs)] @@ -106,6 +121,13 @@ pub enum Response { SetDownloadPolicy(RpcResult), GetSyncPeers(RpcResult), StreamCreated(RpcResult), + AuthorList(RpcResult), + AuthorCreate(RpcResult), + AuthorGetDefault(RpcResult), + AuthorSetDefault(RpcResult), + AuthorImport(RpcResult), + AuthorExport(RpcResult), + AuthorDelete(RpcResult), } /// Subscribe to events for a document. @@ -433,3 +455,86 @@ pub struct GetSyncPeersResponse { /// List of peers ids pub peers: Option>, } + +/// List document authors for which we have a secret key. +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorListRequest {} + +/// Response for [`ListRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorListResponse { + /// The author id + pub author_id: AuthorId, +} + +/// Create a new document author. +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorCreateRequest; + +/// Response for [`CreateRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorCreateResponse { + /// The id of the created author + pub author_id: AuthorId, +} + +/// Get the default author. +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorGetDefaultRequest; + +/// Response for [`GetDefaultRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorGetDefaultResponse { + /// The id of the author + pub author_id: AuthorId, +} + +/// Set the default author. +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorSetDefaultRequest { + /// The id of the author + pub author_id: AuthorId, +} + +/// Response for [`GetDefaultRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorSetDefaultResponse; + +/// Delete an author +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorDeleteRequest { + /// The id of the author to delete + pub author: AuthorId, +} + +/// Response for [`DeleteRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorDeleteResponse; + +/// Exports an author +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorExportRequest { + /// The id of the author to delete + pub author: AuthorId, +} + +/// Response for [`ExportRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorExportResponse { + /// The author + pub author: Option, +} + +/// Import author from secret key +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorImportRequest { + /// The author to import + pub author: Author, +} + +/// Response to [`ImportRequest`] +#[derive(Serialize, Deserialize, Debug)] +pub struct AuthorImportResponse { + /// The author id of the imported author + pub author_id: AuthorId, +} diff --git a/tests/gc.rs b/tests/gc.rs new file mode 100644 index 0000000..3156e8b --- /dev/null +++ b/tests/gc.rs @@ -0,0 +1,40 @@ +// TODO: + +// #[tokio::test] +// async fn redb_doc_import_stress() -> Result<()> { +// let _ = tracing_subscriber::fmt::try_init(); +// let dir = testdir!(); +// let bao_store = iroh_blobs::store::fs::Store::load(dir.join("store")).await?; +// let (node, _) = wrap_in_node(bao_store.clone(), Duration::from_secs(10)).await; +// let client = node.client(); +// let doc = client.docs().create().await?; +// let author = client.authors().create().await?; +// let temp_path = dir.join("temp"); +// tokio::fs::create_dir_all(&temp_path).await?; +// let mut to_import = Vec::new(); +// for i in 0..100 { +// let data = create_test_data(16 * 1024 * 3 + 1); +// let path = temp_path.join(format!("file{}", i)); +// tokio::fs::write(&path, &data).await?; +// let key = Bytes::from(format!("{}", path.display())); +// to_import.push((key, path, data)); +// } +// for (key, path, _) in to_import.iter() { +// let mut progress = doc.import_file(author, key.clone(), path, true).await?; +// while let Some(msg) = progress.next().await { +// tracing::info!("import progress {:?}", msg); +// } +// } +// for (i, (key, _, expected)) in to_import.iter().enumerate() { +// let Some(entry) = doc.get_exact(author, key.clone(), true).await? else { +// anyhow::bail!("doc entry not found {}", i); +// }; +// let hash = entry.content_hash(); +// let Some(content) = bao_store.get(&hash).await? else { +// anyhow::bail!("content not found {} {}", i, &hash.to_hex()[..8]); +// }; +// let data = content.data_reader().read_to_end().await?; +// assert_eq!(data, expected); +// } +// Ok(()) +// } diff --git a/tests/sync.rs b/tests/sync.rs new file mode 100644 index 0000000..c0cb409 --- /dev/null +++ b/tests/sync.rs @@ -0,0 +1,1405 @@ +// TODO: + +// use std::{ +// collections::HashMap, +// future::Future, +// sync::Arc, +// time::{Duration, Instant}, +// }; + +// use anyhow::{anyhow, bail, Context, Result}; +// use bytes::Bytes; +// use futures_lite::Stream; +// use futures_util::{FutureExt, StreamExt, TryStreamExt}; +// use iroh::{ +// base::node_addr::AddrInfoOptions, +// client::{ +// self, +// docs::{Entry, LiveEvent, ShareMode}, +// Doc, RpcService, +// }, +// net::key::{PublicKey, SecretKey}, +// node::{Builder, Node}, +// }; +// use iroh_blobs::Hash; +// use iroh_docs::{ +// store::{DownloadPolicy, FilterKind, Query}, +// AuthorId, ContentStatus, +// }; +// use iroh_net::relay::RelayMode; +// use rand::{CryptoRng, Rng, SeedableRng}; +// use tracing::{debug, error_span, info, Instrument}; +// use tracing_subscriber::{prelude::*, EnvFilter}; + +// const TIMEOUT: Duration = Duration::from_secs(60); + +// fn test_node(secret_key: SecretKey) -> Builder { +// Node::memory() +// .secret_key(secret_key) +// .enable_docs() +// .relay_mode(RelayMode::Disabled) +// } + +// // The function is not `async fn` so that we can take a `&mut` borrow on the `rng` without +// // capturing that `&mut` lifetime in the returned future. This allows to call it in a loop while +// // still collecting the futures before awaiting them altogether (see [`spawn_nodes`]) +// fn spawn_node( +// i: usize, +// rng: &mut (impl CryptoRng + Rng), +// ) -> impl Future>> + 'static { +// let secret_key = SecretKey::generate_with_rng(rng); +// async move { +// let node = test_node(secret_key); +// let node = node.spawn().await?; +// info!(?i, me = %node.node_id().fmt_short(), "node spawned"); +// Ok(node) +// } +// } + +// async fn spawn_nodes( +// n: usize, +// mut rng: &mut (impl CryptoRng + Rng), +// ) -> anyhow::Result>> { +// let mut futs = vec![]; +// for i in 0..n { +// futs.push(spawn_node(i, &mut rng)); +// } +// futures_buffered::join_all(futs).await.into_iter().collect() +// } + +// pub fn test_rng(seed: &[u8]) -> rand_chacha::ChaCha12Rng { +// rand_chacha::ChaCha12Rng::from_seed(*Hash::new(seed).as_bytes()) +// } + +// macro_rules! match_event { +// ($pattern:pat $(if $guard:expr)? $(,)?) => { +// Box::new(move |e| matches!(e, $pattern $(if $guard)?)) +// }; +// } + +// /// This tests the simplest scenario: A node connects to another node, and performs sync. +// #[tokio::test] +// async fn sync_simple() -> Result<()> { +// setup_logging(); +// let mut rng = test_rng(b"sync_simple"); +// let nodes = spawn_nodes(2, &mut rng).await?; +// let clients = nodes.iter().map(|node| node.client()).collect::>(); + +// // create doc on node0 +// let peer0 = nodes[0].node_id(); +// let author0 = clients[0].authors().create().await?; +// let doc0 = clients[0].docs().create().await?; +// let blobs0 = clients[0].blobs(); +// let hash0 = doc0 +// .set_bytes(author0, b"k1".to_vec(), b"v1".to_vec()) +// .await?; +// assert_latest(&blobs0, &doc0, b"k1", b"v1").await; +// let ticket = doc0 +// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) +// .await?; + +// let mut events0 = doc0.subscribe().await?; + +// info!("node1: join"); +// let peer1 = nodes[1].node_id(); +// let doc1 = clients[1].docs().import(ticket.clone()).await?; +// let blobs1 = clients[1].blobs(); +// let mut events1 = doc1.subscribe().await?; +// info!("node1: assert 5 events"); +// assert_next_unordered( +// &mut events1, +// TIMEOUT, +// vec![ +// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer0)), +// Box::new(move |e| matches!(e, LiveEvent::InsertRemote { from, .. } if *from == peer0 )), +// Box::new(move |e| match_sync_finished(e, peer0)), +// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash0)), +// match_event!(LiveEvent::PendingContentReady), +// ], +// ) +// .await; +// assert_latest(&blobs1, &doc1, b"k1", b"v1").await; + +// info!("node0: assert 2 events"); +// assert_next( +// &mut events0, +// TIMEOUT, +// vec![ +// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer1)), +// Box::new(move |e| match_sync_finished(e, peer1)), +// ], +// ) +// .await; + +// for node in nodes { +// node.shutdown().await?; +// } +// Ok(()) +// } + +// /// Test subscribing to replica events (without sync) +// #[tokio::test] +// async fn sync_subscribe_no_sync() -> Result<()> { +// let mut rng = test_rng(b"sync_subscribe"); +// setup_logging(); +// let node = spawn_node(0, &mut rng).await?; +// let client = node.client(); +// let doc = client.docs().create().await?; +// let mut sub = doc.subscribe().await?; +// let author = client.authors().create().await?; +// doc.set_bytes(author, b"k".to_vec(), b"v".to_vec()).await?; +// let event = tokio::time::timeout(Duration::from_millis(100), sub.next()).await?; +// assert!( +// matches!(event, Some(Ok(LiveEvent::InsertLocal { .. }))), +// "expected InsertLocal but got {event:?}" +// ); +// node.shutdown().await?; +// Ok(()) +// } + +// #[tokio::test] +// async fn sync_gossip_bulk() -> Result<()> { +// let n_entries: usize = std::env::var("N_ENTRIES") +// .map(|x| x.parse().expect("N_ENTRIES must be a number")) +// .unwrap_or(100); +// let mut rng = test_rng(b"sync_gossip_bulk"); +// setup_logging(); + +// let nodes = spawn_nodes(2, &mut rng).await?; +// let clients = nodes.iter().map(|node| node.client()).collect::>(); + +// let _peer0 = nodes[0].node_id(); +// let author0 = clients[0].authors().create().await?; +// let doc0 = clients[0].docs().create().await?; +// let mut ticket = doc0 +// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) +// .await?; +// // unset peers to not yet start sync +// let peers = ticket.nodes.clone(); +// ticket.nodes = vec![]; +// let doc1 = clients[1].docs().import(ticket).await?; +// let mut events = doc1.subscribe().await?; + +// // create entries for initial sync. +// let now = Instant::now(); +// let value = b"foo"; +// for i in 0..n_entries { +// let key = format!("init/{i}"); +// doc0.set_bytes(author0, key.as_bytes().to_vec(), value.to_vec()) +// .await?; +// } +// let elapsed = now.elapsed(); +// info!( +// "insert took {elapsed:?} for {n_entries} ({:?} per entry)", +// elapsed / n_entries as u32 +// ); + +// let now = Instant::now(); +// let mut count = 0; +// doc0.start_sync(vec![]).await?; +// doc1.start_sync(peers).await?; +// while let Some(event) = events.next().await { +// let event = event?; +// if matches!(event, LiveEvent::InsertRemote { .. }) { +// count += 1; +// } +// if count == n_entries { +// break; +// } +// } +// let elapsed = now.elapsed(); +// info!( +// "initial sync took {elapsed:?} for {n_entries} ({:?} per entry)", +// elapsed / n_entries as u32 +// ); + +// // publish another 1000 entries +// let mut count = 0; +// let value = b"foo"; +// let now = Instant::now(); +// for i in 0..n_entries { +// let key = format!("gossip/{i}"); +// doc0.set_bytes(author0, key.as_bytes().to_vec(), value.to_vec()) +// .await?; +// } +// let elapsed = now.elapsed(); +// info!( +// "insert took {elapsed:?} for {n_entries} ({:?} per entry)", +// elapsed / n_entries as u32 +// ); + +// while let Some(event) = events.next().await { +// let event = event?; +// if matches!(event, LiveEvent::InsertRemote { .. }) { +// count += 1; +// } +// if count == n_entries { +// break; +// } +// } +// let elapsed = now.elapsed(); +// info!( +// "gossip recv took {elapsed:?} for {n_entries} ({:?} per entry)", +// elapsed / n_entries as u32 +// ); + +// Ok(()) +// } + +// /// This tests basic sync and gossip with 3 peers. +// #[tokio::test] +// #[ignore = "flaky"] +// async fn sync_full_basic() -> Result<()> { +// let mut rng = test_rng(b"sync_full_basic"); +// setup_logging(); +// let mut nodes = spawn_nodes(2, &mut rng).await?; +// let mut clients = nodes +// .iter() +// .map(|node| node.client().clone()) +// .collect::>(); + +// // peer0: create doc and ticket +// let peer0 = nodes[0].node_id(); +// let author0 = clients[0].authors().create().await?; +// let doc0 = clients[0].docs().create().await?; +// let blobs0 = clients[0].blobs(); +// let mut events0 = doc0.subscribe().await?; +// let key0 = b"k1"; +// let value0 = b"v1"; +// let hash0 = doc0 +// .set_bytes(author0, key0.to_vec(), value0.to_vec()) +// .await?; + +// info!("peer0: wait for 1 event (local insert)"); +// let e = next(&mut events0).await; +// assert!( +// matches!(&e, LiveEvent::InsertLocal { entry } if entry.content_hash() == hash0), +// "expected LiveEvent::InsertLocal but got {e:?}", +// ); +// assert_latest(&blobs0, &doc0, key0, value0).await; +// let ticket = doc0 +// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) +// .await?; + +// info!("peer1: spawn"); +// let peer1 = nodes[1].node_id(); +// let author1 = clients[1].authors().create().await?; +// info!("peer1: join doc"); +// let doc1 = clients[1].docs().import(ticket.clone()).await?; +// let blobs1 = clients[1].blobs(); + +// info!("peer1: wait for 4 events (for sync and join with peer0)"); +// let mut events1 = doc1.subscribe().await?; +// assert_next_unordered( +// &mut events1, +// TIMEOUT, +// vec![ +// match_event!(LiveEvent::NeighborUp(peer) if *peer == peer0), +// match_event!(LiveEvent::InsertRemote { from, .. } if *from == peer0 ), +// Box::new(move |e| match_sync_finished(e, peer0)), +// match_event!(LiveEvent::ContentReady { hash } if *hash == hash0), +// match_event!(LiveEvent::PendingContentReady), +// ], +// ) +// .await; + +// info!("peer0: wait for 2 events (join & accept sync finished from peer1)"); +// assert_next( +// &mut events0, +// TIMEOUT, +// vec![ +// match_event!(LiveEvent::NeighborUp(peer) if *peer == peer1), +// Box::new(move |e| match_sync_finished(e, peer1)), +// match_event!(LiveEvent::PendingContentReady), +// ], +// ) +// .await; + +// info!("peer1: insert entry"); +// let key1 = b"k2"; +// let value1 = b"v2"; +// let hash1 = doc1 +// .set_bytes(author1, key1.to_vec(), value1.to_vec()) +// .await?; +// assert_latest(&blobs1, &doc1, key1, value1).await; +// info!("peer1: wait for 1 event (local insert, and pendingcontentready)"); +// assert_next( +// &mut events1, +// TIMEOUT, +// vec![match_event!(LiveEvent::InsertLocal { entry} if entry.content_hash() == hash1)], +// ) +// .await; + +// // peer0: assert events for entry received via gossip +// info!("peer0: wait for 2 events (gossip'ed entry from peer1)"); +// assert_next( +// &mut events0, +// TIMEOUT, +// vec![ +// Box::new( +// move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == peer1), +// ), +// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash1)), +// ], +// ).await; +// assert_latest(&blobs0, &doc0, key1, value1).await; + +// // Note: If we could check gossip messages directly here (we can't easily), we would notice +// // that peer1 will receive a `Op::ContentReady` gossip message, broadcast +// // by peer0 with neighbor scope. This message is superfluous, and peer0 could know that, however +// // our gossip implementation does not allow us to filter message receivers this way. + +// info!("peer2: spawn"); +// nodes.push(spawn_node(nodes.len(), &mut rng).await?); +// clients.push(nodes.last().unwrap().client().clone()); +// let doc2 = clients[2].docs().import(ticket).await?; +// let blobs2 = clients[2].blobs(); +// let peer2 = nodes[2].node_id(); +// let mut events2 = doc2.subscribe().await?; + +// info!("peer2: wait for 9 events (from sync with peers)"); +// assert_next_unordered_with_optionals( +// &mut events2, +// TIMEOUT, +// // required events +// vec![ +// // 2 NeighborUp events +// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer0)), +// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer1)), +// // 2 SyncFinished events +// Box::new(move |e| match_sync_finished(e, peer0)), +// Box::new(move |e| match_sync_finished(e, peer1)), +// // 2 InsertRemote events +// Box::new( +// move |e| matches!(e, LiveEvent::InsertRemote { entry, content_status: ContentStatus::Missing, .. } if entry.content_hash() == hash0), +// ), +// Box::new( +// move |e| matches!(e, LiveEvent::InsertRemote { entry, content_status: ContentStatus::Missing, .. } if entry.content_hash() == hash1), +// ), +// // 2 ContentReady events +// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash0)), +// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash1)), +// // at least 1 PendingContentReady +// match_event!(LiveEvent::PendingContentReady), +// ], +// // optional events +// // it may happen that we run sync two times against our two peers: +// // if the first sync (as a result of us joining the peer manually through the ticket) completes +// // before the peer shows up as a neighbor, we run sync again for the NeighborUp event. +// vec![ +// // 2 SyncFinished events +// Box::new(move |e| match_sync_finished(e, peer0)), +// Box::new(move |e| match_sync_finished(e, peer1)), +// match_event!(LiveEvent::PendingContentReady), +// match_event!(LiveEvent::PendingContentReady), +// ] +// ).await; +// assert_latest(&blobs2, &doc2, b"k1", b"v1").await; +// assert_latest(&blobs2, &doc2, b"k2", b"v2").await; + +// info!("peer0: wait for 2 events (join & accept sync finished from peer2)"); +// assert_next( +// &mut events0, +// TIMEOUT, +// vec![ +// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer2)), +// Box::new(move |e| match_sync_finished(e, peer2)), +// match_event!(LiveEvent::PendingContentReady), +// ], +// ) +// .await; + +// info!("peer1: wait for 2 events (join & accept sync finished from peer2)"); +// assert_next( +// &mut events1, +// TIMEOUT, +// vec![ +// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer2)), +// Box::new(move |e| match_sync_finished(e, peer2)), +// match_event!(LiveEvent::PendingContentReady), +// ], +// ) +// .await; + +// info!("shutdown"); +// for node in nodes { +// node.shutdown().await?; +// } + +// Ok(()) +// } + +// #[tokio::test] +// async fn sync_open_close() -> Result<()> { +// let mut rng = test_rng(b"sync_subscribe_stop_close"); +// setup_logging(); +// let node = spawn_node(0, &mut rng).await?; +// let client = node.client(); + +// let doc = client.docs().create().await?; +// let status = doc.status().await?; +// assert_eq!(status.handles, 1); + +// let doc2 = client.docs().open(doc.id()).await?.unwrap(); +// let status = doc2.status().await?; +// assert_eq!(status.handles, 2); + +// doc.close().await?; +// assert!(doc.status().await.is_err()); + +// let status = doc2.status().await?; +// assert_eq!(status.handles, 1); + +// Ok(()) +// } + +// #[tokio::test] +// async fn sync_subscribe_stop_close() -> Result<()> { +// let mut rng = test_rng(b"sync_subscribe_stop_close"); +// setup_logging(); +// let node = spawn_node(0, &mut rng).await?; +// let client = node.client(); + +// let doc = client.docs().create().await?; +// let author = client.authors().create().await?; + +// let status = doc.status().await?; +// assert_eq!(status.subscribers, 0); +// assert_eq!(status.handles, 1); +// assert!(!status.sync); + +// doc.start_sync(vec![]).await?; +// let status = doc.status().await?; +// assert!(status.sync); +// assert_eq!(status.handles, 2); +// assert_eq!(status.subscribers, 1); + +// let sub = doc.subscribe().await?; +// let status = doc.status().await?; +// assert_eq!(status.subscribers, 2); +// drop(sub); +// // trigger an event that makes the actor check if the event channels are still connected +// doc.set_bytes(author, b"x".to_vec(), b"x".to_vec()).await?; +// let status = doc.status().await?; +// assert_eq!(status.subscribers, 1); + +// doc.leave().await?; +// let status = doc.status().await?; +// assert_eq!(status.subscribers, 0); +// assert_eq!(status.handles, 1); +// assert!(!status.sync); + +// Ok(()) +// } + +// #[tokio::test] +// #[cfg(feature = "test-utils")] +// async fn test_sync_via_relay() -> Result<()> { +// let _guard = iroh_test::logging::setup(); +// let (relay_map, _relay_url, _guard) = iroh_net::test_utils::run_relay_server().await?; + +// let node1 = Node::memory() +// .relay_mode(RelayMode::Custom(relay_map.clone())) +// .insecure_skip_relay_cert_verify(true) +// .enable_docs() +// .spawn() +// .await?; +// let node1_id = node1.node_id(); +// let node2 = Node::memory() +// .bind_random_port() +// .relay_mode(RelayMode::Custom(relay_map.clone())) +// .insecure_skip_relay_cert_verify(true) +// .enable_docs() +// .spawn() +// .await?; + +// let doc1 = node1.docs().create().await?; +// let author1 = node1.authors().create().await?; +// let inserted_hash = doc1 +// .set_bytes(author1, b"foo".to_vec(), b"bar".to_vec()) +// .await?; +// let mut ticket = doc1 +// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) +// .await?; + +// // remove direct addrs to force connect via relay +// ticket.nodes[0].info.direct_addresses = Default::default(); + +// // join +// let doc2 = node2.docs().import(ticket).await?; +// let blobs2 = node2.blobs(); +// let mut events = doc2.subscribe().await?; + +// assert_next_unordered_with_optionals( +// &mut events, +// Duration::from_secs(2), +// vec![ +// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(n) if *n== node1_id)), +// Box::new(move |e| match_sync_finished(e, node1_id)), +// Box::new( +// move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == node1_id), +// ), +// Box::new( +// move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == inserted_hash), +// ), +// match_event!(LiveEvent::PendingContentReady), +// ], +// vec![Box::new(move |e| match_sync_finished(e, node1_id))], +// ).await; +// let actual = blobs2 +// .read_to_bytes( +// doc2.get_exact(author1, b"foo", false) +// .await? +// .expect("entry to exist") +// .content_hash(), +// ) +// .await?; +// assert_eq!(actual.as_ref(), b"bar"); + +// // update +// let updated_hash = doc1 +// .set_bytes(author1, b"foo".to_vec(), b"update".to_vec()) +// .await?; +// assert_next_unordered_with_optionals( +// &mut events, +// Duration::from_secs(2), +// vec![ +// Box::new( +// move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == node1_id), +// ), +// Box::new( +// move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == updated_hash), +// ), +// ], +// vec![ +// Box::new(move |e| match_sync_finished(e, node1_id)), +// Box::new(move |e| matches!(e, LiveEvent::PendingContentReady)), +// ], +// ).await; +// let actual = blobs2 +// .read_to_bytes( +// doc2.get_exact(author1, b"foo", false) +// .await? +// .expect("entry to exist") +// .content_hash(), +// ) +// .await?; +// assert_eq!(actual.as_ref(), b"update"); +// Ok(()) +// } + +// #[tokio::test] +// #[cfg(feature = "test-utils")] +// #[ignore = "flaky"] +// async fn sync_restart_node() -> Result<()> { +// let mut rng = test_rng(b"sync_restart_node"); +// setup_logging(); +// let (relay_map, _relay_url, _guard) = iroh_net::test_utils::run_relay_server().await?; + +// let discovery_server = iroh_net::test_utils::DnsPkarrServer::run().await?; + +// let node1_dir = tempfile::TempDir::with_prefix("test-sync_restart_node-node1")?; +// let secret_key_1 = SecretKey::generate_with_rng(&mut rng); + +// let node1 = Node::persistent(&node1_dir) +// .await? +// .secret_key(secret_key_1.clone()) +// .insecure_skip_relay_cert_verify(true) +// .relay_mode(RelayMode::Custom(relay_map.clone())) +// .dns_resolver(discovery_server.dns_resolver()) +// .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) +// .enable_docs() +// .spawn() +// .await?; +// let id1 = node1.node_id(); + +// // create doc & ticket on node1 +// let doc1 = node1.docs().create().await?; +// let blobs1 = node1.blobs().clone(); +// let mut events1 = doc1.subscribe().await?; +// let ticket = doc1 +// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) +// .await?; + +// // create node2 +// let secret_key_2 = SecretKey::generate_with_rng(&mut rng); +// let node2 = Node::memory() +// .secret_key(secret_key_2.clone()) +// .relay_mode(RelayMode::Custom(relay_map.clone())) +// .insecure_skip_relay_cert_verify(true) +// .dns_resolver(discovery_server.dns_resolver()) +// .node_discovery(discovery_server.discovery(secret_key_2.clone()).into()) +// .enable_docs() +// .spawn() +// .await?; +// let id2 = node2.node_id(); +// let author2 = node2.authors().create().await?; +// let doc2 = node2.docs().import(ticket.clone()).await?; +// let blobs2 = node2.blobs(); + +// info!("node2 set a"); +// let hash_a = doc2.set_bytes(author2, "n2/a", "a").await?; +// assert_latest(&blobs2, &doc2, b"n2/a", b"a").await; + +// assert_next_unordered_with_optionals( +// &mut events1, +// Duration::from_secs(10), +// vec![ +// match_event!(LiveEvent::NeighborUp(n) if *n == id2), +// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), +// match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), +// match_event!(LiveEvent::ContentReady { hash } if *hash == hash_a), +// match_event!(LiveEvent::PendingContentReady), +// ], +// vec![ +// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), +// match_event!(LiveEvent::PendingContentReady), +// ], +// ) +// .await; +// assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; + +// info!(me = id1.fmt_short(), "node1 start shutdown"); +// node1.shutdown().await?; +// info!(me = id1.fmt_short(), "node1 down"); + +// info!(me = id1.fmt_short(), "sleep 1s"); +// tokio::time::sleep(Duration::from_secs(1)).await; + +// info!(me = id2.fmt_short(), "node2 set b"); +// let hash_b = doc2.set_bytes(author2, "n2/b", "b").await?; + +// info!(me = id1.fmt_short(), "node1 respawn"); +// let node1 = Node::persistent(&node1_dir) +// .await? +// .secret_key(secret_key_1.clone()) +// .insecure_skip_relay_cert_verify(true) +// .relay_mode(RelayMode::Custom(relay_map.clone())) +// .dns_resolver(discovery_server.dns_resolver()) +// .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) +// .enable_docs() +// .spawn() +// .await?; +// assert_eq!(id1, node1.node_id()); + +// let doc1 = node1.docs().open(doc1.id()).await?.expect("doc to exist"); +// let blobs1 = node1.blobs(); +// let mut events1 = doc1.subscribe().await?; +// assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; + +// // check that initial resync is working +// doc1.start_sync(vec![]).await?; +// assert_next_unordered_with_optionals( +// &mut events1, +// Duration::from_secs(10), +// vec![ +// match_event!(LiveEvent::NeighborUp(n) if *n== id2), +// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), +// match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), +// match_event!(LiveEvent::ContentReady { hash } if *hash == hash_b), +// ], +// vec![ +// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), +// match_event!(LiveEvent::PendingContentReady), +// ] +// ).await; +// assert_latest(&blobs1, &doc1, b"n2/b", b"b").await; + +// // check that live conn is working +// info!(me = id2.fmt_short(), "node2 set c"); +// let hash_c = doc2.set_bytes(author2, "n2/c", "c").await?; +// assert_next_unordered_with_optionals( +// &mut events1, +// Duration::from_secs(10), +// vec![ +// match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), +// match_event!(LiveEvent::ContentReady { hash } if *hash == hash_c), +// ], +// vec![ +// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), +// match_event!(LiveEvent::PendingContentReady), +// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), +// match_event!(LiveEvent::PendingContentReady), +// ] +// ).await; + +// assert_latest(&blobs1, &doc1, b"n2/c", b"c").await; + +// Ok(()) +// } + +// /// Joins two nodes that write to the same document but have differing download policies and tests +// /// that they both synced the key info but not the content. +// #[tokio::test] +// async fn test_download_policies() -> Result<()> { +// // keys node a has +// let star_wars_movies = &[ +// "star_wars/prequel/the_phantom_menace", +// "star_wars/prequel/attack_of_the_clones", +// "star_wars/prequel/revenge_of_the_sith", +// "star_wars/og/a_new_hope", +// "star_wars/og/the_empire_strikes_back", +// "star_wars/og/return_of_the_jedi", +// ]; +// // keys node b has +// let lotr_movies = &[ +// "lotr/fellowship_of_the_ring", +// "lotr/the_two_towers", +// "lotr/return_of_the_king", +// ]; + +// // content policy for what b wants +// let policy_b = +// DownloadPolicy::EverythingExcept(vec![FilterKind::Prefix("star_wars/og".into())]); +// // content policy for what a wants +// let policy_a = DownloadPolicy::NothingExcept(vec![FilterKind::Exact( +// "lotr/fellowship_of_the_ring".into(), +// )]); + +// // a will sync all lotr keys but download a single key +// const EXPECTED_A_SYNCED: usize = 3; +// const EXPECTED_A_DOWNLOADED: usize = 1; + +// // b will sync all star wars content but download only the prequel keys +// const EXPECTED_B_SYNCED: usize = 6; +// const EXPECTED_B_DOWNLOADED: usize = 3; + +// let mut rng = test_rng(b"sync_download_policies"); +// setup_logging(); +// let nodes = spawn_nodes(2, &mut rng).await?; +// let clients = nodes.iter().map(|node| node.client()).collect::>(); + +// let doc_a = clients[0].docs().create().await?; +// let author_a = clients[0].authors().create().await?; +// let ticket = doc_a +// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) +// .await?; + +// let doc_b = clients[1].docs().import(ticket).await?; +// let author_b = clients[1].authors().create().await?; + +// doc_a.set_download_policy(policy_a).await?; +// doc_b.set_download_policy(policy_b).await?; + +// let mut events_a = doc_a.subscribe().await?; +// let mut events_b = doc_b.subscribe().await?; + +// let mut key_hashes: HashMap = HashMap::default(); + +// // set content in a +// for k in star_wars_movies.iter() { +// let hash = doc_a +// .set_bytes(author_a, k.to_owned(), k.to_owned()) +// .await?; +// key_hashes.insert(hash, k); +// } + +// // set content in b +// for k in lotr_movies.iter() { +// let hash = doc_b +// .set_bytes(author_b, k.to_owned(), k.to_owned()) +// .await?; +// key_hashes.insert(hash, k); +// } + +// assert_eq!(key_hashes.len(), star_wars_movies.len() + lotr_movies.len()); + +// let fut = async { +// use LiveEvent::*; +// let mut downloaded_a: Vec<&'static str> = Vec::new(); +// let mut downloaded_b: Vec<&'static str> = Vec::new(); +// let mut synced_a = 0usize; +// let mut synced_b = 0usize; +// loop { +// tokio::select! { +// Some(Ok(ev)) = events_a.next() => { +// match ev { +// InsertRemote { content_status, entry, .. } => { +// synced_a += 1; +// if let ContentStatus::Complete = content_status { +// downloaded_a.push(key_hashes.get(&entry.content_hash()).unwrap()) +// } +// }, +// ContentReady { hash } => { +// downloaded_a.push(key_hashes.get(&hash).unwrap()); +// }, +// _ => {} +// } +// } +// Some(Ok(ev)) = events_b.next() => { +// match ev { +// InsertRemote { content_status, entry, .. } => { +// synced_b += 1; +// if let ContentStatus::Complete = content_status { +// downloaded_b.push(key_hashes.get(&entry.content_hash()).unwrap()) +// } +// }, +// ContentReady { hash } => { +// downloaded_b.push(key_hashes.get(&hash).unwrap()); +// }, +// _ => {} +// } +// } +// } + +// if synced_a == EXPECTED_A_SYNCED +// && downloaded_a.len() == EXPECTED_A_DOWNLOADED +// && synced_b == EXPECTED_B_SYNCED +// && downloaded_b.len() == EXPECTED_B_DOWNLOADED +// { +// break; +// } +// } +// (downloaded_a, downloaded_b) +// }; + +// let (downloaded_a, mut downloaded_b) = tokio::time::timeout(TIMEOUT, fut) +// .await +// .context("timeout elapsed")?; + +// downloaded_b.sort(); +// assert_eq!(downloaded_a, vec!["lotr/fellowship_of_the_ring"]); +// assert_eq!( +// downloaded_b, +// vec![ +// "star_wars/prequel/attack_of_the_clones", +// "star_wars/prequel/revenge_of_the_sith", +// "star_wars/prequel/the_phantom_menace", +// ] +// ); + +// Ok(()) +// } + +// /// Test sync between many nodes with propagation through sync reports. +// #[tokio::test(flavor = "multi_thread")] +// #[ignore = "flaky"] +// async fn sync_big() -> Result<()> { +// setup_logging(); +// let mut rng = test_rng(b"sync_big"); +// let n_nodes = std::env::var("NODES") +// .map(|v| v.parse().expect("NODES must be a number")) +// .unwrap_or(10); +// let n_entries_init = 1; + +// tokio::task::spawn(async move { +// for i in 0.. { +// tokio::time::sleep(Duration::from_secs(1)).await; +// info!("tick {i}"); +// } +// }); + +// let nodes = spawn_nodes(n_nodes, &mut rng).await?; +// let node_ids = nodes.iter().map(|node| node.node_id()).collect::>(); +// let clients = nodes.iter().map(|node| node.client()).collect::>(); +// let authors = collect_futures(clients.iter().map(|c| c.authors().create())).await?; + +// let doc0 = clients[0].docs().create().await?; +// let mut ticket = doc0 +// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) +// .await?; +// // do not join for now, just import without any peer info +// let peer0 = ticket.nodes[0].clone(); +// ticket.nodes = vec![]; + +// let docs_clients: Vec<_> = clients.iter().skip(1).map(|c| c.docs().clone()).collect(); +// let mut docs = vec![]; +// docs.push(doc0); +// docs.extend_from_slice( +// &collect_futures(docs_clients.into_iter().map(|c| { +// let ticket = ticket.clone(); +// async move { c.import(ticket).await } +// })) +// .await?, +// ); + +// let mut expected = vec![]; + +// // create initial data on each node +// publish(&docs, &mut expected, n_entries_init, |i, j| { +// ( +// authors[i], +// format!("init/{}/{j}", node_ids[i].fmt_short()), +// format!("init:{i}:{j}"), +// ) +// }) +// .await?; + +// // assert initial data +// for (i, doc) in docs.iter().enumerate() { +// let blobs = nodes[i].blobs(); +// let entries = get_all_with_content(blobs, doc).await?; +// let mut expected = expected +// .iter() +// .filter(|e| e.author == authors[i]) +// .cloned() +// .collect::>(); +// expected.sort(); +// assert_eq!(entries, expected, "phase1 pre-sync correct"); +// } + +// // setup event streams +// let events = collect_futures(docs.iter().map(|d| d.subscribe())).await?; + +// // join nodes together +// for (i, doc) in docs.iter().enumerate().skip(1) { +// info!(me = %node_ids[i].fmt_short(), peer = %peer0.node_id.fmt_short(), "join"); +// doc.start_sync(vec![peer0.clone()]).await?; +// } + +// // wait for InsertRemote events stuff to happen +// info!("wait for all peers to receive insert events"); +// let expected_inserts = (n_nodes - 1) * n_entries_init; +// let mut tasks = tokio::task::JoinSet::default(); +// for (i, events) in events.into_iter().enumerate() { +// let doc = docs[i].clone(); +// let me = doc.id().fmt_short(); +// let expected = expected.clone(); +// let fut = async move { +// wait_for_events(events, expected_inserts, TIMEOUT, |e| { +// matches!(e, LiveEvent::InsertRemote { .. }) +// }) +// .await?; +// let entries = get_all(&doc).await?; +// if entries != expected { +// Err(anyhow!( +// "node {i} failed (has {} entries but expected to have {})", +// entries.len(), +// expected.len() +// )) +// } else { +// info!( +// "received and checked all {} expected entries", +// expected.len() +// ); +// Ok(()) +// } +// } +// .instrument(error_span!("sync-test", %me)); +// let fut = fut.map(move |r| r.with_context(move || format!("node {i} ({me})"))); +// tasks.spawn(fut); +// } + +// while let Some(res) = tasks.join_next().await { +// res??; +// } + +// assert_all_docs(&docs, &node_ids, &expected, "after initial sync").await; + +// info!("shutdown"); +// for node in nodes { +// node.shutdown().await?; +// } + +// Ok(()) +// } + +// #[tokio::test] +// #[cfg(feature = "test-utils")] +// async fn test_list_docs_stream() -> Result<()> { +// let node = Node::memory() +// .node_discovery(iroh::node::DiscoveryConfig::None) +// .relay_mode(iroh::net::relay::RelayMode::Disabled) +// .enable_docs() +// .spawn() +// .await?; +// let count = 200; + +// // create docs +// for _i in 0..count { +// let doc = node.docs().create().await?; +// doc.close().await?; +// } + +// // create doc stream +// let mut stream = node.docs().list().await?; + +// // process each doc and call into the docs actor. +// // this makes sure that we don't deadlock the docs actor. +// let mut i = 0; +// let fut = async { +// while let Some((id, _)) = stream.try_next().await.unwrap() { +// let _doc = node.docs().open(id).await.unwrap().unwrap(); +// i += 1; +// } +// }; + +// tokio::time::timeout(Duration::from_secs(2), fut) +// .await +// .expect("not to timeout"); + +// assert_eq!(i, count); + +// Ok(()) +// } + +// /// Get all entries of a document. +// async fn get_all(doc: &Doc) -> anyhow::Result> { +// let entries = doc.get_many(Query::all()).await?; +// let entries = entries.collect::>().await; +// entries.into_iter().collect() +// } + +// /// Get all entries of a document with the blob content. +// async fn get_all_with_content( +// blobs: &client::blobs::Client, +// doc: &Doc, +// ) -> anyhow::Result> { +// let entries = doc.get_many(Query::all()).await?; +// let entries = entries.and_then(|entry| async { +// let hash = entry.content_hash(); +// let content = blobs.read_to_bytes(hash).await; +// content.map(|c| (entry, c)) +// }); +// let entries = entries.collect::>().await; +// let entries = entries.into_iter().collect::>>()?; +// Ok(entries) +// } + +// async fn publish( +// docs: &[Doc], +// expected: &mut Vec, +// n: usize, +// cb: impl Fn(usize, usize) -> (AuthorId, String, String), +// ) -> anyhow::Result<()> { +// for (i, doc) in docs.iter().enumerate() { +// for j in 0..n { +// let (author, key, value) = cb(i, j); +// doc.set_bytes(author, key.as_bytes().to_vec(), value.as_bytes().to_vec()) +// .await?; +// expected.push(ExpectedEntry { author, key, value }); +// } +// } +// expected.sort(); +// Ok(()) +// } + +// /// Collect an iterator into futures by joining them all and failing if any future failed. +// async fn collect_futures( +// futs: impl IntoIterator>>, +// ) -> anyhow::Result> { +// futures_buffered::join_all(futs) +// .await +// .into_iter() +// .collect::>>() +// } + +// /// Collect `count` events from the `events` stream, only collecting events for which `matcher` +// /// returns true. +// async fn wait_for_events( +// mut events: impl Stream> + Send + Unpin + 'static, +// count: usize, +// timeout: Duration, +// matcher: impl Fn(&LiveEvent) -> bool, +// ) -> anyhow::Result> { +// let mut res = Vec::with_capacity(count); +// let sleep = tokio::time::sleep(timeout); +// tokio::pin!(sleep); +// while res.len() < count { +// tokio::select! { +// () = &mut sleep => { +// bail!("Failed to collect {count} elements in {timeout:?} (collected only {})", res.len()); +// }, +// event = events.try_next() => { +// let event = event?; +// match event { +// None => bail!("stream ended after {} items, but expected {count}", res.len()), +// Some(event) => if matcher(&event) { +// res.push(event); +// debug!("recv event {} of {count}", res.len()); +// } +// } +// } +// } +// } +// Ok(res) +// } + +// async fn assert_all_docs( +// docs: &[Doc], +// node_ids: &[PublicKey], +// expected: &Vec, +// label: &str, +// ) { +// info!("validate all peers: {label}"); +// for (i, doc) in docs.iter().enumerate() { +// let entries = get_all(doc).await.unwrap_or_else(|err| { +// panic!("failed to get entries for peer {:?}: {err:?}", node_ids[i]) +// }); +// assert_eq!( +// &entries, +// expected, +// "{label}: peer {i} {:?} failed (have {} but expected {})", +// node_ids[i], +// entries.len(), +// expected.len() +// ); +// } +// } + +// #[derive(Debug, Ord, Eq, PartialEq, PartialOrd, Clone)] +// struct ExpectedEntry { +// author: AuthorId, +// key: String, +// value: String, +// } + +// impl PartialEq for ExpectedEntry { +// fn eq(&self, other: &Entry) -> bool { +// self.key.as_bytes() == other.key() +// && Hash::new(&self.value) == other.content_hash() +// && self.author == other.author() +// } +// } +// impl PartialEq<(Entry, Bytes)> for ExpectedEntry { +// fn eq(&self, (entry, content): &(Entry, Bytes)) -> bool { +// self.key.as_bytes() == entry.key() +// && Hash::new(&self.value) == entry.content_hash() +// && self.author == entry.author() +// && self.value.as_bytes() == content.as_ref() +// } +// } +// impl PartialEq for Entry { +// fn eq(&self, other: &ExpectedEntry) -> bool { +// other.eq(self) +// } +// } +// impl PartialEq for (Entry, Bytes) { +// fn eq(&self, other: &ExpectedEntry) -> bool { +// other.eq(self) +// } +// } + +// #[tokio::test] +// async fn doc_delete() -> Result<()> { +// let node = Node::memory() +// .gc_policy(iroh::node::GcPolicy::Interval(Duration::from_millis(100))) +// .enable_docs() +// .spawn() +// .await?; +// let client = node.client(); +// let doc = client.docs().create().await?; +// let blobs = client.blobs(); +// let author = client.authors().create().await?; +// let hash = doc +// .set_bytes(author, b"foo".to_vec(), b"hi".to_vec()) +// .await?; +// assert_latest(&blobs, &doc, b"foo", b"hi").await; +// let deleted = doc.del(author, b"foo".to_vec()).await?; +// assert_eq!(deleted, 1); + +// let entry = doc.get_exact(author, b"foo".to_vec(), false).await?; +// assert!(entry.is_none()); + +// // wait for gc +// // TODO: allow to manually trigger gc +// tokio::time::sleep(Duration::from_millis(200)).await; +// let bytes = client.blobs().read_to_bytes(hash).await; +// assert!(bytes.is_err()); +// node.shutdown().await?; +// Ok(()) +// } + +// #[tokio::test] +// async fn sync_drop_doc() -> Result<()> { +// let mut rng = test_rng(b"sync_drop_doc"); +// setup_logging(); +// let node = spawn_node(0, &mut rng).await?; +// let client = node.client(); + +// let doc = client.docs().create().await?; +// let author = client.authors().create().await?; + +// let mut sub = doc.subscribe().await?; +// doc.set_bytes(author, b"foo".to_vec(), b"bar".to_vec()) +// .await?; +// let ev = sub.next().await; +// assert!(matches!(ev, Some(Ok(LiveEvent::InsertLocal { .. })))); + +// client.docs().drop_doc(doc.id()).await?; +// let res = doc.get_exact(author, b"foo".to_vec(), true).await; +// assert!(res.is_err()); +// let res = doc +// .set_bytes(author, b"foo".to_vec(), b"bar".to_vec()) +// .await; +// assert!(res.is_err()); +// let res = client.docs().open(doc.id()).await; +// assert!(res.is_err()); +// let ev = sub.next().await; +// assert!(ev.is_none()); + +// Ok(()) +// } + +// async fn assert_latest( +// blobs: &client::blobs::Client, +// doc: &Doc, +// key: &[u8], +// value: &[u8], +// ) { +// let content = get_latest(blobs, doc, key).await.unwrap(); +// assert_eq!(content, value.to_vec()); +// } + +// async fn get_latest( +// blobs: &client::blobs::Client, +// doc: &Doc, +// key: &[u8], +// ) -> anyhow::Result> { +// let query = Query::single_latest_per_key().key_exact(key); +// let entry = doc +// .get_many(query) +// .await? +// .next() +// .await +// .ok_or_else(|| anyhow!("entry not found"))??; +// let content = blobs.read_to_bytes(entry.content_hash()).await?; +// Ok(content.to_vec()) +// } + +// fn setup_logging() { +// tracing_subscriber::registry() +// .with(tracing_subscriber::fmt::layer().with_writer(std::io::stderr)) +// .with(EnvFilter::from_default_env()) +// .try_init() +// .ok(); +// } + +// async fn next(mut stream: impl Stream> + Unpin) -> T { +// let event = stream +// .next() +// .await +// .expect("stream ended") +// .expect("stream produced error"); +// debug!("Event: {event:?}"); +// event +// } + +// #[allow(clippy::type_complexity)] +// fn apply_matchers(item: &T, matchers: &mut Vec bool + Send>>) -> bool { +// for i in 0..matchers.len() { +// if matchers[i](item) { +// let _ = matchers.remove(i); +// return true; +// } +// } +// false +// } + +// /// Receive the next `matchers.len()` elements from a stream and matches them against the functions +// /// in `matchers`, in order. +// /// +// /// Returns all received events. +// #[allow(clippy::type_complexity)] +// async fn assert_next( +// mut stream: impl Stream> + Unpin + Send, +// timeout: Duration, +// matchers: Vec bool + Send>>, +// ) -> Vec { +// let fut = async { +// let mut items = vec![]; +// for (i, f) in matchers.iter().enumerate() { +// let item = stream +// .next() +// .await +// .expect("event stream ended prematurely") +// .expect("event stream errored"); +// if !(f)(&item) { +// panic!("assertion failed for event {i} {item:?}"); +// } +// items.push(item); +// } +// items +// }; +// let res = tokio::time::timeout(timeout, fut).await; +// res.expect("timeout reached") +// } + +// /// Receive `matchers.len()` elements from a stream and assert that each element matches one of the +// /// functions in `matchers`. +// /// +// /// Order of the matchers is not relevant. +// /// +// /// Returns all received events. +// #[allow(clippy::type_complexity)] +// async fn assert_next_unordered( +// stream: impl Stream> + Unpin + Send, +// timeout: Duration, +// matchers: Vec bool + Send>>, +// ) -> Vec { +// assert_next_unordered_with_optionals(stream, timeout, matchers, vec![]).await +// } + +// /// Receive between `min` and `max` elements from the stream and assert that each element matches +// /// either one of the matchers in `required_matchers` or in `optional_matchers`. +// /// +// /// Order of the matchers is not relevant. +// /// +// /// Will return an error if: +// /// * Any element fails to match one of the required or optional matchers +// /// * More than `max` elements were received, but not all required matchers were used yet +// /// * The timeout completes before all required matchers were used +// /// +// /// Returns all received events. +// #[allow(clippy::type_complexity)] +// async fn assert_next_unordered_with_optionals( +// mut stream: impl Stream> + Unpin + Send, +// timeout: Duration, +// mut required_matchers: Vec bool + Send>>, +// mut optional_matchers: Vec bool + Send>>, +// ) -> Vec { +// let max = required_matchers.len() + optional_matchers.len(); +// let required = required_matchers.len(); +// // we have to use a mutex because rustc is not intelligent enough to realize +// // that the mutable borrow terminates when the future completes +// let events = Arc::new(parking_lot::Mutex::new(vec![])); +// let fut = async { +// while let Some(event) = stream.next().await { +// let event = event.context("failed to read from stream")?; +// let len = { +// let mut events = events.lock(); +// events.push(event.clone()); +// events.len() +// }; +// if !apply_matchers(&event, &mut required_matchers) +// && !apply_matchers(&event, &mut optional_matchers) +// { +// bail!("Event didn't match any matcher: {event:?}"); +// } +// if required_matchers.is_empty() || len == max { +// break; +// } +// } +// if !required_matchers.is_empty() { +// bail!( +// "Matched only {} of {required} required matchers", +// required - required_matchers.len() +// ); +// } +// Ok(()) +// }; +// tokio::pin!(fut); +// let res = tokio::time::timeout(timeout, fut) +// .await +// .map_err(|_| anyhow!("Timeout reached ({timeout:?})")) +// .and_then(|res| res); +// let events = events.lock().clone(); +// if let Err(err) = &res { +// println!("Received events: {events:#?}"); +// println!( +// "Received {} events, expected between {required} and {max}", +// events.len() +// ); +// panic!("Failed to receive or match all events: {err:?}"); +// } +// events +// } + +// /// Asserts that the event is a [`LiveEvent::SyncFinished`] and that the contained [`SyncEvent`] +// /// has no error and matches `peer` and `namespace`. +// fn match_sync_finished(event: &LiveEvent, peer: PublicKey) -> bool { +// let LiveEvent::SyncFinished(e) = event else { +// return false; +// }; +// e.peer == peer && e.result.is_ok() +// } From 7044d0ee547076fd8523e849e40e312199521823 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Thu, 7 Nov 2024 12:11:00 +0200 Subject: [PATCH 05/26] use latest quic-rpc --- Cargo.lock | 47 +++++----------------------- Cargo.toml | 4 +-- src/ranger.rs | 1 - src/rpc.rs | 7 +++-- src/rpc/client.rs | 78 ++++++++++++++++------------------------------- 5 files changed, 41 insertions(+), 96 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c1bce96..862d575 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1828,8 +1828,8 @@ dependencies = [ "pin-project", "portable-atomic", "postcard", - "quic-rpc 0.15.0", - "quic-rpc-derive 0.15.0", + "quic-rpc", + "quic-rpc-derive", "rand", "range-collections", "redb 1.5.1", @@ -1877,8 +1877,8 @@ dependencies = [ "portable-atomic", "postcard", "proptest", - "quic-rpc 0.13.0", - "quic-rpc-derive 0.13.0", + "quic-rpc", + "quic-rpc-derive", "rand", "rand_chacha", "rand_core", @@ -1918,8 +1918,8 @@ dependencies = [ "iroh-router", "nested_enum_utils", "postcard", - "quic-rpc 0.15.0", - "quic-rpc-derive 0.15.0", + "quic-rpc", + "quic-rpc-derive", "rand", "rand_core", "serde", @@ -3171,27 +3171,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "quic-rpc" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b0ea1bd0b3124538bb71ed8cedbe92608fd1cf227e4f5ff53fb28746737b794" -dependencies = [ - "anyhow", - "derive_more", - "educe", - "flume", - "futures-lite 2.4.0", - "futures-sink", - "futures-util", - "hex", - "pin-project", - "serde", - "slab", - "tokio", - "tracing", -] - [[package]] name = "quic-rpc" version = "0.15.0" @@ -3213,18 +3192,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "quic-rpc-derive" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94b91a3f7a42657cbfbd0c2499c1f037738eff45bb7f59c6ce3d3d9e890d141c" -dependencies = [ - "proc-macro2", - "quic-rpc 0.13.0", - "quote", - "syn 1.0.109", -] - [[package]] name = "quic-rpc-derive" version = "0.15.0" @@ -3232,7 +3199,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cbef4c942978f74ef296ae40d43d4375c9d730b65a582688a358108cfd5c0cf7" dependencies = [ "proc-macro2", - "quic-rpc 0.15.0", + "quic-rpc", "quote", "syn 1.0.109", ] diff --git a/Cargo.toml b/Cargo.toml index f272acb..28c51aa 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -60,8 +60,8 @@ tracing = "0.1" # rpc nested_enum_utils = { version = "0.1.0", optional = true } -quic-rpc = { version = "0.13", optional = true } -quic-rpc-derive = { version = "0.13", optional = true } +quic-rpc = { version = "0.15", optional = true } +quic-rpc-derive = { version = "0.15", optional = true } serde-error = { version = "0.1.3", optional = true } portable-atomic = { version = "1.9.0", optional = true } diff --git a/src/ranger.rs b/src/ranger.rs index 95b6967..693b6cf 100644 --- a/src/ranger.rs +++ b/src/ranger.rs @@ -1,6 +1,5 @@ //! Implementation of Set Reconcilliation based on //! "Range-Based Set Reconciliation" by Aljoscha Meyer. -//! use std::{cmp::Ordering, fmt::Debug}; diff --git a/src/rpc.rs b/src/rpc.rs index 9f1d9d2..700df0e 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -1,5 +1,8 @@ //! Quic RPC implemenation for docs. +use proto::RpcService; +use quic_rpc::server::{ChannelTypes, RpcChannel}; + use crate::engine::Engine; pub mod client; @@ -12,10 +15,10 @@ type RpcResult = std::result::Result; impl Engine { /// Handle a docs request from the RPC server. - pub async fn handle_rpc_request>( + pub async fn handle_rpc_request>( &self, msg: crate::rpc::proto::Request, - chan: quic_rpc::server::RpcChannel, + chan: RpcChannel, ) -> Result<(), quic_rpc::server::RpcServerError> { use crate::rpc::proto::Request::*; diff --git a/src/rpc/client.rs b/src/rpc/client.rs index f15e375..8a5ca80 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -19,7 +19,7 @@ use iroh_base::{key::PublicKey, node_addr::AddrInfoOptions}; use iroh_blobs::{export::ExportProgress, store::ExportMode, Hash}; use iroh_net::NodeAddr; use portable_atomic::{AtomicBool, Ordering}; -use quic_rpc::{client::BoxedServiceConnection, message::RpcMsg}; +use quic_rpc::{client::BoxedConnector, message::RpcMsg, Connector}; use serde::{Deserialize, Serialize}; use super::proto::{ @@ -42,22 +42,18 @@ use crate::{ /// Iroh docs client. #[derive(Debug, Clone)] -pub struct Client> { - pub(super) rpc: quic_rpc::RpcClient, +pub struct Client> { + pub(super) rpc: quic_rpc::RpcClient, } -impl Client -where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection, -{ +impl> Client { /// Creates a new docs client. - pub fn new(rpc: quic_rpc::RpcClient) -> Self { + pub fn new(rpc: quic_rpc::RpcClient) -> Self { Self { rpc } } /// Creates a client. - pub async fn create(&self) -> Result> { + pub async fn create(&self) -> Result> { let res = self.rpc.rpc(CreateRequest {}).await??; let doc = Doc::new(self.rpc.clone(), res.id); Ok(doc) @@ -76,14 +72,14 @@ where /// Imports a document from a namespace capability. /// /// This does not start sync automatically. Use [`Doc::start_sync`] to start sync. - pub async fn import_namespace(&self, capability: Capability) -> Result> { + pub async fn import_namespace(&self, capability: Capability) -> Result> { let res = self.rpc.rpc(ImportRequest { capability }).await??; let doc = Doc::new(self.rpc.clone(), res.doc_id); Ok(doc) } /// Imports a document from a ticket and joins all peers in the ticket. - pub async fn import(&self, ticket: DocTicket) -> Result> { + pub async fn import(&self, ticket: DocTicket) -> Result> { let DocTicket { capability, nodes } = ticket; let doc = self.import_namespace(capability).await?; doc.start_sync(nodes).await?; @@ -99,7 +95,7 @@ where pub async fn import_and_subscribe( &self, ticket: DocTicket, - ) -> Result<(Doc, impl Stream>)> { + ) -> Result<(Doc, impl Stream>)> { let DocTicket { capability, nodes } = ticket; let res = self.rpc.rpc(ImportRequest { capability }).await??; let doc = Doc::new(self.rpc.clone(), res.doc_id); @@ -117,7 +113,7 @@ where /// Returns a [`Doc`] client for a single document. /// /// Returns None if the document cannot be found. - pub async fn open(&self, id: NamespaceId) -> Result>> { + pub async fn open(&self, id: NamespaceId) -> Result>> { self.rpc.rpc(OpenRequest { doc_id: id }).await??; let doc = Doc::new(self.rpc.clone(), id); Ok(Some(doc)) @@ -195,44 +191,29 @@ where /// Document handle #[derive(Debug, Clone)] -pub struct Doc>(Arc>) +pub struct Doc = BoxedConnector>(Arc>) where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection; + C: quic_rpc::Connector; -impl PartialEq for Doc -where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection, -{ +impl> PartialEq for Doc { fn eq(&self, other: &Self) -> bool { self.0.id == other.0.id } } -impl Eq for Doc -where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection, -{ -} +impl> Eq for Doc {} #[derive(Debug)] -struct DocInner> -where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection, -{ +struct DocInner = BoxedConnector> { id: NamespaceId, - rpc: quic_rpc::RpcClient, + rpc: quic_rpc::RpcClient, closed: AtomicBool, rt: tokio::runtime::Handle, } -impl Drop for DocInner +impl Drop for DocInner where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection, + C: quic_rpc::Connector, { fn drop(&mut self) { let doc_id = self.id; @@ -245,12 +226,8 @@ where } } -impl Doc -where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection, -{ - fn new(rpc: quic_rpc::RpcClient, id: NamespaceId) -> Self { +impl> Doc { + fn new(rpc: quic_rpc::RpcClient, id: NamespaceId) -> Self { Self(Arc::new(DocInner { rpc, id, @@ -516,12 +493,11 @@ where } } -impl<'a, S, C> From<&'a Doc> for &'a quic_rpc::RpcClient +impl<'a, C> From<&'a Doc> for &'a quic_rpc::RpcClient where - S: quic_rpc::Service, - C: quic_rpc::ServiceConnection, + C: quic_rpc::Connector, { - fn from(doc: &'a Doc) -> &'a quic_rpc::RpcClient { + fn from(doc: &'a Doc) -> &'a quic_rpc::RpcClient { &doc.0.rpc } } @@ -890,6 +866,7 @@ mod tests { local_pool.handle().clone(), Default::default(), downloader.clone(), + endpoint.clone(), ); router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), Arc::new(blobs)); @@ -914,10 +891,9 @@ mod tests { let router = router.spawn().await?; // Setup RPC - let (internal_rpc, controller) = - quic_rpc::transport::flume::service_connection::(32); - let controller = quic_rpc::transport::boxed::Connection::new(controller); - let internal_rpc = quic_rpc::transport::boxed::ServerEndpoint::new(internal_rpc); + let (internal_rpc, controller) = quic_rpc::transport::flume::channel(32); + let controller = quic_rpc::transport::boxed::BoxedConnector::new(controller); + let internal_rpc = quic_rpc::transport::boxed::BoxedListener::new(internal_rpc); let internal_rpc = quic_rpc::RpcServer::new(internal_rpc); let rpc_server_task = tokio::task::spawn(async move { From 979c33a82ff341f82bbceb16583d7632531413c6 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Thu, 7 Nov 2024 16:54:47 +0200 Subject: [PATCH 06/26] Add blob store to engine itself as well --- src/engine.rs | 17 ++++++++---- src/protocol.rs | 2 +- src/rpc.rs | 2 +- src/rpc/client.rs | 7 ++--- src/rpc/docs_handle_request.rs | 50 +++++++++++++++++----------------- src/rpc/proto.rs | 12 ++++---- 6 files changed, 47 insertions(+), 43 deletions(-) diff --git a/src/engine.rs b/src/engine.rs index 36f6dcc..b9727e2 100644 --- a/src/engine.rs +++ b/src/engine.rs @@ -42,7 +42,7 @@ const SUBSCRIBE_CHANNEL_CAP: usize = 256; /// The sync engine coordinates actors that manage open documents, set-reconciliation syncs with /// peers and a gossip swarm for each syncing document. #[derive(derive_more::Debug, Clone)] -pub struct Engine { +pub struct Engine { /// [`Endpoint`] used by the engine. pub endpoint: Endpoint, /// Handle to the actor thread. @@ -55,18 +55,19 @@ pub struct Engine { #[debug("ContentStatusCallback")] content_status_cb: ContentStatusCallback, local_pool_handle: LocalPoolHandle, + blob_store: D, } -impl Engine { +impl Engine { /// Start the sync engine. /// /// This will spawn two tokio tasks for the live sync coordination and gossip actors, and a /// thread for the [`crate::actor::SyncHandle`]. - pub async fn spawn( + pub async fn spawn( endpoint: Endpoint, gossip: Gossip, replica_store: crate::store::Store, - bao_store: B, + bao_store: D, downloader: Downloader, default_author_storage: DefaultAuthorStorage, local_pool_handle: LocalPoolHandle, @@ -84,7 +85,7 @@ impl Engine { sync.clone(), endpoint.clone(), gossip.clone(), - bao_store, + bao_store.clone(), downloader, to_live_actor_recv, live_actor_tx.clone(), @@ -116,9 +117,15 @@ impl Engine { content_status_cb, default_author: Arc::new(default_author), local_pool_handle, + blob_store: bao_store, }) } + /// Get the blob store. + pub fn blob_store(&self) -> &D { + &self.blob_store + } + /// Start to sync a document. /// /// If `peers` is non-empty, it will both do an initial set-reconciliation sync with each peer, diff --git a/src/protocol.rs b/src/protocol.rs index 18bccc5..c551584 100644 --- a/src/protocol.rs +++ b/src/protocol.rs @@ -9,7 +9,7 @@ use iroh_router::ProtocolHandler; use crate::engine::Engine; -impl ProtocolHandler for Engine { +impl ProtocolHandler for Engine { fn accept(self: Arc, conn: Connecting) -> BoxedFuture> { Box::pin(async move { self.handle_connection(conn).await }) } diff --git a/src/rpc.rs b/src/rpc.rs index 700df0e..1c59d37 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -13,7 +13,7 @@ mod docs_handle_request; type RpcError = serde_error::Error; type RpcResult = std::result::Result; -impl Engine { +impl Engine { /// Handle a docs request from the RPC server. pub async fn handle_rpc_request>( &self, diff --git a/src/rpc/client.rs b/src/rpc/client.rs index 8a5ca80..e1a1693 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -1,9 +1,6 @@ //! API for document management. //! //! The main entry point is the [`Client`]. -//! -//! You obtain a [`Client`] via [`Iroh::docs()`](crate::client::Iroh::docs). - use std::{ path::{Path, PathBuf}, pin::Pin, @@ -124,7 +121,7 @@ impl> Client { /// You likely want to save the returned [`AuthorId`] somewhere so that you can use this author /// again. /// - /// If you need only a single author, use [`Self::default`]. + /// If you need only a single author, use [`Self::author_default`]. pub async fn author_create(&self) -> Result { let res = self.rpc.rpc(AuthorCreateRequest).await??; Ok(res.author_id) @@ -135,7 +132,7 @@ impl> Client { /// On persistent nodes, the author is created on first start and its public key is saved /// in the data directory. /// - /// The default author can be set with [`Self::set_default`]. + /// The default author can be set with [`Self::author_set_default`]. pub async fn author_default(&self) -> Result { let res = self.rpc.rpc(AuthorGetDefaultRequest).await??; Ok(res.author_id) diff --git a/src/rpc/docs_handle_request.rs b/src/rpc/docs_handle_request.rs index 44cb14f..aa0a9a1 100644 --- a/src/rpc/docs_handle_request.rs +++ b/src/rpc/docs_handle_request.rs @@ -6,6 +6,7 @@ use iroh_blobs::{ export::ExportProgress, store::ImportProgress, util::progress::{AsyncChannelProgressSender, ProgressSender}, + BlobFormat, }; use super::{ @@ -33,7 +34,7 @@ use crate::{engine::Engine, Author, DocTicket, NamespaceSecret}; /// Capacity for the flume channels to forward sync store iterators to async RPC streams. const ITER_CHANNEL_CAP: usize = 64; -impl Engine { +impl Engine { pub(super) async fn author_create( self, _req: AuthorCreateRequest, @@ -283,30 +284,29 @@ impl Engine { } pub(super) async fn doc_set(self, req: SetRequest) -> RpcResult { - todo!() - // let blobs_store = self.blobs_store(); - // let SetRequest { - // doc_id, - // author_id, - // key, - // value, - // } = req; - // let len = value.len(); - // let tag = blobs_store - // .import_bytes(value, BlobFormat::Raw) - // .await - // .map_err(|e| RpcError::new(&e))?; - // self.sync - // .insert_local(doc_id, author_id, key.clone(), *tag.hash(), len as u64) - // .await - // .map_err(|e| RpcError::new(&*e))?; - // let entry = self - // .sync - // .get_exact(doc_id, author_id, key, false) - // .await - // .map_err(|e| RpcError::new(&*e))? - // .ok_or_else(|| RpcError::new(&*anyhow!("failed to get entry after insertion")))?; - // Ok(SetResponse { entry }) + let blobs_store = self.blob_store(); + let SetRequest { + doc_id, + author_id, + key, + value, + } = req; + let len = value.len(); + let tag = blobs_store + .import_bytes(value, BlobFormat::Raw) + .await + .map_err(|e| RpcError::new(&e))?; + self.sync + .insert_local(doc_id, author_id, key.clone(), *tag.hash(), len as u64) + .await + .map_err(|e| RpcError::new(&*e))?; + let entry = self + .sync + .get_exact(doc_id, author_id, key, false) + .await + .map_err(|e| RpcError::new(&*e))? + .ok_or_else(|| RpcError::new(&*anyhow!("failed to get entry after insertion")))?; + Ok(SetResponse { entry }) } pub(super) async fn doc_del(self, req: DelRequest) -> RpcResult { diff --git a/src/rpc/proto.rs b/src/rpc/proto.rs index 62cff1d..4b0581a 100644 --- a/src/rpc/proto.rs +++ b/src/rpc/proto.rs @@ -460,7 +460,7 @@ pub struct GetSyncPeersResponse { #[derive(Serialize, Deserialize, Debug)] pub struct AuthorListRequest {} -/// Response for [`ListRequest`] +/// Response for [`AuthorListRequest`] #[derive(Serialize, Deserialize, Debug)] pub struct AuthorListResponse { /// The author id @@ -471,7 +471,7 @@ pub struct AuthorListResponse { #[derive(Serialize, Deserialize, Debug)] pub struct AuthorCreateRequest; -/// Response for [`CreateRequest`] +/// Response for [`AuthorCreateRequest`] #[derive(Serialize, Deserialize, Debug)] pub struct AuthorCreateResponse { /// The id of the created author @@ -482,7 +482,7 @@ pub struct AuthorCreateResponse { #[derive(Serialize, Deserialize, Debug)] pub struct AuthorGetDefaultRequest; -/// Response for [`GetDefaultRequest`] +/// Response for [`AuthorGetDefaultRequest`] #[derive(Serialize, Deserialize, Debug)] pub struct AuthorGetDefaultResponse { /// The id of the author @@ -496,7 +496,7 @@ pub struct AuthorSetDefaultRequest { pub author_id: AuthorId, } -/// Response for [`GetDefaultRequest`] +/// Response for [`AuthorSetDefaultRequest`] #[derive(Serialize, Deserialize, Debug)] pub struct AuthorSetDefaultResponse; @@ -507,7 +507,7 @@ pub struct AuthorDeleteRequest { pub author: AuthorId, } -/// Response for [`DeleteRequest`] +/// Response for [`AuthorDeleteRequest`] #[derive(Serialize, Deserialize, Debug)] pub struct AuthorDeleteResponse; @@ -518,7 +518,7 @@ pub struct AuthorExportRequest { pub author: AuthorId, } -/// Response for [`ExportRequest`] +/// Response for [`AuthorExportRequest`] #[derive(Serialize, Deserialize, Debug)] pub struct AuthorExportResponse { /// The author From cad1d0bdf8ea6f9ea6e54915fd02f4d2feb483a2 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Thu, 7 Nov 2024 16:59:42 +0200 Subject: [PATCH 07/26] reenable blobs related fns --- src/rpc/client.rs | 4 +- src/rpc/docs_handle_request.rs | 67 ++++++++++++++++------------------ 2 files changed, 34 insertions(+), 37 deletions(-) diff --git a/src/rpc/client.rs b/src/rpc/client.rs index e1a1693..50d3746 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -822,8 +822,8 @@ where mod tests { use iroh_blobs::util::local_pool::LocalPool; use iroh_gossip::net::GOSSIP_ALPN; - use rand::RngCore; - use tokio::io::AsyncWriteExt; + // use rand::RngCore; + // use tokio::io::AsyncWriteExt; use tracing::warn; use super::*; diff --git a/src/rpc/docs_handle_request.rs b/src/rpc/docs_handle_request.rs index aa0a9a1..6938c32 100644 --- a/src/rpc/docs_handle_request.rs +++ b/src/rpc/docs_handle_request.rs @@ -4,9 +4,9 @@ use anyhow::anyhow; use futures_lite::{Stream, StreamExt}; use iroh_blobs::{ export::ExportProgress, - store::ImportProgress, + store::{ExportFormat, ImportProgress}, util::progress::{AsyncChannelProgressSender, ProgressSender}, - BlobFormat, + BlobFormat, HashAndFormat, }; use super::{ @@ -478,26 +478,24 @@ impl Engine { false => ImportMode::Copy, }; - todo!() - // let blobs = self.blobs(); - // let (temp_tag, size) = blobs - // .store() - // .import_file(root, import_mode, BlobFormat::Raw, import_progress) - // .await?; - - // let hash_and_format = temp_tag.inner(); - // let HashAndFormat { hash, .. } = *hash_and_format; - // self.doc_set_hash(SetHashRequest { - // doc_id, - // author_id, - // key: key.clone(), - // hash, - // size, - // }) - // .await?; - // drop(temp_tag); - // progress.send(DocImportProgress::AllDone { key }).await?; - // Ok(()) + let blobs = self.blob_store(); + let (temp_tag, size) = blobs + .import_file(root, import_mode, BlobFormat::Raw, import_progress) + .await?; + + let hash_and_format = temp_tag.inner(); + let HashAndFormat { hash, .. } = *hash_and_format; + self.doc_set_hash(SetHashRequest { + doc_id, + author_id, + key: key.clone(), + hash, + size, + }) + .await?; + drop(temp_tag); + progress.send(DocImportProgress::AllDone { key }).await?; + Ok(()) } pub(super) fn doc_export_file( @@ -533,18 +531,17 @@ impl Engine { x }); - todo!() - // let blobs = self.blobs(); - // iroh_blobs::export::export( - // blobs.store(), - // entry.content_hash(), - // path, - // ExportFormat::Blob, - // mode, - // export_progress, - // ) - // .await?; - // progress.send(ExportProgress::AllDone).await?; - // Ok(()) + let blobs = self.blob_store(); + iroh_blobs::export::export( + blobs, + entry.content_hash(), + path, + ExportFormat::Blob, + mode, + export_progress, + ) + .await?; + progress.send(ExportProgress::AllDone).await?; + Ok(()) } } From 2e988b068af3223f39a10314ffa5e8c817b7282e Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Thu, 7 Nov 2024 17:11:53 +0200 Subject: [PATCH 08/26] Add unicode-3.0 licnese to deny.toml allow list --- deny.toml | 1 + src/rpc.rs | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/deny.toml b/deny.toml index 310d8bd..4197e83 100644 --- a/deny.toml +++ b/deny.toml @@ -21,6 +21,7 @@ allow = [ "Unicode-DFS-2016", "Zlib", "MPL-2.0", # https://fossa.com/blog/open-source-software-licenses-101-mozilla-public-license-2-0/ + "Unicode-3.0" ] [[licenses.clarify]] diff --git a/src/rpc.rs b/src/rpc.rs index 1c59d37..8e196f9 100644 --- a/src/rpc.rs +++ b/src/rpc.rs @@ -1,4 +1,4 @@ -//! Quic RPC implemenation for docs. +//! Quic RPC implementation for docs. use proto::RpcService; use quic_rpc::server::{ChannelTypes, RpcChannel}; From 02de27d5914244cb27f023befafcd53abdb6251c Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Thu, 7 Nov 2024 17:14:03 +0200 Subject: [PATCH 09/26] allow iroh-gossip git --- deny.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/deny.toml b/deny.toml index 4197e83..c5e36d1 100644 --- a/deny.toml +++ b/deny.toml @@ -40,4 +40,5 @@ ignore = [ allow-git = [ "https://github.com/n0-computer/iroh.git", "https://github.com/n0-computer/iroh-blobs.git", + "https://github.com/n0-computer/iroh-gossip.git", ] From fc83d5ce4e3677d28e9151d22a65ac69de28a76d Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Fri, 8 Nov 2024 14:15:50 +0200 Subject: [PATCH 10/26] Add complete iroh gossip and docs node so that tests pass. --- Cargo.lock | 9 + Cargo.toml | 4 + tests/gc.rs | 7 +- tests/sync.rs | 2797 ++++++++++++++++++++++++------------------------- tests/util.rs | 273 +++++ 5 files changed, 1683 insertions(+), 1407 deletions(-) create mode 100644 tests/util.rs diff --git a/Cargo.lock b/Cargo.lock index 862d575..c06db74 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1874,6 +1874,7 @@ dependencies = [ "lru", "nested_enum_utils", "num_enum", + "parking_lot", "portable-atomic", "postcard", "proptest", @@ -1890,11 +1891,13 @@ dependencies = [ "strum", "tempfile", "test-strategy", + "testresult", "thiserror", "tokio", "tokio-stream", "tokio-util", "tracing", + "tracing-subscriber", ] [[package]] @@ -4297,6 +4300,12 @@ dependencies = [ "syn 2.0.87", ] +[[package]] +name = "testresult" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "614b328ff036a4ef882c61570f72918f7e9c5bee1da33f8e7f91e01daee7e56c" + [[package]] name = "thiserror" version = "1.0.68" diff --git a/Cargo.toml b/Cargo.toml index 28c51aa..10243ec 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -72,6 +72,10 @@ tokio = { version = "1", features = ["sync", "macros"] } proptest = "1.2.0" tempfile = "3.4" test-strategy = "0.3.1" +tracing-subscriber = { version = "0.3.18", features = ["env-filter"] } +parking_lot = "0.12.3" +testresult = "0.4.1" +nested_enum_utils = "0.1.0" [features] default = ["net", "metrics", "engine", "rpc"] diff --git a/tests/gc.rs b/tests/gc.rs index 3156e8b..366eacc 100644 --- a/tests/gc.rs +++ b/tests/gc.rs @@ -1,7 +1,10 @@ -// TODO: +// use std::time::Duration; + +// use bytes::Bytes; +// use testresult::TestResult; // #[tokio::test] -// async fn redb_doc_import_stress() -> Result<()> { +// async fn redb_doc_import_stress() -> TestResult<()> { // let _ = tracing_subscriber::fmt::try_init(); // let dir = testdir!(); // let bao_store = iroh_blobs::store::fs::Store::load(dir.join("store")).await?; diff --git a/tests/sync.rs b/tests/sync.rs index c0cb409..8263e87 100644 --- a/tests/sync.rs +++ b/tests/sync.rs @@ -1,1405 +1,1392 @@ -// TODO: - -// use std::{ -// collections::HashMap, -// future::Future, -// sync::Arc, -// time::{Duration, Instant}, -// }; - -// use anyhow::{anyhow, bail, Context, Result}; -// use bytes::Bytes; -// use futures_lite::Stream; -// use futures_util::{FutureExt, StreamExt, TryStreamExt}; -// use iroh::{ -// base::node_addr::AddrInfoOptions, -// client::{ -// self, -// docs::{Entry, LiveEvent, ShareMode}, -// Doc, RpcService, -// }, -// net::key::{PublicKey, SecretKey}, -// node::{Builder, Node}, -// }; -// use iroh_blobs::Hash; -// use iroh_docs::{ -// store::{DownloadPolicy, FilterKind, Query}, -// AuthorId, ContentStatus, -// }; -// use iroh_net::relay::RelayMode; -// use rand::{CryptoRng, Rng, SeedableRng}; -// use tracing::{debug, error_span, info, Instrument}; -// use tracing_subscriber::{prelude::*, EnvFilter}; - -// const TIMEOUT: Duration = Duration::from_secs(60); - -// fn test_node(secret_key: SecretKey) -> Builder { -// Node::memory() -// .secret_key(secret_key) -// .enable_docs() -// .relay_mode(RelayMode::Disabled) -// } - -// // The function is not `async fn` so that we can take a `&mut` borrow on the `rng` without -// // capturing that `&mut` lifetime in the returned future. This allows to call it in a loop while -// // still collecting the futures before awaiting them altogether (see [`spawn_nodes`]) -// fn spawn_node( -// i: usize, -// rng: &mut (impl CryptoRng + Rng), -// ) -> impl Future>> + 'static { -// let secret_key = SecretKey::generate_with_rng(rng); -// async move { -// let node = test_node(secret_key); -// let node = node.spawn().await?; -// info!(?i, me = %node.node_id().fmt_short(), "node spawned"); -// Ok(node) -// } -// } - -// async fn spawn_nodes( -// n: usize, -// mut rng: &mut (impl CryptoRng + Rng), -// ) -> anyhow::Result>> { -// let mut futs = vec![]; -// for i in 0..n { -// futs.push(spawn_node(i, &mut rng)); -// } -// futures_buffered::join_all(futs).await.into_iter().collect() -// } - -// pub fn test_rng(seed: &[u8]) -> rand_chacha::ChaCha12Rng { -// rand_chacha::ChaCha12Rng::from_seed(*Hash::new(seed).as_bytes()) -// } - -// macro_rules! match_event { -// ($pattern:pat $(if $guard:expr)? $(,)?) => { -// Box::new(move |e| matches!(e, $pattern $(if $guard)?)) -// }; -// } - -// /// This tests the simplest scenario: A node connects to another node, and performs sync. -// #[tokio::test] -// async fn sync_simple() -> Result<()> { -// setup_logging(); -// let mut rng = test_rng(b"sync_simple"); -// let nodes = spawn_nodes(2, &mut rng).await?; -// let clients = nodes.iter().map(|node| node.client()).collect::>(); - -// // create doc on node0 -// let peer0 = nodes[0].node_id(); -// let author0 = clients[0].authors().create().await?; -// let doc0 = clients[0].docs().create().await?; -// let blobs0 = clients[0].blobs(); -// let hash0 = doc0 -// .set_bytes(author0, b"k1".to_vec(), b"v1".to_vec()) -// .await?; -// assert_latest(&blobs0, &doc0, b"k1", b"v1").await; -// let ticket = doc0 -// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) -// .await?; - -// let mut events0 = doc0.subscribe().await?; - -// info!("node1: join"); -// let peer1 = nodes[1].node_id(); -// let doc1 = clients[1].docs().import(ticket.clone()).await?; -// let blobs1 = clients[1].blobs(); -// let mut events1 = doc1.subscribe().await?; -// info!("node1: assert 5 events"); -// assert_next_unordered( -// &mut events1, -// TIMEOUT, -// vec![ -// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer0)), -// Box::new(move |e| matches!(e, LiveEvent::InsertRemote { from, .. } if *from == peer0 )), -// Box::new(move |e| match_sync_finished(e, peer0)), -// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash0)), -// match_event!(LiveEvent::PendingContentReady), -// ], -// ) -// .await; -// assert_latest(&blobs1, &doc1, b"k1", b"v1").await; - -// info!("node0: assert 2 events"); -// assert_next( -// &mut events0, -// TIMEOUT, -// vec![ -// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer1)), -// Box::new(move |e| match_sync_finished(e, peer1)), -// ], -// ) -// .await; - -// for node in nodes { -// node.shutdown().await?; -// } -// Ok(()) -// } - -// /// Test subscribing to replica events (without sync) -// #[tokio::test] -// async fn sync_subscribe_no_sync() -> Result<()> { -// let mut rng = test_rng(b"sync_subscribe"); -// setup_logging(); -// let node = spawn_node(0, &mut rng).await?; -// let client = node.client(); -// let doc = client.docs().create().await?; -// let mut sub = doc.subscribe().await?; -// let author = client.authors().create().await?; -// doc.set_bytes(author, b"k".to_vec(), b"v".to_vec()).await?; -// let event = tokio::time::timeout(Duration::from_millis(100), sub.next()).await?; -// assert!( -// matches!(event, Some(Ok(LiveEvent::InsertLocal { .. }))), -// "expected InsertLocal but got {event:?}" -// ); -// node.shutdown().await?; -// Ok(()) -// } - -// #[tokio::test] -// async fn sync_gossip_bulk() -> Result<()> { -// let n_entries: usize = std::env::var("N_ENTRIES") -// .map(|x| x.parse().expect("N_ENTRIES must be a number")) -// .unwrap_or(100); -// let mut rng = test_rng(b"sync_gossip_bulk"); -// setup_logging(); - -// let nodes = spawn_nodes(2, &mut rng).await?; -// let clients = nodes.iter().map(|node| node.client()).collect::>(); - -// let _peer0 = nodes[0].node_id(); -// let author0 = clients[0].authors().create().await?; -// let doc0 = clients[0].docs().create().await?; -// let mut ticket = doc0 -// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) -// .await?; -// // unset peers to not yet start sync -// let peers = ticket.nodes.clone(); -// ticket.nodes = vec![]; -// let doc1 = clients[1].docs().import(ticket).await?; -// let mut events = doc1.subscribe().await?; - -// // create entries for initial sync. -// let now = Instant::now(); -// let value = b"foo"; -// for i in 0..n_entries { -// let key = format!("init/{i}"); -// doc0.set_bytes(author0, key.as_bytes().to_vec(), value.to_vec()) -// .await?; -// } -// let elapsed = now.elapsed(); -// info!( -// "insert took {elapsed:?} for {n_entries} ({:?} per entry)", -// elapsed / n_entries as u32 -// ); - -// let now = Instant::now(); -// let mut count = 0; -// doc0.start_sync(vec![]).await?; -// doc1.start_sync(peers).await?; -// while let Some(event) = events.next().await { -// let event = event?; -// if matches!(event, LiveEvent::InsertRemote { .. }) { -// count += 1; -// } -// if count == n_entries { -// break; -// } -// } -// let elapsed = now.elapsed(); -// info!( -// "initial sync took {elapsed:?} for {n_entries} ({:?} per entry)", -// elapsed / n_entries as u32 -// ); - -// // publish another 1000 entries -// let mut count = 0; -// let value = b"foo"; -// let now = Instant::now(); -// for i in 0..n_entries { -// let key = format!("gossip/{i}"); -// doc0.set_bytes(author0, key.as_bytes().to_vec(), value.to_vec()) -// .await?; -// } -// let elapsed = now.elapsed(); -// info!( -// "insert took {elapsed:?} for {n_entries} ({:?} per entry)", -// elapsed / n_entries as u32 -// ); - -// while let Some(event) = events.next().await { -// let event = event?; -// if matches!(event, LiveEvent::InsertRemote { .. }) { -// count += 1; -// } -// if count == n_entries { -// break; -// } -// } -// let elapsed = now.elapsed(); -// info!( -// "gossip recv took {elapsed:?} for {n_entries} ({:?} per entry)", -// elapsed / n_entries as u32 -// ); - -// Ok(()) -// } - -// /// This tests basic sync and gossip with 3 peers. -// #[tokio::test] -// #[ignore = "flaky"] -// async fn sync_full_basic() -> Result<()> { -// let mut rng = test_rng(b"sync_full_basic"); -// setup_logging(); -// let mut nodes = spawn_nodes(2, &mut rng).await?; -// let mut clients = nodes -// .iter() -// .map(|node| node.client().clone()) -// .collect::>(); - -// // peer0: create doc and ticket -// let peer0 = nodes[0].node_id(); -// let author0 = clients[0].authors().create().await?; -// let doc0 = clients[0].docs().create().await?; -// let blobs0 = clients[0].blobs(); -// let mut events0 = doc0.subscribe().await?; -// let key0 = b"k1"; -// let value0 = b"v1"; -// let hash0 = doc0 -// .set_bytes(author0, key0.to_vec(), value0.to_vec()) -// .await?; - -// info!("peer0: wait for 1 event (local insert)"); -// let e = next(&mut events0).await; -// assert!( -// matches!(&e, LiveEvent::InsertLocal { entry } if entry.content_hash() == hash0), -// "expected LiveEvent::InsertLocal but got {e:?}", -// ); -// assert_latest(&blobs0, &doc0, key0, value0).await; -// let ticket = doc0 -// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) -// .await?; - -// info!("peer1: spawn"); -// let peer1 = nodes[1].node_id(); -// let author1 = clients[1].authors().create().await?; -// info!("peer1: join doc"); -// let doc1 = clients[1].docs().import(ticket.clone()).await?; -// let blobs1 = clients[1].blobs(); - -// info!("peer1: wait for 4 events (for sync and join with peer0)"); -// let mut events1 = doc1.subscribe().await?; -// assert_next_unordered( -// &mut events1, -// TIMEOUT, -// vec![ -// match_event!(LiveEvent::NeighborUp(peer) if *peer == peer0), -// match_event!(LiveEvent::InsertRemote { from, .. } if *from == peer0 ), -// Box::new(move |e| match_sync_finished(e, peer0)), -// match_event!(LiveEvent::ContentReady { hash } if *hash == hash0), -// match_event!(LiveEvent::PendingContentReady), -// ], -// ) -// .await; - -// info!("peer0: wait for 2 events (join & accept sync finished from peer1)"); -// assert_next( -// &mut events0, -// TIMEOUT, -// vec![ -// match_event!(LiveEvent::NeighborUp(peer) if *peer == peer1), -// Box::new(move |e| match_sync_finished(e, peer1)), -// match_event!(LiveEvent::PendingContentReady), -// ], -// ) -// .await; - -// info!("peer1: insert entry"); -// let key1 = b"k2"; -// let value1 = b"v2"; -// let hash1 = doc1 -// .set_bytes(author1, key1.to_vec(), value1.to_vec()) -// .await?; -// assert_latest(&blobs1, &doc1, key1, value1).await; -// info!("peer1: wait for 1 event (local insert, and pendingcontentready)"); -// assert_next( -// &mut events1, -// TIMEOUT, -// vec![match_event!(LiveEvent::InsertLocal { entry} if entry.content_hash() == hash1)], -// ) -// .await; - -// // peer0: assert events for entry received via gossip -// info!("peer0: wait for 2 events (gossip'ed entry from peer1)"); -// assert_next( -// &mut events0, -// TIMEOUT, -// vec![ -// Box::new( -// move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == peer1), -// ), -// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash1)), -// ], -// ).await; -// assert_latest(&blobs0, &doc0, key1, value1).await; - -// // Note: If we could check gossip messages directly here (we can't easily), we would notice -// // that peer1 will receive a `Op::ContentReady` gossip message, broadcast -// // by peer0 with neighbor scope. This message is superfluous, and peer0 could know that, however -// // our gossip implementation does not allow us to filter message receivers this way. - -// info!("peer2: spawn"); -// nodes.push(spawn_node(nodes.len(), &mut rng).await?); -// clients.push(nodes.last().unwrap().client().clone()); -// let doc2 = clients[2].docs().import(ticket).await?; -// let blobs2 = clients[2].blobs(); -// let peer2 = nodes[2].node_id(); -// let mut events2 = doc2.subscribe().await?; - -// info!("peer2: wait for 9 events (from sync with peers)"); -// assert_next_unordered_with_optionals( -// &mut events2, -// TIMEOUT, -// // required events -// vec![ -// // 2 NeighborUp events -// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer0)), -// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer1)), -// // 2 SyncFinished events -// Box::new(move |e| match_sync_finished(e, peer0)), -// Box::new(move |e| match_sync_finished(e, peer1)), -// // 2 InsertRemote events -// Box::new( -// move |e| matches!(e, LiveEvent::InsertRemote { entry, content_status: ContentStatus::Missing, .. } if entry.content_hash() == hash0), -// ), -// Box::new( -// move |e| matches!(e, LiveEvent::InsertRemote { entry, content_status: ContentStatus::Missing, .. } if entry.content_hash() == hash1), -// ), -// // 2 ContentReady events -// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash0)), -// Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash1)), -// // at least 1 PendingContentReady -// match_event!(LiveEvent::PendingContentReady), -// ], -// // optional events -// // it may happen that we run sync two times against our two peers: -// // if the first sync (as a result of us joining the peer manually through the ticket) completes -// // before the peer shows up as a neighbor, we run sync again for the NeighborUp event. -// vec![ -// // 2 SyncFinished events -// Box::new(move |e| match_sync_finished(e, peer0)), -// Box::new(move |e| match_sync_finished(e, peer1)), -// match_event!(LiveEvent::PendingContentReady), -// match_event!(LiveEvent::PendingContentReady), -// ] -// ).await; -// assert_latest(&blobs2, &doc2, b"k1", b"v1").await; -// assert_latest(&blobs2, &doc2, b"k2", b"v2").await; - -// info!("peer0: wait for 2 events (join & accept sync finished from peer2)"); -// assert_next( -// &mut events0, -// TIMEOUT, -// vec![ -// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer2)), -// Box::new(move |e| match_sync_finished(e, peer2)), -// match_event!(LiveEvent::PendingContentReady), -// ], -// ) -// .await; - -// info!("peer1: wait for 2 events (join & accept sync finished from peer2)"); -// assert_next( -// &mut events1, -// TIMEOUT, -// vec![ -// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer2)), -// Box::new(move |e| match_sync_finished(e, peer2)), -// match_event!(LiveEvent::PendingContentReady), -// ], -// ) -// .await; - -// info!("shutdown"); -// for node in nodes { -// node.shutdown().await?; -// } - -// Ok(()) -// } - -// #[tokio::test] -// async fn sync_open_close() -> Result<()> { -// let mut rng = test_rng(b"sync_subscribe_stop_close"); -// setup_logging(); -// let node = spawn_node(0, &mut rng).await?; -// let client = node.client(); - -// let doc = client.docs().create().await?; -// let status = doc.status().await?; -// assert_eq!(status.handles, 1); - -// let doc2 = client.docs().open(doc.id()).await?.unwrap(); -// let status = doc2.status().await?; -// assert_eq!(status.handles, 2); - -// doc.close().await?; -// assert!(doc.status().await.is_err()); - -// let status = doc2.status().await?; -// assert_eq!(status.handles, 1); - -// Ok(()) -// } - -// #[tokio::test] -// async fn sync_subscribe_stop_close() -> Result<()> { -// let mut rng = test_rng(b"sync_subscribe_stop_close"); -// setup_logging(); -// let node = spawn_node(0, &mut rng).await?; -// let client = node.client(); - -// let doc = client.docs().create().await?; -// let author = client.authors().create().await?; - -// let status = doc.status().await?; -// assert_eq!(status.subscribers, 0); -// assert_eq!(status.handles, 1); -// assert!(!status.sync); - -// doc.start_sync(vec![]).await?; -// let status = doc.status().await?; -// assert!(status.sync); -// assert_eq!(status.handles, 2); -// assert_eq!(status.subscribers, 1); - -// let sub = doc.subscribe().await?; -// let status = doc.status().await?; -// assert_eq!(status.subscribers, 2); -// drop(sub); -// // trigger an event that makes the actor check if the event channels are still connected -// doc.set_bytes(author, b"x".to_vec(), b"x".to_vec()).await?; -// let status = doc.status().await?; -// assert_eq!(status.subscribers, 1); - -// doc.leave().await?; -// let status = doc.status().await?; -// assert_eq!(status.subscribers, 0); -// assert_eq!(status.handles, 1); -// assert!(!status.sync); - -// Ok(()) -// } - -// #[tokio::test] -// #[cfg(feature = "test-utils")] -// async fn test_sync_via_relay() -> Result<()> { -// let _guard = iroh_test::logging::setup(); -// let (relay_map, _relay_url, _guard) = iroh_net::test_utils::run_relay_server().await?; - -// let node1 = Node::memory() -// .relay_mode(RelayMode::Custom(relay_map.clone())) -// .insecure_skip_relay_cert_verify(true) -// .enable_docs() -// .spawn() -// .await?; -// let node1_id = node1.node_id(); -// let node2 = Node::memory() -// .bind_random_port() -// .relay_mode(RelayMode::Custom(relay_map.clone())) -// .insecure_skip_relay_cert_verify(true) -// .enable_docs() -// .spawn() -// .await?; - -// let doc1 = node1.docs().create().await?; -// let author1 = node1.authors().create().await?; -// let inserted_hash = doc1 -// .set_bytes(author1, b"foo".to_vec(), b"bar".to_vec()) -// .await?; -// let mut ticket = doc1 -// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) -// .await?; - -// // remove direct addrs to force connect via relay -// ticket.nodes[0].info.direct_addresses = Default::default(); - -// // join -// let doc2 = node2.docs().import(ticket).await?; -// let blobs2 = node2.blobs(); -// let mut events = doc2.subscribe().await?; - -// assert_next_unordered_with_optionals( -// &mut events, -// Duration::from_secs(2), -// vec![ -// Box::new(move |e| matches!(e, LiveEvent::NeighborUp(n) if *n== node1_id)), -// Box::new(move |e| match_sync_finished(e, node1_id)), -// Box::new( -// move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == node1_id), -// ), -// Box::new( -// move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == inserted_hash), -// ), -// match_event!(LiveEvent::PendingContentReady), -// ], -// vec![Box::new(move |e| match_sync_finished(e, node1_id))], -// ).await; -// let actual = blobs2 -// .read_to_bytes( -// doc2.get_exact(author1, b"foo", false) -// .await? -// .expect("entry to exist") -// .content_hash(), -// ) -// .await?; -// assert_eq!(actual.as_ref(), b"bar"); - -// // update -// let updated_hash = doc1 -// .set_bytes(author1, b"foo".to_vec(), b"update".to_vec()) -// .await?; -// assert_next_unordered_with_optionals( -// &mut events, -// Duration::from_secs(2), -// vec![ -// Box::new( -// move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == node1_id), -// ), -// Box::new( -// move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == updated_hash), -// ), -// ], -// vec![ -// Box::new(move |e| match_sync_finished(e, node1_id)), -// Box::new(move |e| matches!(e, LiveEvent::PendingContentReady)), -// ], -// ).await; -// let actual = blobs2 -// .read_to_bytes( -// doc2.get_exact(author1, b"foo", false) -// .await? -// .expect("entry to exist") -// .content_hash(), -// ) -// .await?; -// assert_eq!(actual.as_ref(), b"update"); -// Ok(()) -// } - -// #[tokio::test] -// #[cfg(feature = "test-utils")] -// #[ignore = "flaky"] -// async fn sync_restart_node() -> Result<()> { -// let mut rng = test_rng(b"sync_restart_node"); -// setup_logging(); -// let (relay_map, _relay_url, _guard) = iroh_net::test_utils::run_relay_server().await?; - -// let discovery_server = iroh_net::test_utils::DnsPkarrServer::run().await?; - -// let node1_dir = tempfile::TempDir::with_prefix("test-sync_restart_node-node1")?; -// let secret_key_1 = SecretKey::generate_with_rng(&mut rng); - -// let node1 = Node::persistent(&node1_dir) -// .await? -// .secret_key(secret_key_1.clone()) -// .insecure_skip_relay_cert_verify(true) -// .relay_mode(RelayMode::Custom(relay_map.clone())) -// .dns_resolver(discovery_server.dns_resolver()) -// .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) -// .enable_docs() -// .spawn() -// .await?; -// let id1 = node1.node_id(); - -// // create doc & ticket on node1 -// let doc1 = node1.docs().create().await?; -// let blobs1 = node1.blobs().clone(); -// let mut events1 = doc1.subscribe().await?; -// let ticket = doc1 -// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) -// .await?; - -// // create node2 -// let secret_key_2 = SecretKey::generate_with_rng(&mut rng); -// let node2 = Node::memory() -// .secret_key(secret_key_2.clone()) -// .relay_mode(RelayMode::Custom(relay_map.clone())) -// .insecure_skip_relay_cert_verify(true) -// .dns_resolver(discovery_server.dns_resolver()) -// .node_discovery(discovery_server.discovery(secret_key_2.clone()).into()) -// .enable_docs() -// .spawn() -// .await?; -// let id2 = node2.node_id(); -// let author2 = node2.authors().create().await?; -// let doc2 = node2.docs().import(ticket.clone()).await?; -// let blobs2 = node2.blobs(); - -// info!("node2 set a"); -// let hash_a = doc2.set_bytes(author2, "n2/a", "a").await?; -// assert_latest(&blobs2, &doc2, b"n2/a", b"a").await; - -// assert_next_unordered_with_optionals( -// &mut events1, -// Duration::from_secs(10), -// vec![ -// match_event!(LiveEvent::NeighborUp(n) if *n == id2), -// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), -// match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), -// match_event!(LiveEvent::ContentReady { hash } if *hash == hash_a), -// match_event!(LiveEvent::PendingContentReady), -// ], -// vec![ -// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), -// match_event!(LiveEvent::PendingContentReady), -// ], -// ) -// .await; -// assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; - -// info!(me = id1.fmt_short(), "node1 start shutdown"); -// node1.shutdown().await?; -// info!(me = id1.fmt_short(), "node1 down"); - -// info!(me = id1.fmt_short(), "sleep 1s"); -// tokio::time::sleep(Duration::from_secs(1)).await; - -// info!(me = id2.fmt_short(), "node2 set b"); -// let hash_b = doc2.set_bytes(author2, "n2/b", "b").await?; - -// info!(me = id1.fmt_short(), "node1 respawn"); -// let node1 = Node::persistent(&node1_dir) -// .await? -// .secret_key(secret_key_1.clone()) -// .insecure_skip_relay_cert_verify(true) -// .relay_mode(RelayMode::Custom(relay_map.clone())) -// .dns_resolver(discovery_server.dns_resolver()) -// .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) -// .enable_docs() -// .spawn() -// .await?; -// assert_eq!(id1, node1.node_id()); - -// let doc1 = node1.docs().open(doc1.id()).await?.expect("doc to exist"); -// let blobs1 = node1.blobs(); -// let mut events1 = doc1.subscribe().await?; -// assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; - -// // check that initial resync is working -// doc1.start_sync(vec![]).await?; -// assert_next_unordered_with_optionals( -// &mut events1, -// Duration::from_secs(10), -// vec![ -// match_event!(LiveEvent::NeighborUp(n) if *n== id2), -// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), -// match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), -// match_event!(LiveEvent::ContentReady { hash } if *hash == hash_b), -// ], -// vec![ -// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), -// match_event!(LiveEvent::PendingContentReady), -// ] -// ).await; -// assert_latest(&blobs1, &doc1, b"n2/b", b"b").await; - -// // check that live conn is working -// info!(me = id2.fmt_short(), "node2 set c"); -// let hash_c = doc2.set_bytes(author2, "n2/c", "c").await?; -// assert_next_unordered_with_optionals( -// &mut events1, -// Duration::from_secs(10), -// vec![ -// match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), -// match_event!(LiveEvent::ContentReady { hash } if *hash == hash_c), -// ], -// vec![ -// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), -// match_event!(LiveEvent::PendingContentReady), -// match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), -// match_event!(LiveEvent::PendingContentReady), -// ] -// ).await; - -// assert_latest(&blobs1, &doc1, b"n2/c", b"c").await; - -// Ok(()) -// } - -// /// Joins two nodes that write to the same document but have differing download policies and tests -// /// that they both synced the key info but not the content. -// #[tokio::test] -// async fn test_download_policies() -> Result<()> { -// // keys node a has -// let star_wars_movies = &[ -// "star_wars/prequel/the_phantom_menace", -// "star_wars/prequel/attack_of_the_clones", -// "star_wars/prequel/revenge_of_the_sith", -// "star_wars/og/a_new_hope", -// "star_wars/og/the_empire_strikes_back", -// "star_wars/og/return_of_the_jedi", -// ]; -// // keys node b has -// let lotr_movies = &[ -// "lotr/fellowship_of_the_ring", -// "lotr/the_two_towers", -// "lotr/return_of_the_king", -// ]; - -// // content policy for what b wants -// let policy_b = -// DownloadPolicy::EverythingExcept(vec![FilterKind::Prefix("star_wars/og".into())]); -// // content policy for what a wants -// let policy_a = DownloadPolicy::NothingExcept(vec![FilterKind::Exact( -// "lotr/fellowship_of_the_ring".into(), -// )]); - -// // a will sync all lotr keys but download a single key -// const EXPECTED_A_SYNCED: usize = 3; -// const EXPECTED_A_DOWNLOADED: usize = 1; - -// // b will sync all star wars content but download only the prequel keys -// const EXPECTED_B_SYNCED: usize = 6; -// const EXPECTED_B_DOWNLOADED: usize = 3; - -// let mut rng = test_rng(b"sync_download_policies"); -// setup_logging(); -// let nodes = spawn_nodes(2, &mut rng).await?; -// let clients = nodes.iter().map(|node| node.client()).collect::>(); - -// let doc_a = clients[0].docs().create().await?; -// let author_a = clients[0].authors().create().await?; -// let ticket = doc_a -// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) -// .await?; - -// let doc_b = clients[1].docs().import(ticket).await?; -// let author_b = clients[1].authors().create().await?; - -// doc_a.set_download_policy(policy_a).await?; -// doc_b.set_download_policy(policy_b).await?; - -// let mut events_a = doc_a.subscribe().await?; -// let mut events_b = doc_b.subscribe().await?; - -// let mut key_hashes: HashMap = HashMap::default(); - -// // set content in a -// for k in star_wars_movies.iter() { -// let hash = doc_a -// .set_bytes(author_a, k.to_owned(), k.to_owned()) -// .await?; -// key_hashes.insert(hash, k); -// } - -// // set content in b -// for k in lotr_movies.iter() { -// let hash = doc_b -// .set_bytes(author_b, k.to_owned(), k.to_owned()) -// .await?; -// key_hashes.insert(hash, k); -// } - -// assert_eq!(key_hashes.len(), star_wars_movies.len() + lotr_movies.len()); - -// let fut = async { -// use LiveEvent::*; -// let mut downloaded_a: Vec<&'static str> = Vec::new(); -// let mut downloaded_b: Vec<&'static str> = Vec::new(); -// let mut synced_a = 0usize; -// let mut synced_b = 0usize; -// loop { -// tokio::select! { -// Some(Ok(ev)) = events_a.next() => { -// match ev { -// InsertRemote { content_status, entry, .. } => { -// synced_a += 1; -// if let ContentStatus::Complete = content_status { -// downloaded_a.push(key_hashes.get(&entry.content_hash()).unwrap()) -// } -// }, -// ContentReady { hash } => { -// downloaded_a.push(key_hashes.get(&hash).unwrap()); -// }, -// _ => {} -// } -// } -// Some(Ok(ev)) = events_b.next() => { -// match ev { -// InsertRemote { content_status, entry, .. } => { -// synced_b += 1; -// if let ContentStatus::Complete = content_status { -// downloaded_b.push(key_hashes.get(&entry.content_hash()).unwrap()) -// } -// }, -// ContentReady { hash } => { -// downloaded_b.push(key_hashes.get(&hash).unwrap()); -// }, -// _ => {} -// } -// } -// } - -// if synced_a == EXPECTED_A_SYNCED -// && downloaded_a.len() == EXPECTED_A_DOWNLOADED -// && synced_b == EXPECTED_B_SYNCED -// && downloaded_b.len() == EXPECTED_B_DOWNLOADED -// { -// break; -// } -// } -// (downloaded_a, downloaded_b) -// }; - -// let (downloaded_a, mut downloaded_b) = tokio::time::timeout(TIMEOUT, fut) -// .await -// .context("timeout elapsed")?; - -// downloaded_b.sort(); -// assert_eq!(downloaded_a, vec!["lotr/fellowship_of_the_ring"]); -// assert_eq!( -// downloaded_b, -// vec![ -// "star_wars/prequel/attack_of_the_clones", -// "star_wars/prequel/revenge_of_the_sith", -// "star_wars/prequel/the_phantom_menace", -// ] -// ); - -// Ok(()) -// } - -// /// Test sync between many nodes with propagation through sync reports. -// #[tokio::test(flavor = "multi_thread")] -// #[ignore = "flaky"] -// async fn sync_big() -> Result<()> { -// setup_logging(); -// let mut rng = test_rng(b"sync_big"); -// let n_nodes = std::env::var("NODES") -// .map(|v| v.parse().expect("NODES must be a number")) -// .unwrap_or(10); -// let n_entries_init = 1; - -// tokio::task::spawn(async move { -// for i in 0.. { -// tokio::time::sleep(Duration::from_secs(1)).await; -// info!("tick {i}"); -// } -// }); - -// let nodes = spawn_nodes(n_nodes, &mut rng).await?; -// let node_ids = nodes.iter().map(|node| node.node_id()).collect::>(); -// let clients = nodes.iter().map(|node| node.client()).collect::>(); -// let authors = collect_futures(clients.iter().map(|c| c.authors().create())).await?; - -// let doc0 = clients[0].docs().create().await?; -// let mut ticket = doc0 -// .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) -// .await?; -// // do not join for now, just import without any peer info -// let peer0 = ticket.nodes[0].clone(); -// ticket.nodes = vec![]; - -// let docs_clients: Vec<_> = clients.iter().skip(1).map(|c| c.docs().clone()).collect(); -// let mut docs = vec![]; -// docs.push(doc0); -// docs.extend_from_slice( -// &collect_futures(docs_clients.into_iter().map(|c| { -// let ticket = ticket.clone(); -// async move { c.import(ticket).await } -// })) -// .await?, -// ); - -// let mut expected = vec![]; - -// // create initial data on each node -// publish(&docs, &mut expected, n_entries_init, |i, j| { -// ( -// authors[i], -// format!("init/{}/{j}", node_ids[i].fmt_short()), -// format!("init:{i}:{j}"), -// ) -// }) -// .await?; - -// // assert initial data -// for (i, doc) in docs.iter().enumerate() { -// let blobs = nodes[i].blobs(); -// let entries = get_all_with_content(blobs, doc).await?; -// let mut expected = expected -// .iter() -// .filter(|e| e.author == authors[i]) -// .cloned() -// .collect::>(); -// expected.sort(); -// assert_eq!(entries, expected, "phase1 pre-sync correct"); -// } - -// // setup event streams -// let events = collect_futures(docs.iter().map(|d| d.subscribe())).await?; - -// // join nodes together -// for (i, doc) in docs.iter().enumerate().skip(1) { -// info!(me = %node_ids[i].fmt_short(), peer = %peer0.node_id.fmt_short(), "join"); -// doc.start_sync(vec![peer0.clone()]).await?; -// } - -// // wait for InsertRemote events stuff to happen -// info!("wait for all peers to receive insert events"); -// let expected_inserts = (n_nodes - 1) * n_entries_init; -// let mut tasks = tokio::task::JoinSet::default(); -// for (i, events) in events.into_iter().enumerate() { -// let doc = docs[i].clone(); -// let me = doc.id().fmt_short(); -// let expected = expected.clone(); -// let fut = async move { -// wait_for_events(events, expected_inserts, TIMEOUT, |e| { -// matches!(e, LiveEvent::InsertRemote { .. }) -// }) -// .await?; -// let entries = get_all(&doc).await?; -// if entries != expected { -// Err(anyhow!( -// "node {i} failed (has {} entries but expected to have {})", -// entries.len(), -// expected.len() -// )) -// } else { -// info!( -// "received and checked all {} expected entries", -// expected.len() -// ); -// Ok(()) -// } -// } -// .instrument(error_span!("sync-test", %me)); -// let fut = fut.map(move |r| r.with_context(move || format!("node {i} ({me})"))); -// tasks.spawn(fut); -// } - -// while let Some(res) = tasks.join_next().await { -// res??; -// } - -// assert_all_docs(&docs, &node_ids, &expected, "after initial sync").await; - -// info!("shutdown"); -// for node in nodes { -// node.shutdown().await?; -// } - -// Ok(()) -// } - -// #[tokio::test] -// #[cfg(feature = "test-utils")] -// async fn test_list_docs_stream() -> Result<()> { -// let node = Node::memory() -// .node_discovery(iroh::node::DiscoveryConfig::None) -// .relay_mode(iroh::net::relay::RelayMode::Disabled) -// .enable_docs() -// .spawn() -// .await?; -// let count = 200; - -// // create docs -// for _i in 0..count { -// let doc = node.docs().create().await?; -// doc.close().await?; -// } - -// // create doc stream -// let mut stream = node.docs().list().await?; - -// // process each doc and call into the docs actor. -// // this makes sure that we don't deadlock the docs actor. -// let mut i = 0; -// let fut = async { -// while let Some((id, _)) = stream.try_next().await.unwrap() { -// let _doc = node.docs().open(id).await.unwrap().unwrap(); -// i += 1; -// } -// }; - -// tokio::time::timeout(Duration::from_secs(2), fut) -// .await -// .expect("not to timeout"); - -// assert_eq!(i, count); - -// Ok(()) -// } - -// /// Get all entries of a document. -// async fn get_all(doc: &Doc) -> anyhow::Result> { -// let entries = doc.get_many(Query::all()).await?; -// let entries = entries.collect::>().await; -// entries.into_iter().collect() -// } - -// /// Get all entries of a document with the blob content. -// async fn get_all_with_content( -// blobs: &client::blobs::Client, -// doc: &Doc, -// ) -> anyhow::Result> { -// let entries = doc.get_many(Query::all()).await?; -// let entries = entries.and_then(|entry| async { -// let hash = entry.content_hash(); -// let content = blobs.read_to_bytes(hash).await; -// content.map(|c| (entry, c)) -// }); -// let entries = entries.collect::>().await; -// let entries = entries.into_iter().collect::>>()?; -// Ok(entries) -// } - -// async fn publish( -// docs: &[Doc], -// expected: &mut Vec, -// n: usize, -// cb: impl Fn(usize, usize) -> (AuthorId, String, String), -// ) -> anyhow::Result<()> { -// for (i, doc) in docs.iter().enumerate() { -// for j in 0..n { -// let (author, key, value) = cb(i, j); -// doc.set_bytes(author, key.as_bytes().to_vec(), value.as_bytes().to_vec()) -// .await?; -// expected.push(ExpectedEntry { author, key, value }); -// } -// } -// expected.sort(); -// Ok(()) -// } - -// /// Collect an iterator into futures by joining them all and failing if any future failed. -// async fn collect_futures( -// futs: impl IntoIterator>>, -// ) -> anyhow::Result> { -// futures_buffered::join_all(futs) -// .await -// .into_iter() -// .collect::>>() -// } - -// /// Collect `count` events from the `events` stream, only collecting events for which `matcher` -// /// returns true. -// async fn wait_for_events( -// mut events: impl Stream> + Send + Unpin + 'static, -// count: usize, -// timeout: Duration, -// matcher: impl Fn(&LiveEvent) -> bool, -// ) -> anyhow::Result> { -// let mut res = Vec::with_capacity(count); -// let sleep = tokio::time::sleep(timeout); -// tokio::pin!(sleep); -// while res.len() < count { -// tokio::select! { -// () = &mut sleep => { -// bail!("Failed to collect {count} elements in {timeout:?} (collected only {})", res.len()); -// }, -// event = events.try_next() => { -// let event = event?; -// match event { -// None => bail!("stream ended after {} items, but expected {count}", res.len()), -// Some(event) => if matcher(&event) { -// res.push(event); -// debug!("recv event {} of {count}", res.len()); -// } -// } -// } -// } -// } -// Ok(res) -// } - -// async fn assert_all_docs( -// docs: &[Doc], -// node_ids: &[PublicKey], -// expected: &Vec, -// label: &str, -// ) { -// info!("validate all peers: {label}"); -// for (i, doc) in docs.iter().enumerate() { -// let entries = get_all(doc).await.unwrap_or_else(|err| { -// panic!("failed to get entries for peer {:?}: {err:?}", node_ids[i]) -// }); -// assert_eq!( -// &entries, -// expected, -// "{label}: peer {i} {:?} failed (have {} but expected {})", -// node_ids[i], -// entries.len(), -// expected.len() -// ); -// } -// } - -// #[derive(Debug, Ord, Eq, PartialEq, PartialOrd, Clone)] -// struct ExpectedEntry { -// author: AuthorId, -// key: String, -// value: String, -// } - -// impl PartialEq for ExpectedEntry { -// fn eq(&self, other: &Entry) -> bool { -// self.key.as_bytes() == other.key() -// && Hash::new(&self.value) == other.content_hash() -// && self.author == other.author() -// } -// } -// impl PartialEq<(Entry, Bytes)> for ExpectedEntry { -// fn eq(&self, (entry, content): &(Entry, Bytes)) -> bool { -// self.key.as_bytes() == entry.key() -// && Hash::new(&self.value) == entry.content_hash() -// && self.author == entry.author() -// && self.value.as_bytes() == content.as_ref() -// } -// } -// impl PartialEq for Entry { -// fn eq(&self, other: &ExpectedEntry) -> bool { -// other.eq(self) -// } -// } -// impl PartialEq for (Entry, Bytes) { -// fn eq(&self, other: &ExpectedEntry) -> bool { -// other.eq(self) -// } -// } - -// #[tokio::test] -// async fn doc_delete() -> Result<()> { -// let node = Node::memory() -// .gc_policy(iroh::node::GcPolicy::Interval(Duration::from_millis(100))) -// .enable_docs() -// .spawn() -// .await?; -// let client = node.client(); -// let doc = client.docs().create().await?; -// let blobs = client.blobs(); -// let author = client.authors().create().await?; -// let hash = doc -// .set_bytes(author, b"foo".to_vec(), b"hi".to_vec()) -// .await?; -// assert_latest(&blobs, &doc, b"foo", b"hi").await; -// let deleted = doc.del(author, b"foo".to_vec()).await?; -// assert_eq!(deleted, 1); - -// let entry = doc.get_exact(author, b"foo".to_vec(), false).await?; -// assert!(entry.is_none()); - -// // wait for gc -// // TODO: allow to manually trigger gc -// tokio::time::sleep(Duration::from_millis(200)).await; -// let bytes = client.blobs().read_to_bytes(hash).await; -// assert!(bytes.is_err()); -// node.shutdown().await?; -// Ok(()) -// } - -// #[tokio::test] -// async fn sync_drop_doc() -> Result<()> { -// let mut rng = test_rng(b"sync_drop_doc"); -// setup_logging(); -// let node = spawn_node(0, &mut rng).await?; -// let client = node.client(); - -// let doc = client.docs().create().await?; -// let author = client.authors().create().await?; - -// let mut sub = doc.subscribe().await?; -// doc.set_bytes(author, b"foo".to_vec(), b"bar".to_vec()) -// .await?; -// let ev = sub.next().await; -// assert!(matches!(ev, Some(Ok(LiveEvent::InsertLocal { .. })))); - -// client.docs().drop_doc(doc.id()).await?; -// let res = doc.get_exact(author, b"foo".to_vec(), true).await; -// assert!(res.is_err()); -// let res = doc -// .set_bytes(author, b"foo".to_vec(), b"bar".to_vec()) -// .await; -// assert!(res.is_err()); -// let res = client.docs().open(doc.id()).await; -// assert!(res.is_err()); -// let ev = sub.next().await; -// assert!(ev.is_none()); - -// Ok(()) -// } - -// async fn assert_latest( -// blobs: &client::blobs::Client, -// doc: &Doc, -// key: &[u8], -// value: &[u8], -// ) { -// let content = get_latest(blobs, doc, key).await.unwrap(); -// assert_eq!(content, value.to_vec()); -// } - -// async fn get_latest( -// blobs: &client::blobs::Client, -// doc: &Doc, -// key: &[u8], -// ) -> anyhow::Result> { -// let query = Query::single_latest_per_key().key_exact(key); -// let entry = doc -// .get_many(query) -// .await? -// .next() -// .await -// .ok_or_else(|| anyhow!("entry not found"))??; -// let content = blobs.read_to_bytes(entry.content_hash()).await?; -// Ok(content.to_vec()) -// } - -// fn setup_logging() { -// tracing_subscriber::registry() -// .with(tracing_subscriber::fmt::layer().with_writer(std::io::stderr)) -// .with(EnvFilter::from_default_env()) -// .try_init() -// .ok(); -// } - -// async fn next(mut stream: impl Stream> + Unpin) -> T { -// let event = stream -// .next() -// .await -// .expect("stream ended") -// .expect("stream produced error"); -// debug!("Event: {event:?}"); -// event -// } - -// #[allow(clippy::type_complexity)] -// fn apply_matchers(item: &T, matchers: &mut Vec bool + Send>>) -> bool { -// for i in 0..matchers.len() { -// if matchers[i](item) { -// let _ = matchers.remove(i); -// return true; -// } -// } -// false -// } - -// /// Receive the next `matchers.len()` elements from a stream and matches them against the functions -// /// in `matchers`, in order. -// /// -// /// Returns all received events. -// #[allow(clippy::type_complexity)] -// async fn assert_next( -// mut stream: impl Stream> + Unpin + Send, -// timeout: Duration, -// matchers: Vec bool + Send>>, -// ) -> Vec { -// let fut = async { -// let mut items = vec![]; -// for (i, f) in matchers.iter().enumerate() { -// let item = stream -// .next() -// .await -// .expect("event stream ended prematurely") -// .expect("event stream errored"); -// if !(f)(&item) { -// panic!("assertion failed for event {i} {item:?}"); -// } -// items.push(item); -// } -// items -// }; -// let res = tokio::time::timeout(timeout, fut).await; -// res.expect("timeout reached") -// } - -// /// Receive `matchers.len()` elements from a stream and assert that each element matches one of the -// /// functions in `matchers`. -// /// -// /// Order of the matchers is not relevant. -// /// -// /// Returns all received events. -// #[allow(clippy::type_complexity)] -// async fn assert_next_unordered( -// stream: impl Stream> + Unpin + Send, -// timeout: Duration, -// matchers: Vec bool + Send>>, -// ) -> Vec { -// assert_next_unordered_with_optionals(stream, timeout, matchers, vec![]).await -// } - -// /// Receive between `min` and `max` elements from the stream and assert that each element matches -// /// either one of the matchers in `required_matchers` or in `optional_matchers`. -// /// -// /// Order of the matchers is not relevant. -// /// -// /// Will return an error if: -// /// * Any element fails to match one of the required or optional matchers -// /// * More than `max` elements were received, but not all required matchers were used yet -// /// * The timeout completes before all required matchers were used -// /// -// /// Returns all received events. -// #[allow(clippy::type_complexity)] -// async fn assert_next_unordered_with_optionals( -// mut stream: impl Stream> + Unpin + Send, -// timeout: Duration, -// mut required_matchers: Vec bool + Send>>, -// mut optional_matchers: Vec bool + Send>>, -// ) -> Vec { -// let max = required_matchers.len() + optional_matchers.len(); -// let required = required_matchers.len(); -// // we have to use a mutex because rustc is not intelligent enough to realize -// // that the mutable borrow terminates when the future completes -// let events = Arc::new(parking_lot::Mutex::new(vec![])); -// let fut = async { -// while let Some(event) = stream.next().await { -// let event = event.context("failed to read from stream")?; -// let len = { -// let mut events = events.lock(); -// events.push(event.clone()); -// events.len() -// }; -// if !apply_matchers(&event, &mut required_matchers) -// && !apply_matchers(&event, &mut optional_matchers) -// { -// bail!("Event didn't match any matcher: {event:?}"); -// } -// if required_matchers.is_empty() || len == max { -// break; -// } -// } -// if !required_matchers.is_empty() { -// bail!( -// "Matched only {} of {required} required matchers", -// required - required_matchers.len() -// ); -// } -// Ok(()) -// }; -// tokio::pin!(fut); -// let res = tokio::time::timeout(timeout, fut) -// .await -// .map_err(|_| anyhow!("Timeout reached ({timeout:?})")) -// .and_then(|res| res); -// let events = events.lock().clone(); -// if let Err(err) = &res { -// println!("Received events: {events:#?}"); -// println!( -// "Received {} events, expected between {required} and {max}", -// events.len() -// ); -// panic!("Failed to receive or match all events: {err:?}"); -// } -// events -// } - -// /// Asserts that the event is a [`LiveEvent::SyncFinished`] and that the contained [`SyncEvent`] -// /// has no error and matches `peer` and `namespace`. -// fn match_sync_finished(event: &LiveEvent, peer: PublicKey) -> bool { -// let LiveEvent::SyncFinished(e) = event else { -// return false; -// }; -// e.peer == peer && e.result.is_ok() -// } +use std::{ + collections::HashMap, + future::Future, + sync::Arc, + time::{Duration, Instant}, +}; + +use anyhow::{anyhow, bail, Context, Result}; +use bytes::Bytes; +use futures_lite::Stream; +use futures_util::{FutureExt, StreamExt, TryStreamExt}; +use iroh_base::node_addr::AddrInfoOptions; +use iroh_blobs::Hash; +use iroh_docs::{ + rpc::client::{Doc, Entry, LiveEvent, ShareMode}, + store::{DownloadPolicy, FilterKind, Query}, + AuthorId, ContentStatus, +}; +use iroh_net::{ + key::{PublicKey, SecretKey}, + relay::RelayMode, +}; +use rand::{CryptoRng, Rng, SeedableRng}; +use tracing::{debug, error_span, info, Instrument}; +use tracing_subscriber::{prelude::*, EnvFilter}; +mod util; +use util::{Builder, Node}; + +const TIMEOUT: Duration = Duration::from_secs(60); + +fn test_node(secret_key: SecretKey) -> Builder { + Node::memory() + .secret_key(secret_key) + .relay_mode(RelayMode::Disabled) +} + +// The function is not `async fn` so that we can take a `&mut` borrow on the `rng` without +// capturing that `&mut` lifetime in the returned future. This allows to call it in a loop while +// still collecting the futures before awaiting them altogether (see [`spawn_nodes`]) +fn spawn_node( + i: usize, + rng: &mut (impl CryptoRng + Rng), +) -> impl Future>> + 'static { + let secret_key = SecretKey::generate_with_rng(rng); + async move { + let node = test_node(secret_key); + let node = node.spawn().await?; + info!(?i, me = %node.node_id().fmt_short(), "node spawned"); + Ok(node) + } +} + +async fn spawn_nodes( + n: usize, + mut rng: &mut (impl CryptoRng + Rng), +) -> anyhow::Result>> { + let mut futs = vec![]; + for i in 0..n { + futs.push(spawn_node(i, &mut rng)); + } + futures_buffered::join_all(futs).await.into_iter().collect() +} + +pub fn test_rng(seed: &[u8]) -> rand_chacha::ChaCha12Rng { + rand_chacha::ChaCha12Rng::from_seed(*Hash::new(seed).as_bytes()) +} + +macro_rules! match_event { + ($pattern:pat $(if $guard:expr)? $(,)?) => { + Box::new(move |e| matches!(e, $pattern $(if $guard)?)) + }; +} + +/// This tests the simplest scenario: A node connects to another node, and performs sync. +#[tokio::test] +async fn sync_simple() -> Result<()> { + setup_logging(); + let mut rng = test_rng(b"sync_simple"); + let nodes = spawn_nodes(2, &mut rng).await?; + let clients = nodes.iter().map(|node| node.client()).collect::>(); + + // create doc on node0 + let peer0 = nodes[0].node_id(); + let author0 = clients[0].docs().author_create().await?; + let doc0 = clients[0].docs().create().await?; + let blobs0 = clients[0].blobs(); + let hash0 = doc0 + .set_bytes(author0, b"k1".to_vec(), b"v1".to_vec()) + .await?; + assert_latest(&blobs0, &doc0, b"k1", b"v1").await; + let ticket = doc0 + .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) + .await?; + + let mut events0 = doc0.subscribe().await?; + + info!("node1: join"); + let peer1 = nodes[1].node_id(); + let doc1 = clients[1].docs().import(ticket.clone()).await?; + let blobs1 = clients[1].blobs(); + let mut events1 = doc1.subscribe().await?; + info!("node1: assert 5 events"); + assert_next_unordered( + &mut events1, + TIMEOUT, + vec![ + Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer0)), + Box::new(move |e| matches!(e, LiveEvent::InsertRemote { from, .. } if *from == peer0 )), + Box::new(move |e| match_sync_finished(e, peer0)), + Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash0)), + match_event!(LiveEvent::PendingContentReady), + ], + ) + .await; + assert_latest(&blobs1, &doc1, b"k1", b"v1").await; + + info!("node0: assert 2 events"); + assert_next( + &mut events0, + TIMEOUT, + vec![ + Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer1)), + Box::new(move |e| match_sync_finished(e, peer1)), + ], + ) + .await; + + for node in nodes { + node.shutdown().await?; + } + Ok(()) +} + +/// Test subscribing to replica events (without sync) +#[tokio::test] +async fn sync_subscribe_no_sync() -> Result<()> { + let mut rng = test_rng(b"sync_subscribe"); + setup_logging(); + let node = spawn_node(0, &mut rng).await?; + let client = node.client(); + let doc = client.docs().create().await?; + let mut sub = doc.subscribe().await?; + let author = client.docs().author_create().await?; + doc.set_bytes(author, b"k".to_vec(), b"v".to_vec()).await?; + let event = tokio::time::timeout(Duration::from_millis(100), sub.next()).await?; + assert!( + matches!(event, Some(Ok(LiveEvent::InsertLocal { .. }))), + "expected InsertLocal but got {event:?}" + ); + node.shutdown().await?; + Ok(()) +} + +#[tokio::test] +async fn sync_gossip_bulk() -> Result<()> { + let n_entries: usize = std::env::var("N_ENTRIES") + .map(|x| x.parse().expect("N_ENTRIES must be a number")) + .unwrap_or(100); + let mut rng = test_rng(b"sync_gossip_bulk"); + setup_logging(); + + let nodes = spawn_nodes(2, &mut rng).await?; + let clients = nodes.iter().map(|node| node.client()).collect::>(); + + let _peer0 = nodes[0].node_id(); + let author0 = clients[0].docs().author_create().await?; + let doc0 = clients[0].docs().create().await?; + let mut ticket = doc0 + .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) + .await?; + // unset peers to not yet start sync + let peers = ticket.nodes.clone(); + ticket.nodes = vec![]; + let doc1 = clients[1].docs().import(ticket).await?; + let mut events = doc1.subscribe().await?; + + // create entries for initial sync. + let now = Instant::now(); + let value = b"foo"; + for i in 0..n_entries { + let key = format!("init/{i}"); + doc0.set_bytes(author0, key.as_bytes().to_vec(), value.to_vec()) + .await?; + } + let elapsed = now.elapsed(); + info!( + "insert took {elapsed:?} for {n_entries} ({:?} per entry)", + elapsed / n_entries as u32 + ); + + let now = Instant::now(); + let mut count = 0; + doc0.start_sync(vec![]).await?; + doc1.start_sync(peers).await?; + while let Some(event) = events.next().await { + let event = event?; + if matches!(event, LiveEvent::InsertRemote { .. }) { + count += 1; + } + if count == n_entries { + break; + } + } + let elapsed = now.elapsed(); + info!( + "initial sync took {elapsed:?} for {n_entries} ({:?} per entry)", + elapsed / n_entries as u32 + ); + + // publish another 1000 entries + let mut count = 0; + let value = b"foo"; + let now = Instant::now(); + for i in 0..n_entries { + let key = format!("gossip/{i}"); + doc0.set_bytes(author0, key.as_bytes().to_vec(), value.to_vec()) + .await?; + } + let elapsed = now.elapsed(); + info!( + "insert took {elapsed:?} for {n_entries} ({:?} per entry)", + elapsed / n_entries as u32 + ); + + while let Some(event) = events.next().await { + let event = event?; + if matches!(event, LiveEvent::InsertRemote { .. }) { + count += 1; + } + if count == n_entries { + break; + } + } + let elapsed = now.elapsed(); + info!( + "gossip recv took {elapsed:?} for {n_entries} ({:?} per entry)", + elapsed / n_entries as u32 + ); + + Ok(()) +} + +/// This tests basic sync and gossip with 3 peers. +#[tokio::test] +#[ignore = "flaky"] +async fn sync_full_basic() -> Result<()> { + let mut rng = test_rng(b"sync_full_basic"); + setup_logging(); + let mut nodes = spawn_nodes(2, &mut rng).await?; + let mut clients = nodes + .iter() + .map(|node| node.client().clone()) + .collect::>(); + + // peer0: create doc and ticket + let peer0 = nodes[0].node_id(); + let author0 = clients[0].docs().author_create().await?; + let doc0 = clients[0].docs().create().await?; + let blobs0 = clients[0].blobs(); + let mut events0 = doc0.subscribe().await?; + let key0 = b"k1"; + let value0 = b"v1"; + let hash0 = doc0 + .set_bytes(author0, key0.to_vec(), value0.to_vec()) + .await?; + + info!("peer0: wait for 1 event (local insert)"); + let e = next(&mut events0).await; + assert!( + matches!(&e, LiveEvent::InsertLocal { entry } if entry.content_hash() == hash0), + "expected LiveEvent::InsertLocal but got {e:?}", + ); + assert_latest(&blobs0, &doc0, key0, value0).await; + let ticket = doc0 + .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) + .await?; + + info!("peer1: spawn"); + let peer1 = nodes[1].node_id(); + let author1 = clients[1].docs().author_create().await?; + info!("peer1: join doc"); + let doc1 = clients[1].docs().import(ticket.clone()).await?; + let blobs1 = clients[1].blobs(); + + info!("peer1: wait for 4 events (for sync and join with peer0)"); + let mut events1 = doc1.subscribe().await?; + assert_next_unordered( + &mut events1, + TIMEOUT, + vec![ + match_event!(LiveEvent::NeighborUp(peer) if *peer == peer0), + match_event!(LiveEvent::InsertRemote { from, .. } if *from == peer0 ), + Box::new(move |e| match_sync_finished(e, peer0)), + match_event!(LiveEvent::ContentReady { hash } if *hash == hash0), + match_event!(LiveEvent::PendingContentReady), + ], + ) + .await; + + info!("peer0: wait for 2 events (join & accept sync finished from peer1)"); + assert_next( + &mut events0, + TIMEOUT, + vec![ + match_event!(LiveEvent::NeighborUp(peer) if *peer == peer1), + Box::new(move |e| match_sync_finished(e, peer1)), + match_event!(LiveEvent::PendingContentReady), + ], + ) + .await; + + info!("peer1: insert entry"); + let key1 = b"k2"; + let value1 = b"v2"; + let hash1 = doc1 + .set_bytes(author1, key1.to_vec(), value1.to_vec()) + .await?; + assert_latest(&blobs1, &doc1, key1, value1).await; + info!("peer1: wait for 1 event (local insert, and pendingcontentready)"); + assert_next( + &mut events1, + TIMEOUT, + vec![match_event!(LiveEvent::InsertLocal { entry} if entry.content_hash() == hash1)], + ) + .await; + + // peer0: assert events for entry received via gossip + info!("peer0: wait for 2 events (gossip'ed entry from peer1)"); + assert_next( + &mut events0, + TIMEOUT, + vec![ + Box::new( + move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == peer1), + ), + Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash1)), + ], + ).await; + assert_latest(&blobs0, &doc0, key1, value1).await; + + // Note: If we could check gossip messages directly here (we can't easily), we would notice + // that peer1 will receive a `Op::ContentReady` gossip message, broadcast + // by peer0 with neighbor scope. This message is superfluous, and peer0 could know that, however + // our gossip implementation does not allow us to filter message receivers this way. + + info!("peer2: spawn"); + nodes.push(spawn_node(nodes.len(), &mut rng).await?); + clients.push(nodes.last().unwrap().client().clone()); + let doc2 = clients[2].docs().import(ticket).await?; + let blobs2 = clients[2].blobs(); + let peer2 = nodes[2].node_id(); + let mut events2 = doc2.subscribe().await?; + + info!("peer2: wait for 9 events (from sync with peers)"); + assert_next_unordered_with_optionals( + &mut events2, + TIMEOUT, + // required events + vec![ + // 2 NeighborUp events + Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer0)), + Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer1)), + // 2 SyncFinished events + Box::new(move |e| match_sync_finished(e, peer0)), + Box::new(move |e| match_sync_finished(e, peer1)), + // 2 InsertRemote events + Box::new( + move |e| matches!(e, LiveEvent::InsertRemote { entry, content_status: ContentStatus::Missing, .. } if entry.content_hash() == hash0), + ), + Box::new( + move |e| matches!(e, LiveEvent::InsertRemote { entry, content_status: ContentStatus::Missing, .. } if entry.content_hash() == hash1), + ), + // 2 ContentReady events + Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash0)), + Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash1)), + // at least 1 PendingContentReady + match_event!(LiveEvent::PendingContentReady), + ], + // optional events + // it may happen that we run sync two times against our two peers: + // if the first sync (as a result of us joining the peer manually through the ticket) completes + // before the peer shows up as a neighbor, we run sync again for the NeighborUp event. + vec![ + // 2 SyncFinished events + Box::new(move |e| match_sync_finished(e, peer0)), + Box::new(move |e| match_sync_finished(e, peer1)), + match_event!(LiveEvent::PendingContentReady), + match_event!(LiveEvent::PendingContentReady), + ] + ).await; + assert_latest(&blobs2, &doc2, b"k1", b"v1").await; + assert_latest(&blobs2, &doc2, b"k2", b"v2").await; + + info!("peer0: wait for 2 events (join & accept sync finished from peer2)"); + assert_next( + &mut events0, + TIMEOUT, + vec![ + Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer2)), + Box::new(move |e| match_sync_finished(e, peer2)), + match_event!(LiveEvent::PendingContentReady), + ], + ) + .await; + + info!("peer1: wait for 2 events (join & accept sync finished from peer2)"); + assert_next( + &mut events1, + TIMEOUT, + vec![ + Box::new(move |e| matches!(e, LiveEvent::NeighborUp(peer) if *peer == peer2)), + Box::new(move |e| match_sync_finished(e, peer2)), + match_event!(LiveEvent::PendingContentReady), + ], + ) + .await; + + info!("shutdown"); + for node in nodes { + node.shutdown().await?; + } + + Ok(()) +} + +#[tokio::test] +async fn sync_open_close() -> Result<()> { + let mut rng = test_rng(b"sync_subscribe_stop_close"); + setup_logging(); + let node = spawn_node(0, &mut rng).await?; + let client = node.client(); + + let doc = client.docs().create().await?; + let status = doc.status().await?; + assert_eq!(status.handles, 1); + + let doc2 = client.docs().open(doc.id()).await?.unwrap(); + let status = doc2.status().await?; + assert_eq!(status.handles, 2); + + doc.close().await?; + assert!(doc.status().await.is_err()); + + let status = doc2.status().await?; + assert_eq!(status.handles, 1); + + Ok(()) +} + +#[tokio::test] +async fn sync_subscribe_stop_close() -> Result<()> { + let mut rng = test_rng(b"sync_subscribe_stop_close"); + setup_logging(); + let node = spawn_node(0, &mut rng).await?; + let client = node.client(); + + let doc = client.docs().create().await?; + let author = client.docs().author_create().await?; + + let status = doc.status().await?; + assert_eq!(status.subscribers, 0); + assert_eq!(status.handles, 1); + assert!(!status.sync); + + doc.start_sync(vec![]).await?; + let status = doc.status().await?; + assert!(status.sync); + assert_eq!(status.handles, 2); + assert_eq!(status.subscribers, 1); + + let sub = doc.subscribe().await?; + let status = doc.status().await?; + assert_eq!(status.subscribers, 2); + drop(sub); + // trigger an event that makes the actor check if the event channels are still connected + doc.set_bytes(author, b"x".to_vec(), b"x".to_vec()).await?; + let status = doc.status().await?; + assert_eq!(status.subscribers, 1); + + doc.leave().await?; + let status = doc.status().await?; + assert_eq!(status.subscribers, 0); + assert_eq!(status.handles, 1); + assert!(!status.sync); + + Ok(()) +} + +#[tokio::test] +#[cfg(feature = "test-utils")] +async fn test_sync_via_relay() -> Result<()> { + let _guard = iroh_test::logging::setup(); + let (relay_map, _relay_url, _guard) = iroh_net::test_utils::run_relay_server().await?; + + let node1 = Node::memory() + .relay_mode(RelayMode::Custom(relay_map.clone())) + .insecure_skip_relay_cert_verify(true) + .spawn() + .await?; + let node1_id = node1.node_id(); + let node2 = Node::memory() + .bind_random_port() + .relay_mode(RelayMode::Custom(relay_map.clone())) + .insecure_skip_relay_cert_verify(true) + .spawn() + .await?; + + let doc1 = node1.docs().create().await?; + let author1 = node1.authors().create().await?; + let inserted_hash = doc1 + .set_bytes(author1, b"foo".to_vec(), b"bar".to_vec()) + .await?; + let mut ticket = doc1 + .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) + .await?; + + // remove direct addrs to force connect via relay + ticket.nodes[0].info.direct_addresses = Default::default(); + + // join + let doc2 = node2.docs().import(ticket).await?; + let blobs2 = node2.blobs(); + let mut events = doc2.subscribe().await?; + + assert_next_unordered_with_optionals( + &mut events, + Duration::from_secs(2), + vec![ + Box::new(move |e| matches!(e, LiveEvent::NeighborUp(n) if *n== node1_id)), + Box::new(move |e| match_sync_finished(e, node1_id)), + Box::new( + move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == node1_id), + ), + Box::new( + move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == inserted_hash), + ), + match_event!(LiveEvent::PendingContentReady), + ], + vec![Box::new(move |e| match_sync_finished(e, node1_id))], + ).await; + let actual = blobs2 + .read_to_bytes( + doc2.get_exact(author1, b"foo", false) + .await? + .expect("entry to exist") + .content_hash(), + ) + .await?; + assert_eq!(actual.as_ref(), b"bar"); + + // update + let updated_hash = doc1 + .set_bytes(author1, b"foo".to_vec(), b"update".to_vec()) + .await?; + assert_next_unordered_with_optionals( + &mut events, + Duration::from_secs(2), + vec![ + Box::new( + move |e| matches!(e, LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == node1_id), + ), + Box::new( + move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == updated_hash), + ), + ], + vec![ + Box::new(move |e| match_sync_finished(e, node1_id)), + Box::new(move |e| matches!(e, LiveEvent::PendingContentReady)), + ], + ).await; + let actual = blobs2 + .read_to_bytes( + doc2.get_exact(author1, b"foo", false) + .await? + .expect("entry to exist") + .content_hash(), + ) + .await?; + assert_eq!(actual.as_ref(), b"update"); + Ok(()) +} + +#[tokio::test] +#[cfg(feature = "test-utils")] +#[ignore = "flaky"] +async fn sync_restart_node() -> Result<()> { + let mut rng = test_rng(b"sync_restart_node"); + setup_logging(); + let (relay_map, _relay_url, _guard) = iroh_net::test_utils::run_relay_server().await?; + + let discovery_server = iroh_net::test_utils::DnsPkarrServer::run().await?; + + let node1_dir = tempfile::TempDir::with_prefix("test-sync_restart_node-node1")?; + let secret_key_1 = SecretKey::generate_with_rng(&mut rng); + + let node1 = Node::persistent(&node1_dir) + .await? + .secret_key(secret_key_1.clone()) + .insecure_skip_relay_cert_verify(true) + .relay_mode(RelayMode::Custom(relay_map.clone())) + .dns_resolver(discovery_server.dns_resolver()) + .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) + .spawn() + .await?; + let id1 = node1.node_id(); + + // create doc & ticket on node1 + let doc1 = node1.docs().create().await?; + let blobs1 = node1.blobs().clone(); + let mut events1 = doc1.subscribe().await?; + let ticket = doc1 + .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) + .await?; + + // create node2 + let secret_key_2 = SecretKey::generate_with_rng(&mut rng); + let node2 = Node::memory() + .secret_key(secret_key_2.clone()) + .relay_mode(RelayMode::Custom(relay_map.clone())) + .insecure_skip_relay_cert_verify(true) + .dns_resolver(discovery_server.dns_resolver()) + .node_discovery(discovery_server.discovery(secret_key_2.clone()).into()) + .spawn() + .await?; + let id2 = node2.node_id(); + let author2 = node2.authors().create().await?; + let doc2 = node2.docs().import(ticket.clone()).await?; + let blobs2 = node2.blobs(); + + info!("node2 set a"); + let hash_a = doc2.set_bytes(author2, "n2/a", "a").await?; + assert_latest(&blobs2, &doc2, b"n2/a", b"a").await; + + assert_next_unordered_with_optionals( + &mut events1, + Duration::from_secs(10), + vec![ + match_event!(LiveEvent::NeighborUp(n) if *n == id2), + match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), + match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), + match_event!(LiveEvent::ContentReady { hash } if *hash == hash_a), + match_event!(LiveEvent::PendingContentReady), + ], + vec![ + match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), + match_event!(LiveEvent::PendingContentReady), + ], + ) + .await; + assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; + + info!(me = id1.fmt_short(), "node1 start shutdown"); + node1.shutdown().await?; + info!(me = id1.fmt_short(), "node1 down"); + + info!(me = id1.fmt_short(), "sleep 1s"); + tokio::time::sleep(Duration::from_secs(1)).await; + + info!(me = id2.fmt_short(), "node2 set b"); + let hash_b = doc2.set_bytes(author2, "n2/b", "b").await?; + + info!(me = id1.fmt_short(), "node1 respawn"); + let node1 = Node::persistent(&node1_dir) + .await? + .secret_key(secret_key_1.clone()) + .insecure_skip_relay_cert_verify(true) + .relay_mode(RelayMode::Custom(relay_map.clone())) + .dns_resolver(discovery_server.dns_resolver()) + .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) + .spawn() + .await?; + assert_eq!(id1, node1.node_id()); + + let doc1 = node1.docs().open(doc1.id()).await?.expect("doc to exist"); + let blobs1 = node1.blobs(); + let mut events1 = doc1.subscribe().await?; + assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; + + // check that initial resync is working + doc1.start_sync(vec![]).await?; + assert_next_unordered_with_optionals( + &mut events1, + Duration::from_secs(10), + vec![ + match_event!(LiveEvent::NeighborUp(n) if *n== id2), + match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), + match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), + match_event!(LiveEvent::ContentReady { hash } if *hash == hash_b), + ], + vec![ + match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), + match_event!(LiveEvent::PendingContentReady), + ] + ).await; + assert_latest(&blobs1, &doc1, b"n2/b", b"b").await; + + // check that live conn is working + info!(me = id2.fmt_short(), "node2 set c"); + let hash_c = doc2.set_bytes(author2, "n2/c", "c").await?; + assert_next_unordered_with_optionals( + &mut events1, + Duration::from_secs(10), + vec![ + match_event!(LiveEvent::InsertRemote { from, content_status: ContentStatus::Missing, .. } if *from == id2), + match_event!(LiveEvent::ContentReady { hash } if *hash == hash_c), + ], + vec![ + match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), + match_event!(LiveEvent::PendingContentReady), + match_event!(LiveEvent::SyncFinished(e) if e.peer == id2 && e.result.is_ok()), + match_event!(LiveEvent::PendingContentReady), + ] + ).await; + + assert_latest(&blobs1, &doc1, b"n2/c", b"c").await; + + Ok(()) +} + +/// Joins two nodes that write to the same document but have differing download policies and tests +/// that they both synced the key info but not the content. +#[tokio::test] +async fn test_download_policies() -> Result<()> { + // keys node a has + let star_wars_movies = &[ + "star_wars/prequel/the_phantom_menace", + "star_wars/prequel/attack_of_the_clones", + "star_wars/prequel/revenge_of_the_sith", + "star_wars/og/a_new_hope", + "star_wars/og/the_empire_strikes_back", + "star_wars/og/return_of_the_jedi", + ]; + // keys node b has + let lotr_movies = &[ + "lotr/fellowship_of_the_ring", + "lotr/the_two_towers", + "lotr/return_of_the_king", + ]; + + // content policy for what b wants + let policy_b = + DownloadPolicy::EverythingExcept(vec![FilterKind::Prefix("star_wars/og".into())]); + // content policy for what a wants + let policy_a = DownloadPolicy::NothingExcept(vec![FilterKind::Exact( + "lotr/fellowship_of_the_ring".into(), + )]); + + // a will sync all lotr keys but download a single key + const EXPECTED_A_SYNCED: usize = 3; + const EXPECTED_A_DOWNLOADED: usize = 1; + + // b will sync all star wars content but download only the prequel keys + const EXPECTED_B_SYNCED: usize = 6; + const EXPECTED_B_DOWNLOADED: usize = 3; + + let mut rng = test_rng(b"sync_download_policies"); + setup_logging(); + let nodes = spawn_nodes(2, &mut rng).await?; + let clients = nodes.iter().map(|node| node.client()).collect::>(); + + let doc_a = clients[0].docs().create().await?; + let author_a = clients[0].docs().author_create().await?; + let ticket = doc_a + .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) + .await?; + + let doc_b = clients[1].docs().import(ticket).await?; + let author_b = clients[1].docs().author_create().await?; + + doc_a.set_download_policy(policy_a).await?; + doc_b.set_download_policy(policy_b).await?; + + let mut events_a = doc_a.subscribe().await?; + let mut events_b = doc_b.subscribe().await?; + + let mut key_hashes: HashMap = HashMap::default(); + + // set content in a + for k in star_wars_movies.iter() { + let hash = doc_a + .set_bytes(author_a, k.to_owned(), k.to_owned()) + .await?; + key_hashes.insert(hash, k); + } + + // set content in b + for k in lotr_movies.iter() { + let hash = doc_b + .set_bytes(author_b, k.to_owned(), k.to_owned()) + .await?; + key_hashes.insert(hash, k); + } + + assert_eq!(key_hashes.len(), star_wars_movies.len() + lotr_movies.len()); + + let fut = async { + use LiveEvent::*; + let mut downloaded_a: Vec<&'static str> = Vec::new(); + let mut downloaded_b: Vec<&'static str> = Vec::new(); + let mut synced_a = 0usize; + let mut synced_b = 0usize; + loop { + tokio::select! { + Some(Ok(ev)) = events_a.next() => { + match ev { + InsertRemote { content_status, entry, .. } => { + synced_a += 1; + if let ContentStatus::Complete = content_status { + downloaded_a.push(key_hashes.get(&entry.content_hash()).unwrap()) + } + }, + ContentReady { hash } => { + downloaded_a.push(key_hashes.get(&hash).unwrap()); + }, + _ => {} + } + } + Some(Ok(ev)) = events_b.next() => { + match ev { + InsertRemote { content_status, entry, .. } => { + synced_b += 1; + if let ContentStatus::Complete = content_status { + downloaded_b.push(key_hashes.get(&entry.content_hash()).unwrap()) + } + }, + ContentReady { hash } => { + downloaded_b.push(key_hashes.get(&hash).unwrap()); + }, + _ => {} + } + } + } + + if synced_a == EXPECTED_A_SYNCED + && downloaded_a.len() == EXPECTED_A_DOWNLOADED + && synced_b == EXPECTED_B_SYNCED + && downloaded_b.len() == EXPECTED_B_DOWNLOADED + { + break; + } + } + (downloaded_a, downloaded_b) + }; + + let (downloaded_a, mut downloaded_b) = tokio::time::timeout(TIMEOUT, fut) + .await + .context("timeout elapsed")?; + + downloaded_b.sort(); + assert_eq!(downloaded_a, vec!["lotr/fellowship_of_the_ring"]); + assert_eq!( + downloaded_b, + vec![ + "star_wars/prequel/attack_of_the_clones", + "star_wars/prequel/revenge_of_the_sith", + "star_wars/prequel/the_phantom_menace", + ] + ); + + Ok(()) +} + +/// Test sync between many nodes with propagation through sync reports. +#[tokio::test(flavor = "multi_thread")] +#[ignore = "flaky"] +async fn sync_big() -> Result<()> { + setup_logging(); + let mut rng = test_rng(b"sync_big"); + let n_nodes = std::env::var("NODES") + .map(|v| v.parse().expect("NODES must be a number")) + .unwrap_or(10); + let n_entries_init = 1; + + tokio::task::spawn(async move { + for i in 0.. { + tokio::time::sleep(Duration::from_secs(1)).await; + info!("tick {i}"); + } + }); + + let nodes = spawn_nodes(n_nodes, &mut rng).await?; + let node_ids = nodes.iter().map(|node| node.node_id()).collect::>(); + let clients = nodes.iter().map(|node| node.client()).collect::>(); + let authors = collect_futures(clients.iter().map(|c| c.docs().author_create())).await?; + + let doc0 = clients[0].docs().create().await?; + let mut ticket = doc0 + .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) + .await?; + // do not join for now, just import without any peer info + let peer0 = ticket.nodes[0].clone(); + ticket.nodes = vec![]; + + let docs_clients: Vec<_> = clients.iter().skip(1).collect(); + let mut docs = vec![]; + docs.push(doc0); + docs.extend_from_slice( + &collect_futures(docs_clients.into_iter().map(|c| { + let ticket = ticket.clone(); + async move { c.docs().import(ticket).await } + })) + .await?, + ); + + let mut expected = vec![]; + + // create initial data on each node + publish(&docs, &mut expected, n_entries_init, |i, j| { + ( + authors[i], + format!("init/{}/{j}", node_ids[i].fmt_short()), + format!("init:{i}:{j}"), + ) + }) + .await?; + + // assert initial data + for (i, doc) in docs.iter().enumerate() { + let blobs = nodes[i].client().blobs(); + let entries = get_all_with_content(&blobs, doc).await?; + let mut expected = expected + .iter() + .filter(|e| e.author == authors[i]) + .cloned() + .collect::>(); + expected.sort(); + assert_eq!(entries, expected, "phase1 pre-sync correct"); + } + + // setup event streams + let events = collect_futures(docs.iter().map(|d| d.subscribe())).await?; + + // join nodes together + for (i, doc) in docs.iter().enumerate().skip(1) { + info!(me = %node_ids[i].fmt_short(), peer = %peer0.node_id.fmt_short(), "join"); + doc.start_sync(vec![peer0.clone()]).await?; + } + + // wait for InsertRemote events stuff to happen + info!("wait for all peers to receive insert events"); + let expected_inserts = (n_nodes - 1) * n_entries_init; + let mut tasks = tokio::task::JoinSet::default(); + for (i, events) in events.into_iter().enumerate() { + let doc = docs[i].clone(); + let me = doc.id().fmt_short(); + let expected = expected.clone(); + let fut = async move { + wait_for_events(events, expected_inserts, TIMEOUT, |e| { + matches!(e, LiveEvent::InsertRemote { .. }) + }) + .await?; + let entries = get_all(&doc).await?; + if entries != expected { + Err(anyhow!( + "node {i} failed (has {} entries but expected to have {})", + entries.len(), + expected.len() + )) + } else { + info!( + "received and checked all {} expected entries", + expected.len() + ); + Ok(()) + } + } + .instrument(error_span!("sync-test", %me)); + let fut = fut.map(move |r| r.with_context(move || format!("node {i} ({me})"))); + tasks.spawn(fut); + } + + while let Some(res) = tasks.join_next().await { + res??; + } + + assert_all_docs(&docs, &node_ids, &expected, "after initial sync").await; + + info!("shutdown"); + for node in nodes { + node.shutdown().await?; + } + + Ok(()) +} + +#[tokio::test] +#[cfg(feature = "test-utils")] +async fn test_list_docs_stream() -> Result<()> { + let node = Node::memory() + .node_discovery(iroh::node::DiscoveryConfig::None) + .relay_mode(iroh::net::relay::RelayMode::Disabled) + .spawn() + .await?; + let count = 200; + + // create docs + for _i in 0..count { + let doc = node.docs().create().await?; + doc.close().await?; + } + + // create doc stream + let mut stream = node.docs().list().await?; + + // process each doc and call into the docs actor. + // this makes sure that we don't deadlock the docs actor. + let mut i = 0; + let fut = async { + while let Some((id, _)) = stream.try_next().await.unwrap() { + let _doc = node.docs().open(id).await.unwrap().unwrap(); + i += 1; + } + }; + + tokio::time::timeout(Duration::from_secs(2), fut) + .await + .expect("not to timeout"); + + assert_eq!(i, count); + + Ok(()) +} + +/// Get all entries of a document. +async fn get_all(doc: &Doc) -> anyhow::Result> { + let entries = doc.get_many(Query::all()).await?; + let entries = entries.collect::>().await; + entries.into_iter().collect() +} + +/// Get all entries of a document with the blob content. +async fn get_all_with_content( + blobs: &iroh_blobs::rpc::client::blobs::Client, + doc: &Doc, +) -> anyhow::Result> { + let entries = doc.get_many(Query::all()).await?; + let entries = entries.and_then(|entry| async { + let hash = entry.content_hash(); + let content = blobs.read_to_bytes(hash).await; + content.map(|c| (entry, c)) + }); + let entries = entries.collect::>().await; + let entries = entries.into_iter().collect::>>()?; + Ok(entries) +} + +async fn publish( + docs: &[Doc], + expected: &mut Vec, + n: usize, + cb: impl Fn(usize, usize) -> (AuthorId, String, String), +) -> anyhow::Result<()> { + for (i, doc) in docs.iter().enumerate() { + for j in 0..n { + let (author, key, value) = cb(i, j); + doc.set_bytes(author, key.as_bytes().to_vec(), value.as_bytes().to_vec()) + .await?; + expected.push(ExpectedEntry { author, key, value }); + } + } + expected.sort(); + Ok(()) +} + +/// Collect an iterator into futures by joining them all and failing if any future failed. +async fn collect_futures( + futs: impl IntoIterator>>, +) -> anyhow::Result> { + futures_buffered::join_all(futs) + .await + .into_iter() + .collect::>>() +} + +/// Collect `count` events from the `events` stream, only collecting events for which `matcher` +/// returns true. +async fn wait_for_events( + mut events: impl Stream> + Send + Unpin + 'static, + count: usize, + timeout: Duration, + matcher: impl Fn(&LiveEvent) -> bool, +) -> anyhow::Result> { + let mut res = Vec::with_capacity(count); + let sleep = tokio::time::sleep(timeout); + tokio::pin!(sleep); + while res.len() < count { + tokio::select! { + () = &mut sleep => { + bail!("Failed to collect {count} elements in {timeout:?} (collected only {})", res.len()); + }, + event = events.try_next() => { + let event = event?; + match event { + None => bail!("stream ended after {} items, but expected {count}", res.len()), + Some(event) => if matcher(&event) { + res.push(event); + debug!("recv event {} of {count}", res.len()); + } + } + } + } + } + Ok(res) +} + +async fn assert_all_docs( + docs: &[Doc], + node_ids: &[PublicKey], + expected: &Vec, + label: &str, +) { + info!("validate all peers: {label}"); + for (i, doc) in docs.iter().enumerate() { + let entries = get_all(doc).await.unwrap_or_else(|err| { + panic!("failed to get entries for peer {:?}: {err:?}", node_ids[i]) + }); + assert_eq!( + &entries, + expected, + "{label}: peer {i} {:?} failed (have {} but expected {})", + node_ids[i], + entries.len(), + expected.len() + ); + } +} + +#[derive(Debug, Ord, Eq, PartialEq, PartialOrd, Clone)] +struct ExpectedEntry { + author: AuthorId, + key: String, + value: String, +} + +impl PartialEq for ExpectedEntry { + fn eq(&self, other: &Entry) -> bool { + self.key.as_bytes() == other.key() + && Hash::new(&self.value) == other.content_hash() + && self.author == other.author() + } +} +impl PartialEq<(Entry, Bytes)> for ExpectedEntry { + fn eq(&self, (entry, content): &(Entry, Bytes)) -> bool { + self.key.as_bytes() == entry.key() + && Hash::new(&self.value) == entry.content_hash() + && self.author == entry.author() + && self.value.as_bytes() == content.as_ref() + } +} +impl PartialEq for Entry { + fn eq(&self, other: &ExpectedEntry) -> bool { + other.eq(self) + } +} +impl PartialEq for (Entry, Bytes) { + fn eq(&self, other: &ExpectedEntry) -> bool { + other.eq(self) + } +} + +#[tokio::test] +async fn doc_delete() -> Result<()> { + let node = Node::memory() + .gc_interval(Some(Duration::from_millis(100))) + .spawn() + .await?; + let client = node.client(); + let doc = client.docs().create().await?; + let blobs = client.blobs(); + let author = client.docs().author_create().await?; + let hash = doc + .set_bytes(author, b"foo".to_vec(), b"hi".to_vec()) + .await?; + assert_latest(&blobs, &doc, b"foo", b"hi").await; + let deleted = doc.del(author, b"foo".to_vec()).await?; + assert_eq!(deleted, 1); + + let entry = doc.get_exact(author, b"foo".to_vec(), false).await?; + assert!(entry.is_none()); + + // wait for gc + // TODO: allow to manually trigger gc + tokio::time::sleep(Duration::from_millis(200)).await; + let bytes = client.blobs().read_to_bytes(hash).await; + assert!(bytes.is_err()); + node.shutdown().await?; + Ok(()) +} + +#[tokio::test] +async fn sync_drop_doc() -> Result<()> { + let mut rng = test_rng(b"sync_drop_doc"); + setup_logging(); + let node = spawn_node(0, &mut rng).await?; + let client = node.client(); + + let doc = client.docs().create().await?; + let author = client.docs().author_create().await?; + + let mut sub = doc.subscribe().await?; + doc.set_bytes(author, b"foo".to_vec(), b"bar".to_vec()) + .await?; + let ev = sub.next().await; + assert!(matches!(ev, Some(Ok(LiveEvent::InsertLocal { .. })))); + + client.docs().drop_doc(doc.id()).await?; + let res = doc.get_exact(author, b"foo".to_vec(), true).await; + assert!(res.is_err()); + let res = doc + .set_bytes(author, b"foo".to_vec(), b"bar".to_vec()) + .await; + assert!(res.is_err()); + let res = client.docs().open(doc.id()).await; + assert!(res.is_err()); + let ev = sub.next().await; + assert!(ev.is_none()); + + Ok(()) +} + +async fn assert_latest( + blobs: &iroh_blobs::rpc::client::blobs::Client, + doc: &Doc, + key: &[u8], + value: &[u8], +) { + let content = get_latest(blobs, doc, key).await.unwrap(); + assert_eq!(content, value.to_vec()); +} + +async fn get_latest( + blobs: &iroh_blobs::rpc::client::blobs::Client, + doc: &Doc, + key: &[u8], +) -> anyhow::Result> { + let query = Query::single_latest_per_key().key_exact(key); + let entry = doc + .get_many(query) + .await? + .next() + .await + .ok_or_else(|| anyhow!("entry not found"))??; + let content = blobs.read_to_bytes(entry.content_hash()).await?; + Ok(content.to_vec()) +} + +fn setup_logging() { + tracing_subscriber::registry() + .with(tracing_subscriber::fmt::layer().with_writer(std::io::stderr)) + .with(EnvFilter::from_default_env()) + .try_init() + .ok(); +} + +async fn next(mut stream: impl Stream> + Unpin) -> T { + let event = stream + .next() + .await + .expect("stream ended") + .expect("stream produced error"); + debug!("Event: {event:?}"); + event +} + +#[allow(clippy::type_complexity)] +fn apply_matchers(item: &T, matchers: &mut Vec bool + Send>>) -> bool { + for i in 0..matchers.len() { + if matchers[i](item) { + let _ = matchers.remove(i); + return true; + } + } + false +} + +/// Receive the next `matchers.len()` elements from a stream and matches them against the functions +/// in `matchers`, in order. +/// +/// Returns all received events. +#[allow(clippy::type_complexity)] +async fn assert_next( + mut stream: impl Stream> + Unpin + Send, + timeout: Duration, + matchers: Vec bool + Send>>, +) -> Vec { + let fut = async { + let mut items = vec![]; + for (i, f) in matchers.iter().enumerate() { + let item = stream + .next() + .await + .expect("event stream ended prematurely") + .expect("event stream errored"); + if !(f)(&item) { + panic!("assertion failed for event {i} {item:?}"); + } + items.push(item); + } + items + }; + let res = tokio::time::timeout(timeout, fut).await; + res.expect("timeout reached") +} + +/// Receive `matchers.len()` elements from a stream and assert that each element matches one of the +/// functions in `matchers`. +/// +/// Order of the matchers is not relevant. +/// +/// Returns all received events. +#[allow(clippy::type_complexity)] +async fn assert_next_unordered( + stream: impl Stream> + Unpin + Send, + timeout: Duration, + matchers: Vec bool + Send>>, +) -> Vec { + assert_next_unordered_with_optionals(stream, timeout, matchers, vec![]).await +} + +/// Receive between `min` and `max` elements from the stream and assert that each element matches +/// either one of the matchers in `required_matchers` or in `optional_matchers`. +/// +/// Order of the matchers is not relevant. +/// +/// Will return an error if: +/// * Any element fails to match one of the required or optional matchers +/// * More than `max` elements were received, but not all required matchers were used yet +/// * The timeout completes before all required matchers were used +/// +/// Returns all received events. +#[allow(clippy::type_complexity)] +async fn assert_next_unordered_with_optionals( + mut stream: impl Stream> + Unpin + Send, + timeout: Duration, + mut required_matchers: Vec bool + Send>>, + mut optional_matchers: Vec bool + Send>>, +) -> Vec { + let max = required_matchers.len() + optional_matchers.len(); + let required = required_matchers.len(); + // we have to use a mutex because rustc is not intelligent enough to realize + // that the mutable borrow terminates when the future completes + let events = Arc::new(parking_lot::Mutex::new(vec![])); + let fut = async { + while let Some(event) = stream.next().await { + let event = event.context("failed to read from stream")?; + let len = { + let mut events = events.lock(); + events.push(event.clone()); + events.len() + }; + if !apply_matchers(&event, &mut required_matchers) + && !apply_matchers(&event, &mut optional_matchers) + { + bail!("Event didn't match any matcher: {event:?}"); + } + if required_matchers.is_empty() || len == max { + break; + } + } + if !required_matchers.is_empty() { + bail!( + "Matched only {} of {required} required matchers", + required - required_matchers.len() + ); + } + Ok(()) + }; + tokio::pin!(fut); + let res = tokio::time::timeout(timeout, fut) + .await + .map_err(|_| anyhow!("Timeout reached ({timeout:?})")) + .and_then(|res| res); + let events = events.lock().clone(); + if let Err(err) = &res { + println!("Received events: {events:#?}"); + println!( + "Received {} events, expected between {required} and {max}", + events.len() + ); + panic!("Failed to receive or match all events: {err:?}"); + } + events +} + +/// Asserts that the event is a [`LiveEvent::SyncFinished`] and that the contained [`SyncEvent`] +/// has no error and matches `peer` and `namespace`. +fn match_sync_finished(event: &LiveEvent, peer: PublicKey) -> bool { + let LiveEvent::SyncFinished(e) = event else { + return false; + }; + e.peer == peer && e.result.is_ok() +} diff --git a/tests/util.rs b/tests/util.rs new file mode 100644 index 0000000..5775057 --- /dev/null +++ b/tests/util.rs @@ -0,0 +1,273 @@ +//! An iroh node that just has the blobs transport +use std::{collections::BTreeSet, sync::Arc}; + +use iroh_blobs::{ + store::{GcConfig, Store as BlobStore}, + util::local_pool::{LocalPool, Run}, +}; +use iroh_net::{key::SecretKey, relay::RelayMode, NodeId}; +use nested_enum_utils::enum_conversions; +use quic_rpc::transport::{Connector, Listener}; +use serde::{Deserialize, Serialize}; +use tokio_util::task::AbortOnDropHandle; + +/// An iroh node that just has the blobs transport +#[derive(Debug)] +pub struct Node { + router: iroh_router::Router, + client: Client, + _store: S, + _local_pool: LocalPool, + _rpc_task: AbortOnDropHandle<()>, + _gc_task: Option>, +} + +#[derive(Debug, Serialize, Deserialize)] +#[enum_conversions] +enum Request { + BlobsOrTags(iroh_blobs::rpc::proto::Request), + Docs(iroh_docs::rpc::proto::Request), +} + +#[derive(Debug, Serialize, Deserialize)] +#[enum_conversions] +enum Response { + BlobsOrTags(iroh_blobs::rpc::proto::Response), + Docs(iroh_docs::rpc::proto::Response), +} + +#[derive(Debug, Clone, Copy)] +struct Service; + +impl quic_rpc::Service for Service { + type Req = Request; + type Res = Response; +} + +#[derive(Debug, Clone)] +pub struct Client { + blobs: iroh_blobs::rpc::client::blobs::Client, + docs: iroh_docs::rpc::client::Client, +} + +impl Client { + fn new(client: quic_rpc::RpcClient) -> Self { + Self { + blobs: iroh_blobs::rpc::client::blobs::Client::new(client.clone().map().boxed()), + docs: iroh_docs::rpc::client::Client::new(client.map().boxed()), + } + } + + pub fn blobs(&self) -> &iroh_blobs::rpc::client::blobs::Client { + &self.blobs + } + + pub fn docs(&self) -> &iroh_docs::rpc::client::Client { + &self.docs + } +} + +/// An iroh node builder +#[derive(Debug)] +pub struct Builder { + store: S, + secret_key: Option, + relay_mode: RelayMode, + gc_interval: Option, +} + +impl Builder { + /// Spawns the node + pub async fn spawn(self) -> anyhow::Result> { + let (client, router, rpc_task, docs, local_pool) = + setup_router(self.store.clone(), self.relay_mode).await?; + let _gc_task = if let Some(period) = self.gc_interval { + let store = self.store.clone(); + let local_pool = local_pool.clone(); + let docs = docs.clone(); + let protected_cb = move || { + let docs = docs.clone(); + async move { + let mut live = BTreeSet::default(); + let doc_hashes = match docs.sync.content_hashes().await { + Ok(hashes) => hashes, + Err(err) => { + tracing::warn!("Error getting doc hashes: {}", err); + return live; + } + }; + for hash in doc_hashes { + match hash { + Ok(hash) => { + live.insert(hash); + } + Err(err) => { + tracing::error!("Error getting doc hash: {}", err); + } + } + } + live + } + }; + Some(local_pool.spawn(move || async move { + store + .gc_run( + GcConfig { + period, + done_callback: None, + }, + protected_cb, + ) + .await + })) + } else { + None + }; + let client = Client::new(client); + Ok(Node { + router, + client, + _store: self.store, + _rpc_task: AbortOnDropHandle::new(rpc_task), + _gc_task, + _local_pool: local_pool, + }) + } + + pub fn secret_key(mut self, value: SecretKey) -> Self { + self.secret_key = Some(value); + self + } + + pub fn relay_mode(mut self, value: RelayMode) -> Self { + self.relay_mode = value; + self + } + + pub fn gc_interval(mut self, value: Option) -> Self { + self.gc_interval = value; + self + } +} + +impl Node { + /// Creates a new node with memory storage + pub fn memory() -> Builder { + Builder { + store: iroh_blobs::store::mem::Store::new(), + secret_key: None, + relay_mode: RelayMode::Default, + gc_interval: None, + } + } +} + +impl Node { + /// Returns the node id + pub fn node_id(&self) -> NodeId { + self.router.endpoint().node_id() + } + + /// Shuts down the node + pub async fn shutdown(self) -> anyhow::Result<()> { + self.router.shutdown().await + } + + /// Returns the client + pub fn client(&self) -> &Client { + &self.client + } +} + +async fn setup_router( + store: S, + relay_mode: RelayMode, +) -> anyhow::Result<( + quic_rpc::RpcClient, + iroh_router::Router, + tokio::task::JoinHandle<()>, + iroh_docs::engine::Engine, + LocalPool, +)> { + let endpoint = iroh_net::Endpoint::builder() + .discovery_n0() + .relay_mode(relay_mode) + .bind() + .await?; + let addr = endpoint.node_addr().await?; + let local_pool = LocalPool::single(); + let mut router = iroh_router::Router::builder(endpoint.clone()); + + // Setup blobs + let downloader = iroh_blobs::downloader::Downloader::new( + store.clone(), + endpoint.clone(), + local_pool.handle().clone(), + ); + let blobs = Arc::new(iroh_blobs::net_protocol::Blobs::new_with_events( + store.clone(), + local_pool.handle().clone(), + Default::default(), + downloader.clone(), + endpoint.clone(), + )); + let gossip = + iroh_gossip::net::Gossip::from_endpoint(endpoint.clone(), Default::default(), &addr.info); + let docs = iroh_docs::engine::Engine::spawn( + endpoint, + gossip.clone(), + iroh_docs::store::Store::memory(), + store.clone(), + downloader, + iroh_docs::engine::DefaultAuthorStorage::Mem, + local_pool.handle().clone(), + ) + .await?; + router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), blobs.clone()); + router = router.accept(iroh_docs::net::DOCS_ALPN.to_vec(), Arc::new(docs.clone())); + router = router.accept( + iroh_gossip::net::GOSSIP_ALPN.to_vec(), + Arc::new(gossip.clone()), + ); + + // Build the router + let router = router.spawn().await?; + + // Setup RPC + let (internal_rpc, controller) = quic_rpc::transport::flume::channel::(32); + let controller = controller.boxed(); + let internal_rpc = internal_rpc.boxed(); + let internal_rpc = quic_rpc::RpcServer::::new(internal_rpc); + + let docs2 = docs.clone(); + let rpc_server_task: tokio::task::JoinHandle<()> = tokio::task::spawn(async move { + loop { + let request = internal_rpc.accept().await; + match request { + Ok(accepting) => { + let blobs = blobs.clone(); + let docs = docs2.clone(); + tokio::task::spawn(async move { + let (msg, chan) = accepting.read_first().await?; + match msg { + Request::BlobsOrTags(msg) => { + blobs.handle_rpc_request(msg, chan.map().boxed()).await?; + } + Request::Docs(msg) => { + docs.handle_rpc_request(msg, chan.map().boxed()).await?; + } + } + anyhow::Ok(()) + }); + } + Err(err) => { + tracing::warn!("rpc error: {:?}", err); + } + } + } + }); + + let client = quic_rpc::RpcClient::new(controller); + + Ok((client, router, rpc_server_task, docs, local_pool)) +} From f745aa8b9d551857377066001f8e9055ee454574 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Mon, 11 Nov 2024 14:27:51 +0200 Subject: [PATCH 11/26] Reactivate all old tests and fix warnings --- Cargo.lock | 294 ++++++++++++++++++++++++++- Cargo.toml | 3 +- tests/gc.rs | 545 ++++++++++++++++++++++++++++++++++++++++++++++---- tests/sync.rs | 15 +- tests/util.rs | 290 +++++++++++++++++---------- 5 files changed, 982 insertions(+), 165 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c06db74..cca798e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -70,6 +70,55 @@ dependencies = [ "libc", ] +[[package]] +name = "anstream" +version = "0.6.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8acc5369981196006228e28809f761875c0327210a891e941f4c683b3a99529b" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "is_terminal_polyfill", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" + +[[package]] +name = "anstyle-parse" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b2d16507662817a6a20a9ea92df6652ee4f94f914589377d69f3b21bc5798a9" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79947af37f4177cfead1110013d678905c37501914fba0efea834c3fe9a8d60c" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2109dbce0e72be3ec00bed26e6a7479ca384ad226efdd66db8fa2e3a38c83125" +dependencies = [ + "anstyle", + "windows-sys 0.59.0", +] + [[package]] name = "anyhow" version = "1.0.93" @@ -184,6 +233,61 @@ version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" +[[package]] +name = "axum" +version = "0.7.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "504e3947307ac8326a5437504c517c4b56716c9d98fac0028c2acc7ca47d70ae" +dependencies = [ + "async-trait", + "axum-core", + "bytes", + "futures-util", + "http 1.1.0", + "http-body", + "http-body-util", + "hyper", + "hyper-util", + "itoa", + "matchit", + "memchr", + "mime", + "percent-encoding", + "pin-project-lite", + "rustversion", + "serde", + "serde_json", + "serde_path_to_error", + "serde_urlencoded", + "sync_wrapper 1.0.1", + "tokio", + "tower", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "axum-core" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09f2bd6146b97ae3359fa0cc6d6b376d9539582c7b4220f041a33ec24c226199" +dependencies = [ + "async-trait", + "bytes", + "futures-util", + "http 1.1.0", + "http-body", + "http-body-util", + "mime", + "pin-project-lite", + "rustversion", + "sync_wrapper 1.0.1", + "tower-layer", + "tower-service", + "tracing", +] + [[package]] name = "backoff" version = "0.4.0" @@ -233,6 +337,12 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c7f02d4ea65f2c1853089ffd8d2787bdbc63de2f0d29dedbcf8ccdfa0ccd4cf" +[[package]] +name = "base64" +version = "0.21.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d297deb1925b89f2ccc13d7635fa0714f12c87adce1c75356b39ca9b7178567" + [[package]] name = "base64" version = "0.22.1" @@ -378,12 +488,58 @@ dependencies = [ "zeroize", ] +[[package]] +name = "clap" +version = "4.5.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b97f376d85a664d5837dbae44bf546e6477a679ff6610010f17276f686d867e8" +dependencies = [ + "clap_builder", + "clap_derive", +] + +[[package]] +name = "clap_builder" +version = "4.5.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19bc80abd44e4bed93ca373a0704ccbd1b710dc5749406201bb018272808dc54" +dependencies = [ + "anstream", + "anstyle", + "clap_lex", + "strsim", +] + +[[package]] +name = "clap_derive" +version = "4.5.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ac6a0c7b1a9e9a5186361f67dfa1b88213572f427fb9ab038efb2bd8c582dab" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.87", +] + +[[package]] +name = "clap_lex" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1462739cb27611015575c0c11df5df7601141071f07518d56fcc1be504cbec97" + [[package]] name = "cobs" version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "67ba02a97a2bd10f4b59b25c7973101c79642302776489e030cd13cdab09ed15" +[[package]] +name = "colorchoice" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" + [[package]] name = "combine" version = "4.6.7" @@ -1972,9 +2128,11 @@ version = "0.28.1" source = "git+https://github.com/n0-computer/iroh?branch=main#91d44dc4061b071847f0797a7c638aa4405dd3f7" dependencies = [ "anyhow", + "axum", "backoff", - "base64", + "base64 0.22.1", "bytes", + "clap", "der", "derive_more", "duct", @@ -2014,10 +2172,12 @@ dependencies = [ "postcard", "rand", "rcgen", + "regex", "reqwest", "ring", "rtnetlink", "rustls", + "rustls-pemfile", "rustls-webpki", "serde", "smallvec", @@ -2029,11 +2189,14 @@ dependencies = [ "time", "tokio", "tokio-rustls", + "tokio-rustls-acme", "tokio-stream", "tokio-tungstenite", "tokio-tungstenite-wasm", "tokio-util", + "toml", "tracing", + "tracing-subscriber", "tungstenite", "url", "watchable", @@ -2120,6 +2283,12 @@ dependencies = [ "tracing-subscriber", ] +[[package]] +name = "is_terminal_polyfill" +version = "1.70.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" + [[package]] name = "itoa" version = "1.0.11" @@ -2282,6 +2451,12 @@ dependencies = [ "regex-automata 0.1.10", ] +[[package]] +name = "matchit" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" + [[package]] name = "md5" version = "0.7.0" @@ -2744,7 +2919,7 @@ version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e459365e590736a54c3fa561947c84837534b8e9af6fc5bf781307e82658fae" dependencies = [ - "base64", + "base64 0.22.1", "serde", ] @@ -2952,7 +3127,7 @@ version = "0.1.0" source = "git+https://github.com/n0-computer/iroh?branch=main#91d44dc4061b071847f0797a7c638aa4405dd3f7" dependencies = [ "anyhow", - "base64", + "base64 0.22.1", "bytes", "derive_more", "futures-lite 2.4.0", @@ -3467,7 +3642,7 @@ version = "0.12.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a77c62af46e79de0a562e1a9849205ffcb7fc1238876e9bd743357570e04046f" dependencies = [ - "base64", + "base64 0.22.1", "bytes", "futures-core", "futures-util", @@ -3491,7 +3666,7 @@ dependencies = [ "serde", "serde_json", "serde_urlencoded", - "sync_wrapper", + "sync_wrapper 1.0.1", "tokio", "tokio-rustls", "tower-service", @@ -3874,6 +4049,25 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_path_to_error" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af99884400da37c88f5e9146b7f1fd0fbcae8f6eec4e9da38b67d05486f814a6" +dependencies = [ + "itoa", + "serde", +] + +[[package]] +name = "serde_spanned" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87607cb1398ed59d48732e575a4c28a7a8ebf2454b964fe3f224f2afc07909e1" +dependencies = [ + "serde", +] + [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -4081,6 +4275,12 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +[[package]] +name = "strsim" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" + [[package]] name = "struct_iterable" version = "0.1.1" @@ -4161,7 +4361,7 @@ version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b79cc624c9a747353810310af44f1f03f71eb4561284a894acc0396e6d0de76e" dependencies = [ - "base64", + "base64 0.22.1", "bounded-integer", "byteorder", "crc", @@ -4234,6 +4434,12 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "sync_wrapper" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" + [[package]] name = "sync_wrapper" version = "1.0.1" @@ -4432,6 +4638,35 @@ dependencies = [ "tokio", ] +[[package]] +name = "tokio-rustls-acme" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4ee7cbca7da86fa030e33b0deac55bad0e0bf8ab909f1a84666f04447f6339b" +dependencies = [ + "async-trait", + "base64 0.21.7", + "chrono", + "futures", + "log", + "num-bigint", + "pem", + "proc-macro2", + "rcgen", + "reqwest", + "ring", + "rustls", + "serde", + "serde_json", + "thiserror", + "time", + "tokio", + "tokio-rustls", + "url", + "webpki-roots", + "x509-parser", +] + [[package]] name = "tokio-stream" version = "0.1.16" @@ -4490,11 +4725,26 @@ dependencies = [ "tokio", ] +[[package]] +name = "toml" +version = "0.8.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1ed1f98e3fdc28d6d910e6737ae6ab1a93bf1985935a1193e68f93eeb68d24e" +dependencies = [ + "serde", + "serde_spanned", + "toml_datetime", + "toml_edit", +] + [[package]] name = "toml_datetime" version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0dd7358ecb8fc2f8d014bf86f6f638ce72ba252a2c3a2572f2a795f1d23efb41" +dependencies = [ + "serde", +] [[package]] name = "toml_edit" @@ -4503,10 +4753,34 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" dependencies = [ "indexmap", + "serde", + "serde_spanned", "toml_datetime", "winnow", ] +[[package]] +name = "tower" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2873938d487c3cfb9aed7546dc9f2711d867c9f90c46b889989a2cb84eba6b4f" +dependencies = [ + "futures-core", + "futures-util", + "pin-project-lite", + "sync_wrapper 0.1.2", + "tokio", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower-layer" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "121c2a6cda46980bb0fcd1647ffaf6cd3fc79a013de288782836f6df9c48780e" + [[package]] name = "tower-service" version = "0.3.3" @@ -4695,7 +4969,7 @@ version = "2.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b74fc6b57825be3373f7054754755f03ac3a8f5d70015ccad699ba2029956f4a" dependencies = [ - "base64", + "base64 0.22.1", "log", "once_cell", "rustls", @@ -4734,6 +5008,12 @@ version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" +[[package]] +name = "utf8parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" + [[package]] name = "valuable" version = "0.1.0" diff --git a/Cargo.toml b/Cargo.toml index 10243ec..41fb628 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -78,10 +78,11 @@ testresult = "0.4.1" nested_enum_utils = "0.1.0" [features] -default = ["net", "metrics", "engine", "rpc"] +default = ["net", "metrics", "engine", "rpc", "test-utils"] net = ["dep:iroh-net", "tokio/io-util", "dep:tokio-stream", "dep:tokio-util"] metrics = ["iroh-metrics/metrics"] engine = ["net", "dep:iroh-gossip", "dep:iroh-blobs", "dep:iroh-router"] +test-utils = ["iroh-net/test-utils"] rpc = [ "engine", "dep:nested_enum_utils", diff --git a/tests/gc.rs b/tests/gc.rs index 366eacc..c521c1e 100644 --- a/tests/gc.rs +++ b/tests/gc.rs @@ -1,43 +1,502 @@ -// use std::time::Duration; - -// use bytes::Bytes; -// use testresult::TestResult; - -// #[tokio::test] -// async fn redb_doc_import_stress() -> TestResult<()> { -// let _ = tracing_subscriber::fmt::try_init(); -// let dir = testdir!(); -// let bao_store = iroh_blobs::store::fs::Store::load(dir.join("store")).await?; -// let (node, _) = wrap_in_node(bao_store.clone(), Duration::from_secs(10)).await; -// let client = node.client(); -// let doc = client.docs().create().await?; -// let author = client.authors().create().await?; -// let temp_path = dir.join("temp"); -// tokio::fs::create_dir_all(&temp_path).await?; -// let mut to_import = Vec::new(); -// for i in 0..100 { -// let data = create_test_data(16 * 1024 * 3 + 1); -// let path = temp_path.join(format!("file{}", i)); -// tokio::fs::write(&path, &data).await?; -// let key = Bytes::from(format!("{}", path.display())); -// to_import.push((key, path, data)); -// } -// for (key, path, _) in to_import.iter() { -// let mut progress = doc.import_file(author, key.clone(), path, true).await?; -// while let Some(msg) = progress.next().await { -// tracing::info!("import progress {:?}", msg); -// } -// } -// for (i, (key, _, expected)) in to_import.iter().enumerate() { -// let Some(entry) = doc.get_exact(author, key.clone(), true).await? else { -// anyhow::bail!("doc entry not found {}", i); -// }; -// let hash = entry.content_hash(); -// let Some(content) = bao_store.get(&hash).await? else { -// anyhow::bail!("content not found {} {}", i, &hash.to_hex()[..8]); -// }; -// let data = content.data_reader().read_to_end().await?; -// assert_eq!(data, expected); -// } -// Ok(()) -// } +use std::{ + io::{Cursor, Write}, + time::Duration, +}; + +use anyhow::Result; +use bao_tree::{blake3, io::sync::Outboard, ChunkRanges}; +use bytes::Bytes; +use iroh_blobs::{ + hashseq::HashSeq, + store::{bao_tree, EntryStatus, MapMut, Store}, + util::Tag, + BlobFormat, HashAndFormat, IROH_BLOCK_SIZE, +}; +use rand::RngCore; +use util::{Builder, Node}; + +mod util; + +pub fn create_test_data(size: usize) -> Bytes { + let mut rand = rand::thread_rng(); + let mut res = vec![0u8; size]; + rand.fill_bytes(&mut res); + res.into() +} + +/// Take some data and encode it +pub fn simulate_remote(data: &[u8]) -> (blake3::Hash, Cursor) { + let outboard = bao_tree::io::outboard::PostOrderMemOutboard::create(data, IROH_BLOCK_SIZE); + let mut encoded = Vec::new(); + encoded + .write_all(outboard.tree.size().to_le_bytes().as_ref()) + .unwrap(); + bao_tree::io::sync::encode_ranges_validated(data, &outboard, &ChunkRanges::all(), &mut encoded) + .unwrap(); + let hash = outboard.root(); + (hash, Cursor::new(encoded.into())) +} + +/// Wrap a bao store in a node that has gc enabled. +async fn wrap_in_node( + bao_store: S, + gc_period: Duration, +) -> (Node, async_channel::Receiver<()>) +where + S: iroh_blobs::store::Store, +{ + let (gc_send, gc_recv) = async_channel::unbounded(); + let node = Builder::new(bao_store) + .gc_interval(Some(gc_period)) + .register_gc_done_cb(Box::new(move || { + gc_send.send_blocking(()).ok(); + })) + .spawn() + .await + .unwrap(); + (node, gc_recv) +} + +async fn gc_test_node() -> ( + Node, + iroh_blobs::store::mem::Store, + async_channel::Receiver<()>, +) { + let bao_store = iroh_blobs::store::mem::Store::new(); + let (node, gc_recv) = wrap_in_node(bao_store.clone(), Duration::from_millis(500)).await; + (node, bao_store, gc_recv) +} + +async fn step(evs: &async_channel::Receiver<()>) { + // drain the event queue, we want a new GC + while evs.try_recv().is_ok() {} + // wait for several GC cycles + for _ in 0..3 { + evs.recv().await.unwrap(); + } +} + +/// Test the absolute basics of gc, temp tags and tags for blobs. +#[tokio::test] +async fn gc_basics() -> Result<()> { + let _ = tracing_subscriber::fmt::try_init(); + let (node, bao_store, evs) = gc_test_node().await; + let data1 = create_test_data(1234); + let tt1 = bao_store.import_bytes(data1, BlobFormat::Raw).await?; + let data2 = create_test_data(5678); + let tt2 = bao_store.import_bytes(data2, BlobFormat::Raw).await?; + let h1 = *tt1.hash(); + let h2 = *tt2.hash(); + // temp tags are still there, so the entries should be there + step(&evs).await; + assert_eq!(bao_store.entry_status(&h1).await?, EntryStatus::Complete); + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::Complete); + + // drop the first tag, the entry should be gone after some time + drop(tt1); + step(&evs).await; + assert_eq!(bao_store.entry_status(&h1).await?, EntryStatus::NotFound); + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::Complete); + + // create an explicit tag for h1 (as raw) and then delete the temp tag. Entry should still be there. + let tag = Tag::from("test"); + bao_store + .set_tag(tag.clone(), Some(HashAndFormat::raw(h2))) + .await?; + drop(tt2); + tracing::info!("dropped tt2"); + step(&evs).await; + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::Complete); + + // delete the explicit tag, entry should be gone + bao_store.set_tag(tag, None).await?; + step(&evs).await; + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::NotFound); + + node.shutdown().await?; + Ok(()) +} + +/// Test gc for sequences of hashes that protect their children from deletion. +#[tokio::test] +async fn gc_hashseq_impl() -> Result<()> { + let _ = tracing_subscriber::fmt::try_init(); + let (node, bao_store, evs) = gc_test_node().await; + let data1 = create_test_data(1234); + let tt1 = bao_store.import_bytes(data1, BlobFormat::Raw).await?; + let data2 = create_test_data(5678); + let tt2 = bao_store.import_bytes(data2, BlobFormat::Raw).await?; + let seq = vec![*tt1.hash(), *tt2.hash()] + .into_iter() + .collect::(); + let ttr = bao_store + .import_bytes(seq.into_inner(), BlobFormat::HashSeq) + .await?; + let h1 = *tt1.hash(); + let h2 = *tt2.hash(); + let hr = *ttr.hash(); + drop(tt1); + drop(tt2); + + // there is a temp tag for the link seq, so it and its entries should be there + step(&evs).await; + assert_eq!(bao_store.entry_status(&h1).await?, EntryStatus::Complete); + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::Complete); + assert_eq!(bao_store.entry_status(&hr).await?, EntryStatus::Complete); + + // make a permanent tag for the link seq, then delete the temp tag. Entries should still be there. + let tag = Tag::from("test"); + bao_store + .set_tag(tag.clone(), Some(HashAndFormat::hash_seq(hr))) + .await?; + drop(ttr); + step(&evs).await; + assert_eq!(bao_store.entry_status(&h1).await?, EntryStatus::Complete); + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::Complete); + assert_eq!(bao_store.entry_status(&hr).await?, EntryStatus::Complete); + + // change the permanent tag to be just for the linkseq itself as a blob. Only the linkseq should be there, not the entries. + bao_store + .set_tag(tag.clone(), Some(HashAndFormat::raw(hr))) + .await?; + step(&evs).await; + assert_eq!(bao_store.entry_status(&h1).await?, EntryStatus::NotFound); + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::NotFound); + assert_eq!(bao_store.entry_status(&hr).await?, EntryStatus::Complete); + + // delete the permanent tag, everything should be gone + bao_store.set_tag(tag, None).await?; + step(&evs).await; + assert_eq!(bao_store.entry_status(&h1).await?, EntryStatus::NotFound); + assert_eq!(bao_store.entry_status(&h2).await?, EntryStatus::NotFound); + assert_eq!(bao_store.entry_status(&hr).await?, EntryStatus::NotFound); + + node.shutdown().await?; + Ok(()) +} + +#[cfg(feature = "fs-store")] +mod file { + use std::{io, path::PathBuf}; + + use bao_tree::{ + io::fsm::{BaoContentItem, ResponseDecoderNext}, + BaoTree, + }; + use futures_lite::StreamExt; + use iroh_blobs::{ + store::{BaoBatchWriter, ConsistencyCheckProgress, Map, MapEntryMut, ReportLevel}, + util::progress::{AsyncChannelProgressSender, ProgressSender as _}, + TempTag, + }; + use iroh_io::AsyncSliceReaderExt; + use testdir::testdir; + use tokio::io::AsyncReadExt; + + use super::*; + + fn path(root: PathBuf, suffix: &'static str) -> impl Fn(&iroh_blobs::Hash) -> PathBuf { + move |hash| root.join(format!("{}.{}", hash.to_hex(), suffix)) + } + + fn data_path(root: PathBuf) -> impl Fn(&iroh_blobs::Hash) -> PathBuf { + // this assumes knowledge of the internal directory structure of the flat store + path(root.join("data"), "data") + } + + fn outboard_path(root: PathBuf) -> impl Fn(&iroh_blobs::Hash) -> PathBuf { + // this assumes knowledge of the internal directory structure of the flat store + path(root.join("data"), "obao4") + } + + async fn check_consistency(store: &impl Store) -> anyhow::Result { + let mut max_level = ReportLevel::Trace; + let (tx, rx) = async_channel::bounded(1); + let task = tokio::task::spawn(async move { + while let Ok(ev) = rx.recv().await { + if let ConsistencyCheckProgress::Update { level, .. } = &ev { + max_level = max_level.max(*level); + } + } + }); + store + .consistency_check(false, AsyncChannelProgressSender::new(tx).boxed()) + .await?; + task.await?; + Ok(max_level) + } + + #[tokio::test] + async fn redb_doc_import_stress() -> Result<()> { + let _ = tracing_subscriber::fmt::try_init(); + let dir = testdir!(); + let bao_store = iroh_blobs::store::fs::Store::load(dir.join("store")).await?; + let (node, _) = wrap_in_node(bao_store.clone(), Duration::from_secs(10)).await; + let client = node.client(); + let doc = client.docs().create().await?; + let author = client.authors().create().await?; + let temp_path = dir.join("temp"); + tokio::fs::create_dir_all(&temp_path).await?; + let mut to_import = Vec::new(); + for i in 0..100 { + let data = create_test_data(16 * 1024 * 3 + 1); + let path = temp_path.join(format!("file{}", i)); + tokio::fs::write(&path, &data).await?; + let key = Bytes::from(format!("{}", path.display())); + to_import.push((key, path, data)); + } + for (key, path, _) in to_import.iter() { + let mut progress = doc.import_file(author, key.clone(), path, true).await?; + while let Some(msg) = progress.next().await { + tracing::info!("import progress {:?}", msg); + } + } + for (i, (key, _, expected)) in to_import.iter().enumerate() { + let Some(entry) = doc.get_exact(author, key.clone(), true).await? else { + anyhow::bail!("doc entry not found {}", i); + }; + let hash = entry.content_hash(); + let Some(content) = bao_store.get(&hash).await? else { + anyhow::bail!("content not found {} {}", i, &hash.to_hex()[..8]); + }; + let data = content.data_reader().read_to_end().await?; + assert_eq!(data, expected); + } + Ok(()) + } + + /// Test gc for sequences of hashes that protect their children from deletion. + #[tokio::test] + async fn gc_file_basics() -> Result<()> { + let _ = tracing_subscriber::fmt::try_init(); + let dir = testdir!(); + let path = data_path(dir.clone()); + let outboard_path = outboard_path(dir.clone()); + + let bao_store = iroh_blobs::store::fs::Store::load(dir.clone()).await?; + let (node, evs) = wrap_in_node(bao_store.clone(), Duration::from_millis(100)).await; + let data1 = create_test_data(10000000); + let tt1 = bao_store + .import_bytes(data1.clone(), BlobFormat::Raw) + .await?; + let data2 = create_test_data(1000000); + let tt2 = bao_store + .import_bytes(data2.clone(), BlobFormat::Raw) + .await?; + let seq = vec![*tt1.hash(), *tt2.hash()] + .into_iter() + .collect::(); + let ttr = bao_store + .import_bytes(seq.into_inner(), BlobFormat::HashSeq) + .await?; + + let h1 = *tt1.hash(); + let h2 = *tt2.hash(); + let hr = *ttr.hash(); + + // data is protected by the temp tag + step(&evs).await; + bao_store.sync().await?; + assert!(check_consistency(&bao_store).await? <= ReportLevel::Info); + // h1 is for a giant file, so we will have both data and outboard files + assert!(path(&h1).exists()); + assert!(outboard_path(&h1).exists()); + // h2 is for a mid sized file, so we will have just the data file + assert!(path(&h2).exists()); + assert!(!outboard_path(&h2).exists()); + // hr so small that data will be inlined and outboard will not exist at all + assert!(!path(&hr).exists()); + assert!(!outboard_path(&hr).exists()); + + drop(tt1); + drop(tt2); + let tag = Tag::from("test"); + bao_store + .set_tag(tag.clone(), Some(HashAndFormat::hash_seq(*ttr.hash()))) + .await?; + drop(ttr); + + // data is now protected by a normal tag, nothing should be gone + step(&evs).await; + bao_store.sync().await?; + assert!(check_consistency(&bao_store).await? <= ReportLevel::Info); + // h1 is for a giant file, so we will have both data and outboard files + assert!(path(&h1).exists()); + assert!(outboard_path(&h1).exists()); + // h2 is for a mid sized file, so we will have just the data file + assert!(path(&h2).exists()); + assert!(!outboard_path(&h2).exists()); + // hr so small that data will be inlined and outboard will not exist at all + assert!(!path(&hr).exists()); + assert!(!outboard_path(&hr).exists()); + + tracing::info!("changing tag from hashseq to raw, this should orphan the children"); + bao_store + .set_tag(tag.clone(), Some(HashAndFormat::raw(hr))) + .await?; + + // now only hr itself should be protected, but not its children + step(&evs).await; + bao_store.sync().await?; + assert!(check_consistency(&bao_store).await? <= ReportLevel::Info); + // h1 should be gone + assert!(!path(&h1).exists()); + assert!(!outboard_path(&h1).exists()); + // h2 should still not be there + assert!(!path(&h2).exists()); + assert!(!outboard_path(&h2).exists()); + // hr should still not be there + assert!(!path(&hr).exists()); + assert!(!outboard_path(&hr).exists()); + + bao_store.set_tag(tag, None).await?; + step(&evs).await; + bao_store.sync().await?; + assert!(check_consistency(&bao_store).await? <= ReportLevel::Info); + // h1 should be gone + assert!(!path(&h1).exists()); + assert!(!outboard_path(&h1).exists()); + // h2 should still not be there + assert!(!path(&h2).exists()); + assert!(!outboard_path(&h2).exists()); + // hr should still not be there + assert!(!path(&hr).exists()); + assert!(!outboard_path(&hr).exists()); + + node.shutdown().await?; + + Ok(()) + } + + /// Add a file to the store in the same way a download works. + /// + /// we know the hash in advance, create a partial entry, write the data to it and + /// the outboard file, then commit it to a complete entry. + /// + /// During this time, the partial entry is protected by a temp tag. + async fn simulate_download_partial( + bao_store: &S, + data: Bytes, + ) -> io::Result<(S::EntryMut, TempTag)> { + // simulate the remote side. + let (hash, mut response) = simulate_remote(data.as_ref()); + // simulate the local side. + // we got a hash and a response from the remote side. + let tt = bao_store.temp_tag(HashAndFormat::raw(hash.into())); + // get the size + let size = response.read_u64_le().await?; + // start reading the response + let mut reading = bao_tree::io::fsm::ResponseDecoder::new( + hash, + ChunkRanges::all(), + BaoTree::new(size, IROH_BLOCK_SIZE), + response, + ); + // create the partial entry + let entry = bao_store.get_or_create(hash.into(), size).await?; + // create the + let mut bw = entry.batch_writer().await?; + let mut buf = Vec::new(); + while let ResponseDecoderNext::More((next, res)) = reading.next().await { + let item = res?; + match &item { + BaoContentItem::Parent(_) => { + buf.push(item); + } + BaoContentItem::Leaf(_) => { + buf.push(item); + let batch = std::mem::take(&mut buf); + bw.write_batch(size, batch).await?; + } + } + reading = next; + } + bw.sync().await?; + drop(bw); + Ok((entry, tt)) + } + + async fn simulate_download_complete( + bao_store: &S, + data: Bytes, + ) -> io::Result { + let (entry, tt) = simulate_download_partial(bao_store, data).await?; + // commit the entry + bao_store.insert_complete(entry).await?; + Ok(tt) + } + + /// Test that partial files are deleted. + #[tokio::test] + async fn gc_file_partial() -> Result<()> { + let _ = tracing_subscriber::fmt::try_init(); + let dir = testdir!(); + let path = data_path(dir.clone()); + let outboard_path = outboard_path(dir.clone()); + + let bao_store = iroh_blobs::store::fs::Store::load(dir.clone()).await?; + let (node, evs) = wrap_in_node(bao_store.clone(), Duration::from_millis(10)).await; + + let data1: Bytes = create_test_data(10000000); + let (_entry, tt1) = simulate_download_partial(&bao_store, data1.clone()).await?; + drop(_entry); + let h1 = *tt1.hash(); + // partial data and outboard files should be there + step(&evs).await; + bao_store.sync().await?; + assert!(check_consistency(&bao_store).await? <= ReportLevel::Info); + assert!(path(&h1).exists()); + assert!(outboard_path(&h1).exists()); + + drop(tt1); + // partial data and outboard files should be gone + step(&evs).await; + bao_store.sync().await?; + assert!(check_consistency(&bao_store).await? <= ReportLevel::Info); + assert!(!path(&h1).exists()); + assert!(!outboard_path(&h1).exists()); + + node.shutdown().await?; + Ok(()) + } + + #[tokio::test] + async fn gc_file_stress() -> Result<()> { + let _ = tracing_subscriber::fmt::try_init(); + let dir = testdir!(); + + let bao_store = iroh_blobs::store::fs::Store::load(dir.clone()).await?; + let (node, evs) = wrap_in_node(bao_store.clone(), Duration::from_secs(1)).await; + + let mut deleted = Vec::new(); + let mut live = Vec::new(); + // download + for i in 0..100 { + let data: Bytes = create_test_data(16 * 1024 * 3 + 1); + let tt = simulate_download_complete(&bao_store, data).await.unwrap(); + if i % 100 == 0 { + let tag = Tag::from(format!("test{}", i)); + bao_store + .set_tag(tag.clone(), Some(HashAndFormat::raw(*tt.hash()))) + .await?; + live.push(*tt.hash()); + } else { + deleted.push(*tt.hash()); + } + } + step(&evs).await; + + for h in deleted.iter() { + assert_eq!(bao_store.entry_status(h).await?, EntryStatus::NotFound); + assert!(!dir.join(format!("data/{}.data", h.to_hex())).exists()); + } + + for h in live.iter() { + assert_eq!(bao_store.entry_status(h).await?, EntryStatus::Complete); + assert!(dir.join(format!("data/{}.data", h.to_hex())).exists()); + } + + node.shutdown().await?; + Ok(()) + } +} diff --git a/tests/sync.rs b/tests/sync.rs index 8263e87..f65b2f7 100644 --- a/tests/sync.rs +++ b/tests/sync.rs @@ -506,7 +506,7 @@ async fn test_sync_via_relay() -> Result<()> { .await?; let doc1 = node1.docs().create().await?; - let author1 = node1.authors().create().await?; + let author1 = node1.docs().author_create().await?; let inserted_hash = doc1 .set_bytes(author1, b"foo".to_vec(), b"bar".to_vec()) .await?; @@ -599,7 +599,7 @@ async fn sync_restart_node() -> Result<()> { .insecure_skip_relay_cert_verify(true) .relay_mode(RelayMode::Custom(relay_map.clone())) .dns_resolver(discovery_server.dns_resolver()) - .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) + .node_discovery(discovery_server.discovery(secret_key_1.clone())) .spawn() .await?; let id1 = node1.node_id(); @@ -619,11 +619,11 @@ async fn sync_restart_node() -> Result<()> { .relay_mode(RelayMode::Custom(relay_map.clone())) .insecure_skip_relay_cert_verify(true) .dns_resolver(discovery_server.dns_resolver()) - .node_discovery(discovery_server.discovery(secret_key_2.clone()).into()) + .node_discovery(discovery_server.discovery(secret_key_2.clone())) .spawn() .await?; let id2 = node2.node_id(); - let author2 = node2.authors().create().await?; + let author2 = node2.docs().author_create().await?; let doc2 = node2.docs().import(ticket.clone()).await?; let blobs2 = node2.blobs(); @@ -666,7 +666,7 @@ async fn sync_restart_node() -> Result<()> { .insecure_skip_relay_cert_verify(true) .relay_mode(RelayMode::Custom(relay_map.clone())) .dns_resolver(discovery_server.dns_resolver()) - .node_discovery(discovery_server.discovery(secret_key_1.clone()).into()) + .node_discovery(discovery_server.discovery(secret_key_1.clone())) .spawn() .await?; assert_eq!(id1, node1.node_id()); @@ -916,7 +916,7 @@ async fn sync_big() -> Result<()> { // assert initial data for (i, doc) in docs.iter().enumerate() { - let blobs = nodes[i].client().blobs(); + let blobs = nodes[i].blobs(); let entries = get_all_with_content(&blobs, doc).await?; let mut expected = expected .iter() @@ -987,8 +987,7 @@ async fn sync_big() -> Result<()> { #[cfg(feature = "test-utils")] async fn test_list_docs_stream() -> Result<()> { let node = Node::memory() - .node_discovery(iroh::node::DiscoveryConfig::None) - .relay_mode(iroh::net::relay::RelayMode::Disabled) + .relay_mode(RelayMode::Disabled) .spawn() .await?; let count = 200; diff --git a/tests/util.rs b/tests/util.rs index 5775057..68f3e81 100644 --- a/tests/util.rs +++ b/tests/util.rs @@ -1,16 +1,34 @@ -//! An iroh node that just has the blobs transport -use std::{collections::BTreeSet, sync::Arc}; +#![allow(dead_code)] +use std::{ + collections::BTreeSet, + net::{Ipv4Addr, Ipv6Addr, SocketAddrV4, SocketAddrV6}, + ops::Deref, + path::Path, + sync::Arc, +}; use iroh_blobs::{ store::{GcConfig, Store as BlobStore}, util::local_pool::{LocalPool, Run}, }; -use iroh_net::{key::SecretKey, relay::RelayMode, NodeId}; +use iroh_net::{discovery::Discovery, dns::DnsResolver, key::SecretKey, relay::RelayMode, NodeId}; use nested_enum_utils::enum_conversions; use quic_rpc::transport::{Connector, Listener}; use serde::{Deserialize, Serialize}; use tokio_util::task::AbortOnDropHandle; +/// Default bind address for the node. +/// 11204 is "iroh" in leetspeak +pub const DEFAULT_BIND_PORT: u16 = 11204; + +/// The default bind address for the iroh IPv4 socket. +pub const DEFAULT_BIND_ADDR_V4: SocketAddrV4 = + SocketAddrV4::new(Ipv4Addr::UNSPECIFIED, DEFAULT_BIND_PORT); + +/// The default bind address for the iroh IPv6 socket. +pub const DEFAULT_BIND_ADDR_V6: SocketAddrV6 = + SocketAddrV6::new(Ipv6Addr::UNSPECIFIED, DEFAULT_BIND_PORT + 1, 0, 0); + /// An iroh node that just has the blobs transport #[derive(Debug)] pub struct Node { @@ -22,6 +40,14 @@ pub struct Node { _gc_task: Option>, } +impl Deref for Node { + type Target = Client; + + fn deref(&self) -> &Self::Target { + &self.client + } +} + #[derive(Debug, Serialize, Deserialize)] #[enum_conversions] enum Request { @@ -68,19 +94,119 @@ impl Client { } /// An iroh node builder -#[derive(Debug)] +#[derive(derive_more::Debug)] pub struct Builder { store: S, secret_key: Option, relay_mode: RelayMode, + dns_resolver: Option, + node_discovery: Option>, gc_interval: Option, + #[debug(skip)] + register_gc_done_cb: Option>, + insecure_skip_relay_cert_verify: bool, + bind_random_port: bool, } impl Builder { /// Spawns the node pub async fn spawn(self) -> anyhow::Result> { - let (client, router, rpc_task, docs, local_pool) = - setup_router(self.store.clone(), self.relay_mode).await?; + let store = self.store.clone(); + let mut addr_v4 = DEFAULT_BIND_ADDR_V4; + let mut addr_v6 = DEFAULT_BIND_ADDR_V6; + if self.bind_random_port { + addr_v4.set_port(0); + addr_v6.set_port(0); + } + let mut builder = iroh_net::Endpoint::builder() + .bind_addr_v4(addr_v4) + .bind_addr_v6(addr_v6) + .discovery_n0() + .relay_mode(self.relay_mode.clone()) + .insecure_skip_relay_cert_verify(self.insecure_skip_relay_cert_verify); + if let Some(dns_resolver) = self.dns_resolver.clone() { + builder = builder.dns_resolver(dns_resolver); + } + let endpoint = builder.bind().await?; + let addr = endpoint.node_addr().await?; + let local_pool = LocalPool::single(); + let mut router = iroh_router::Router::builder(endpoint.clone()); + + // Setup blobs + let downloader = iroh_blobs::downloader::Downloader::new( + store.clone(), + endpoint.clone(), + local_pool.handle().clone(), + ); + let blobs = Arc::new(iroh_blobs::net_protocol::Blobs::new_with_events( + store.clone(), + local_pool.handle().clone(), + Default::default(), + downloader.clone(), + endpoint.clone(), + )); + let gossip = iroh_gossip::net::Gossip::from_endpoint( + endpoint.clone(), + Default::default(), + &addr.info, + ); + let docs = iroh_docs::engine::Engine::spawn( + endpoint, + gossip.clone(), + iroh_docs::store::Store::memory(), + store.clone(), + downloader, + iroh_docs::engine::DefaultAuthorStorage::Mem, + local_pool.handle().clone(), + ) + .await?; + router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), blobs.clone()); + router = router.accept(iroh_docs::net::DOCS_ALPN.to_vec(), Arc::new(docs.clone())); + router = router.accept( + iroh_gossip::net::GOSSIP_ALPN.to_vec(), + Arc::new(gossip.clone()), + ); + + // Build the router + let router = router.spawn().await?; + + // Setup RPC + let (internal_rpc, controller) = + quic_rpc::transport::flume::channel::(1); + let controller = controller.boxed(); + let internal_rpc = internal_rpc.boxed(); + let internal_rpc = quic_rpc::RpcServer::::new(internal_rpc); + + let docs2 = docs.clone(); + let rpc_task: tokio::task::JoinHandle<()> = tokio::task::spawn(async move { + loop { + let request = internal_rpc.accept().await; + match request { + Ok(accepting) => { + let blobs = blobs.clone(); + let docs = docs2.clone(); + tokio::task::spawn(async move { + let (msg, chan) = accepting.read_first().await?; + match msg { + Request::BlobsOrTags(msg) => { + blobs.handle_rpc_request(msg, chan.map().boxed()).await?; + } + Request::Docs(msg) => { + docs.handle_rpc_request(msg, chan.map().boxed()).await?; + } + } + anyhow::Ok(()) + }); + } + Err(err) => { + tracing::warn!("rpc error: {:?}", err); + } + } + } + }); + + let client = quic_rpc::RpcClient::new(controller); + let _gc_task = if let Some(period) = self.gc_interval { let store = self.store.clone(); let local_pool = local_pool.clone(); @@ -114,7 +240,7 @@ impl Builder { .gc_run( GcConfig { period, - done_callback: None, + done_callback: self.register_gc_done_cb, }, protected_cb, ) @@ -144,24 +270,69 @@ impl Builder { self } + pub fn dns_resolver(mut self, value: DnsResolver) -> Self { + self.dns_resolver = Some(value); + self + } + + pub fn node_discovery(mut self, value: Box) -> Self { + self.node_discovery = Some(value); + self + } + pub fn gc_interval(mut self, value: Option) -> Self { self.gc_interval = value; self } -} -impl Node { - /// Creates a new node with memory storage - pub fn memory() -> Builder { - Builder { - store: iroh_blobs::store::mem::Store::new(), + pub fn register_gc_done_cb(mut self, value: Box) -> Self { + self.register_gc_done_cb = Some(value); + self + } + + pub fn insecure_skip_relay_cert_verify(mut self, value: bool) -> Self { + self.insecure_skip_relay_cert_verify = value; + self + } + + pub fn bind_random_port(mut self) -> Self { + self.bind_random_port = true; + self + } + + pub fn new(store: S) -> Self { + Self { + store, secret_key: None, relay_mode: RelayMode::Default, gc_interval: None, + insecure_skip_relay_cert_verify: false, + bind_random_port: false, + dns_resolver: None, + node_discovery: None, + register_gc_done_cb: None, } } } +impl Node { + /// Creates a new node with memory storage + pub fn memory() -> Builder { + Builder::new(iroh_blobs::store::mem::Store::new()) + } +} + +impl Node { + /// Creates a new node with persistent storage + pub async fn persistent( + path: impl AsRef, + ) -> anyhow::Result> { + Ok(Builder::new( + iroh_blobs::store::fs::Store::load(path).await?, + )) + } +} + impl Node { /// Returns the node id pub fn node_id(&self) -> NodeId { @@ -178,96 +349,3 @@ impl Node { &self.client } } - -async fn setup_router( - store: S, - relay_mode: RelayMode, -) -> anyhow::Result<( - quic_rpc::RpcClient, - iroh_router::Router, - tokio::task::JoinHandle<()>, - iroh_docs::engine::Engine, - LocalPool, -)> { - let endpoint = iroh_net::Endpoint::builder() - .discovery_n0() - .relay_mode(relay_mode) - .bind() - .await?; - let addr = endpoint.node_addr().await?; - let local_pool = LocalPool::single(); - let mut router = iroh_router::Router::builder(endpoint.clone()); - - // Setup blobs - let downloader = iroh_blobs::downloader::Downloader::new( - store.clone(), - endpoint.clone(), - local_pool.handle().clone(), - ); - let blobs = Arc::new(iroh_blobs::net_protocol::Blobs::new_with_events( - store.clone(), - local_pool.handle().clone(), - Default::default(), - downloader.clone(), - endpoint.clone(), - )); - let gossip = - iroh_gossip::net::Gossip::from_endpoint(endpoint.clone(), Default::default(), &addr.info); - let docs = iroh_docs::engine::Engine::spawn( - endpoint, - gossip.clone(), - iroh_docs::store::Store::memory(), - store.clone(), - downloader, - iroh_docs::engine::DefaultAuthorStorage::Mem, - local_pool.handle().clone(), - ) - .await?; - router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), blobs.clone()); - router = router.accept(iroh_docs::net::DOCS_ALPN.to_vec(), Arc::new(docs.clone())); - router = router.accept( - iroh_gossip::net::GOSSIP_ALPN.to_vec(), - Arc::new(gossip.clone()), - ); - - // Build the router - let router = router.spawn().await?; - - // Setup RPC - let (internal_rpc, controller) = quic_rpc::transport::flume::channel::(32); - let controller = controller.boxed(); - let internal_rpc = internal_rpc.boxed(); - let internal_rpc = quic_rpc::RpcServer::::new(internal_rpc); - - let docs2 = docs.clone(); - let rpc_server_task: tokio::task::JoinHandle<()> = tokio::task::spawn(async move { - loop { - let request = internal_rpc.accept().await; - match request { - Ok(accepting) => { - let blobs = blobs.clone(); - let docs = docs2.clone(); - tokio::task::spawn(async move { - let (msg, chan) = accepting.read_first().await?; - match msg { - Request::BlobsOrTags(msg) => { - blobs.handle_rpc_request(msg, chan.map().boxed()).await?; - } - Request::Docs(msg) => { - docs.handle_rpc_request(msg, chan.map().boxed()).await?; - } - } - anyhow::Ok(()) - }); - } - Err(err) => { - tracing::warn!("rpc error: {:?}", err); - } - } - } - }); - - let client = quic_rpc::RpcClient::new(controller); - - Ok((client, router, rpc_server_task, docs, local_pool)) -} From db497b0a8e69e887dac5f01f7fbe86788b12c9d1 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Mon, 11 Nov 2024 14:36:27 +0200 Subject: [PATCH 12/26] Only run the big tests when --all-features is given --- tests/gc.rs | 1 + tests/sync.rs | 1 + tests/util.rs | 1 + 3 files changed, 3 insertions(+) diff --git a/tests/gc.rs b/tests/gc.rs index c521c1e..02e5f91 100644 --- a/tests/gc.rs +++ b/tests/gc.rs @@ -1,3 +1,4 @@ +#![cfg(feature = "rpc")] use std::{ io::{Cursor, Write}, time::Duration, diff --git a/tests/sync.rs b/tests/sync.rs index f65b2f7..aa6bb59 100644 --- a/tests/sync.rs +++ b/tests/sync.rs @@ -1,3 +1,4 @@ +#![cfg(feature = "rpc")] use std::{ collections::HashMap, future::Future, diff --git a/tests/util.rs b/tests/util.rs index 68f3e81..1cf3ded 100644 --- a/tests/util.rs +++ b/tests/util.rs @@ -1,3 +1,4 @@ +#![cfg(feature = "rpc")] #![allow(dead_code)] use std::{ collections::BTreeSet, From a2a8c64f5f102cd7ea762323f87c1b62599b6f69 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Mon, 11 Nov 2024 15:23:58 +0200 Subject: [PATCH 13/26] more tests - will it ever end --- Cargo.lock | 90 +++++++++++++++ Cargo.toml | 2 + src/rpc/client.rs | 275 +------------------------------------------- tests/client.rs | 282 ++++++++++++++++++++++++++++++++++++++++++++++ tests/gc.rs | 3 +- 5 files changed, 377 insertions(+), 275 deletions(-) create mode 100644 tests/client.rs diff --git a/Cargo.lock b/Cargo.lock index cca798e..ffa716e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -424,6 +424,37 @@ dependencies = [ "serde", ] +[[package]] +name = "camino" +version = "1.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b96ec4966b5813e2c0507c1f86115c8c5abaadc3980879c3424042a02fd1ad3" +dependencies = [ + "serde", +] + +[[package]] +name = "cargo-platform" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24b1f0365a6c6bb4020cd05806fd0d33c44d38046b8bd7f0e40814b9763cabfc" +dependencies = [ + "serde", +] + +[[package]] +name = "cargo_metadata" +version = "0.14.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4acbb09d9ee8e23699b9634375c72795d095bf268439da88562cf9b501f181fa" +dependencies = [ + "camino", + "cargo-platform", + "semver", + "serde", + "serde_json", +] + [[package]] name = "cc" version = "1.1.36" @@ -2023,6 +2054,7 @@ dependencies = [ "iroh-blake3", "iroh-blobs", "iroh-gossip", + "iroh-io", "iroh-metrics", "iroh-net", "iroh-router", @@ -2047,6 +2079,7 @@ dependencies = [ "strum", "tempfile", "test-strategy", + "testdir", "testresult", "thiserror", "tokio", @@ -2678,6 +2711,15 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "38bf9645c8b145698bb0b18a4637dcacbc421ea49bef2317e4fd8065a387cf21" +[[package]] +name = "ntapi" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8a3895c6391c39d7fe7ebc444a87eb2991b2a0bc718fdabd071eec617fc68e4" +dependencies = [ + "winapi", +] + [[package]] name = "nu-ansi-term" version = "0.46.0" @@ -3988,6 +4030,9 @@ name = "semver" version = "1.0.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" +dependencies = [ + "serde", +] [[package]] name = "serde" @@ -4460,6 +4505,20 @@ dependencies = [ "syn 2.0.87", ] +[[package]] +name = "sysinfo" +version = "0.26.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c18a6156d1f27a9592ee18c1a846ca8dd5c258b7179fc193ae87c74ebb666f5" +dependencies = [ + "cfg-if", + "core-foundation-sys", + "libc", + "ntapi", + "once_cell", + "winapi", +] + [[package]] name = "system-configuration" version = "0.6.1" @@ -4506,6 +4565,20 @@ dependencies = [ "syn 2.0.87", ] +[[package]] +name = "testdir" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee79e927b64d193f5abb60d20a0eb56be0ee5a242fdeb8ce3bf054177006de52" +dependencies = [ + "anyhow", + "backtrace", + "cargo_metadata", + "once_cell", + "sysinfo", + "whoami", +] + [[package]] name = "testresult" version = "0.4.1" @@ -5066,6 +5139,12 @@ version = "0.11.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" +[[package]] +name = "wasite" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" + [[package]] name = "wasm-bindgen" version = "0.2.95" @@ -5164,6 +5243,17 @@ dependencies = [ "rustls-pki-types", ] +[[package]] +name = "whoami" +version = "1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "372d5b87f58ec45c384ba03563b03544dc5fadc3983e434b286913f5b4a9bb6d" +dependencies = [ + "redox_syscall", + "wasite", + "web-sys", +] + [[package]] name = "widestring" version = "1.1.0" diff --git a/Cargo.toml b/Cargo.toml index 41fb628..12edbf0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -76,6 +76,8 @@ tracing-subscriber = { version = "0.3.18", features = ["env-filter"] } parking_lot = "0.12.3" testresult = "0.4.1" nested_enum_utils = "0.1.0" +iroh-io = "0.6.1" +testdir = "0.9.1" [features] default = ["net", "metrics", "engine", "rpc", "test-utils"] diff --git a/src/rpc/client.rs b/src/rpc/client.rs index 50d3746..92a2e43 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -791,9 +791,9 @@ impl ExportFileProgress { #[derive(Debug, Clone, PartialEq, Eq)] pub struct ExportFileOutcome { /// The size of the entry - size: u64, + pub size: u64, /// The path to which the entry was saved - path: PathBuf, + pub path: PathBuf, } impl Stream for ExportFileProgress { @@ -936,275 +936,4 @@ mod tests { Ok(()) } - - // /// Test that closing a doc does not close other instances. - // #[tokio::test] - // async fn test_doc_close() -> Result<()> { - // let _guard = iroh_test::logging::setup(); - - // let node = iroh::node::Node::memory().enable_docs().spawn().await?; - // let author = node.authors().default().await?; - // // open doc two times - // let doc1 = node.docs().create().await?; - // let doc2 = node.docs().open(doc1.id()).await?.expect("doc to exist"); - // // close doc1 instance - // doc1.close().await?; - // // operations on doc1 now fail. - // assert!(doc1.set_bytes(author, "foo", "bar").await.is_err()); - // // dropping doc1 will close the doc if not already closed - // // wait a bit because the close-on-drop spawns a task for which we cannot track completion. - // drop(doc1); - // tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; - - // // operations on doc2 still succeed - // doc2.set_bytes(author, "foo", "bar").await?; - // Ok(()) - // } - - // #[tokio::test] - // async fn test_doc_import_export() -> Result<()> { - // let _guard = iroh_test::logging::setup(); - - // let node = iroh::node::Node::memory().enable_docs().spawn().await?; - - // // create temp file - // let temp_dir = tempfile::tempdir().context("tempdir")?; - - // let in_root = temp_dir.path().join("in"); - // tokio::fs::create_dir_all(in_root.clone()) - // .await - // .context("create dir all")?; - // let out_root = temp_dir.path().join("out"); - - // let path = in_root.join("test"); - - // let size = 100; - // let mut buf = vec![0u8; size]; - // rand::thread_rng().fill_bytes(&mut buf); - // let mut file = tokio::fs::File::create(path.clone()) - // .await - // .context("create file")?; - // file.write_all(&buf.clone()).await.context("write_all")?; - // file.flush().await.context("flush")?; - - // // create doc & author - // let client = node.rpc_client().clone(); - // let docs_client = Client::from_service(client); - // let doc = docs_client.create().await.context("doc create")?; - // let author = client.authors().create().await.context("author create")?; - - // // import file - // let import_outcome = doc - // .import_file( - // author, - // iroh::util::fs::path_to_key(path.clone(), None, Some(in_root))?, - // path, - // true, - // ) - // .await - // .context("import file")? - // .finish() - // .await - // .context("import finish")?; - - // // export file - // let entry = doc - // .get_one(Query::author(author).key_exact(import_outcome.key)) - // .await - // .context("get one")? - // .unwrap(); - // let key = entry.key().to_vec(); - // let export_outcome = doc - // .export_file( - // entry, - // iroh::util::fs::key_to_path(key, None, Some(out_root))?, - // ExportMode::Copy, - // ) - // .await - // .context("export file")? - // .finish() - // .await - // .context("export finish")?; - - // let got_bytes = tokio::fs::read(export_outcome.path) - // .await - // .context("tokio read")?; - // assert_eq!(buf, got_bytes); - - // Ok(()) - // } - - // #[tokio::test] - // async fn test_authors() -> Result<()> { - // let node = Node::memory().enable_docs().spawn().await?; - - // // default author always exists - // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; - // assert_eq!(authors.len(), 1); - // let default_author = node.authors().default().await?; - // assert_eq!(authors, vec![default_author]); - - // let author_id = node.authors().create().await?; - - // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; - // assert_eq!(authors.len(), 2); - - // let author = node - // .authors() - // .export(author_id) - // .await? - // .expect("should have author"); - // node.authors().delete(author_id).await?; - // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; - // assert_eq!(authors.len(), 1); - - // node.authors().import(author).await?; - - // let authors: Vec<_> = node.authors().list().await?.try_collect().await?; - // assert_eq!(authors.len(), 2); - - // assert!(node.authors().default().await? != author_id); - // node.authors().set_default(author_id).await?; - // assert_eq!(node.authors().default().await?, author_id); - - // Ok(()) - // } - - // #[tokio::test] - // async fn test_default_author_memory() -> Result<()> { - // let iroh = Node::memory().enable_docs().spawn().await?; - // let author = iroh.authors().default().await?; - // assert!(iroh.authors().export(author).await?.is_some()); - // assert!(iroh.authors().delete(author).await.is_err()); - // Ok(()) - // } - - // #[cfg(feature = "fs-store")] - // #[tokio::test] - // async fn test_default_author_persist() -> Result<()> { - // use crate::util::path::IrohPaths; - - // let _guard = iroh_test::logging::setup(); - - // let iroh_root_dir = tempfile::TempDir::new().unwrap(); - // let iroh_root = iroh_root_dir.path(); - - // // check that the default author exists and cannot be deleted. - // let default_author = { - // let iroh = Node::persistent(iroh_root) - // .await - // .unwrap() - // .enable_docs() - // .spawn() - // .await - // .unwrap(); - // let author = iroh.authors().default().await.unwrap(); - // assert!(iroh.authors().export(author).await.unwrap().is_some()); - // assert!(iroh.authors().delete(author).await.is_err()); - // iroh.shutdown().await.unwrap(); - // author - // }; - - // // check that the default author is persisted across restarts. - // { - // let iroh = Node::persistent(iroh_root) - // .await - // .unwrap() - // .enable_docs() - // .spawn() - // .await - // .unwrap(); - // let author = iroh.authors().default().await.unwrap(); - // assert_eq!(author, default_author); - // assert!(iroh.authors().export(author).await.unwrap().is_some()); - // assert!(iroh.authors().delete(author).await.is_err()); - // iroh.shutdown().await.unwrap(); - // }; - - // // check that a new default author is created if the default author file is deleted - // // manually. - // let default_author = { - // tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) - // .await - // .unwrap(); - // let iroh = Node::persistent(iroh_root) - // .await - // .unwrap() - // .enable_docs() - // .spawn() - // .await - // .unwrap(); - // let author = iroh.authors().default().await.unwrap(); - // assert!(author != default_author); - // assert!(iroh.authors().export(author).await.unwrap().is_some()); - // assert!(iroh.authors().delete(author).await.is_err()); - // iroh.shutdown().await.unwrap(); - // author - // }; - - // // check that the node fails to start if the default author is missing from the docs store. - // { - // let mut docs_store = iroh_docs::store::fs::Store::persistent( - // IrohPaths::DocsDatabase.with_root(iroh_root), - // ) - // .unwrap(); - // docs_store.delete_author(default_author).unwrap(); - // docs_store.flush().unwrap(); - // drop(docs_store); - // let iroh = Node::persistent(iroh_root) - // .await - // .unwrap() - // .enable_docs() - // .spawn() - // .await; - // assert!(iroh.is_err()); - - // // somehow the blob store is not shutdown correctly (yet?) on macos. - // // so we give it some time until we find a proper fix. - // #[cfg(target_os = "macos")] - // tokio::time::sleep(Duration::from_secs(1)).await; - - // tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) - // .await - // .unwrap(); - // drop(iroh); - // let iroh = Node::persistent(iroh_root) - // .await - // .unwrap() - // .enable_docs() - // .spawn() - // .await; - // assert!(iroh.is_ok()); - // iroh.unwrap().shutdown().await.unwrap(); - // } - - // // check that the default author can be set manually and is persisted. - // let default_author = { - // let iroh = Node::persistent(iroh_root) - // .await - // .unwrap() - // .enable_docs() - // .spawn() - // .await - // .unwrap(); - // let author = iroh.authors().create().await.unwrap(); - // iroh.authors().set_default(author).await.unwrap(); - // assert_eq!(iroh.authors().default().await.unwrap(), author); - // iroh.shutdown().await.unwrap(); - // author - // }; - // { - // let iroh = Node::persistent(iroh_root) - // .await - // .unwrap() - // .enable_docs() - // .spawn() - // .await - // .unwrap(); - // assert_eq!(iroh.authors().default().await.unwrap(), default_author); - // iroh.shutdown().await.unwrap(); - // } - - // Ok(()) - // } } diff --git a/tests/client.rs b/tests/client.rs new file mode 100644 index 0000000..aaae906 --- /dev/null +++ b/tests/client.rs @@ -0,0 +1,282 @@ +use anyhow::{Context, Result}; +use futures_util::TryStreamExt; +use iroh_blobs::{ + store::ExportMode, + util::fs::{key_to_path, path_to_key}, +}; +use iroh_docs::store::Query; +use rand::RngCore; +use tokio::io::AsyncWriteExt; +use util::Node; + +mod util; + +/// Test that closing a doc does not close other instances. +#[tokio::test] +async fn test_doc_close() -> Result<()> { + let _guard = iroh_test::logging::setup(); + + let node = Node::memory().spawn().await?; + let author = node.docs().author_default().await?; + // open doc two times + let doc1 = node.docs().create().await?; + let doc2 = node.docs().open(doc1.id()).await?.expect("doc to exist"); + // close doc1 instance + doc1.close().await?; + // operations on doc1 now fail. + assert!(doc1.set_bytes(author, "foo", "bar").await.is_err()); + // dropping doc1 will close the doc if not already closed + // wait a bit because the close-on-drop spawns a task for which we cannot track completion. + drop(doc1); + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + + // operations on doc2 still succeed + doc2.set_bytes(author, "foo", "bar").await?; + Ok(()) +} + +#[tokio::test] +async fn test_doc_import_export() -> Result<()> { + let _guard = iroh_test::logging::setup(); + + let node = Node::memory().spawn().await?; + + // create temp file + let temp_dir = tempfile::tempdir().context("tempdir")?; + + let in_root = temp_dir.path().join("in"); + tokio::fs::create_dir_all(in_root.clone()) + .await + .context("create dir all")?; + let out_root = temp_dir.path().join("out"); + + let path = in_root.join("test"); + + let size = 100; + let mut buf = vec![0u8; size]; + rand::thread_rng().fill_bytes(&mut buf); + let mut file = tokio::fs::File::create(path.clone()) + .await + .context("create file")?; + file.write_all(&buf.clone()).await.context("write_all")?; + file.flush().await.context("flush")?; + + // create doc & author + let client = node.client(); + let docs_client = client.docs(); + let doc = docs_client.create().await.context("doc create")?; + let author = docs_client.author_create().await.context("author create")?; + + // import file + let import_outcome = doc + .import_file( + author, + path_to_key(path.clone(), None, Some(in_root))?, + path, + true, + ) + .await + .context("import file")? + .finish() + .await + .context("import finish")?; + + // export file + let entry = doc + .get_one(Query::author(author).key_exact(import_outcome.key)) + .await + .context("get one")? + .unwrap(); + let key = entry.key().to_vec(); + let export_outcome = doc + .export_file( + entry, + key_to_path(key, None, Some(out_root))?, + ExportMode::Copy, + ) + .await + .context("export file")? + .finish() + .await + .context("export finish")?; + + let got_bytes = tokio::fs::read(export_outcome.path) + .await + .context("tokio read")?; + assert_eq!(buf, got_bytes); + + Ok(()) +} + +#[tokio::test] +async fn test_authors() -> Result<()> { + let node = Node::memory().spawn().await?; + + // default author always exists + let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + assert_eq!(authors.len(), 1); + let default_author = node.docs().author_default().await?; + assert_eq!(authors, vec![default_author]); + + let author_id = node.docs().author_create().await?; + + let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + assert_eq!(authors.len(), 2); + + let author = node + .docs() + .author_export(author_id) + .await? + .expect("should have author"); + node.docs().author_delete(author_id).await?; + let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + assert_eq!(authors.len(), 1); + + node.docs().author_import(author).await?; + + let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + assert_eq!(authors.len(), 2); + + assert!(node.docs().author_default().await? != author_id); + node.docs().author_set_default(author_id).await?; + assert_eq!(node.docs().author_default().await?, author_id); + + Ok(()) +} + +#[tokio::test] +async fn test_default_author_memory() -> Result<()> { + let iroh = Node::memory().spawn().await?; + let author = iroh.docs().author_default().await?; + assert!(iroh.docs().author_export(author).await?.is_some()); + assert!(iroh.docs().author_delete(author).await.is_err()); + Ok(()) +} + +#[cfg(feature = "fs-store")] +#[tokio::test] +async fn test_default_author_persist() -> Result<()> { + use crate::util::path::IrohPaths; + + let _guard = iroh_test::logging::setup(); + + let iroh_root_dir = tempfile::TempDir::new().unwrap(); + let iroh_root = iroh_root_dir.path(); + + // check that the default author exists and cannot be deleted. + let default_author = { + let iroh = Node::persistent(iroh_root) + .await + .unwrap() + .enable_docs() + .spawn() + .await + .unwrap(); + let author = iroh.authors().default().await.unwrap(); + assert!(iroh.authors().export(author).await.unwrap().is_some()); + assert!(iroh.authors().delete(author).await.is_err()); + iroh.shutdown().await.unwrap(); + author + }; + + // check that the default author is persisted across restarts. + { + let iroh = Node::persistent(iroh_root) + .await + .unwrap() + .enable_docs() + .spawn() + .await + .unwrap(); + let author = iroh.authors().default().await.unwrap(); + assert_eq!(author, default_author); + assert!(iroh.authors().export(author).await.unwrap().is_some()); + assert!(iroh.authors().delete(author).await.is_err()); + iroh.shutdown().await.unwrap(); + }; + + // check that a new default author is created if the default author file is deleted + // manually. + let default_author = { + tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) + .await + .unwrap(); + let iroh = Node::persistent(iroh_root) + .await + .unwrap() + .enable_docs() + .spawn() + .await + .unwrap(); + let author = iroh.authors().default().await.unwrap(); + assert!(author != default_author); + assert!(iroh.authors().export(author).await.unwrap().is_some()); + assert!(iroh.authors().delete(author).await.is_err()); + iroh.shutdown().await.unwrap(); + author + }; + + // check that the node fails to start if the default author is missing from the docs store. + { + let mut docs_store = + iroh_docs::store::fs::Store::persistent(IrohPaths::DocsDatabase.with_root(iroh_root)) + .unwrap(); + docs_store.delete_author(default_author).unwrap(); + docs_store.flush().unwrap(); + drop(docs_store); + let iroh = Node::persistent(iroh_root) + .await + .unwrap() + .enable_docs() + .spawn() + .await; + assert!(iroh.is_err()); + + // somehow the blob store is not shutdown correctly (yet?) on macos. + // so we give it some time until we find a proper fix. + #[cfg(target_os = "macos")] + tokio::time::sleep(Duration::from_secs(1)).await; + + tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) + .await + .unwrap(); + drop(iroh); + let iroh = Node::persistent(iroh_root) + .await + .unwrap() + .enable_docs() + .spawn() + .await; + assert!(iroh.is_ok()); + iroh.unwrap().shutdown().await.unwrap(); + } + + // check that the default author can be set manually and is persisted. + let default_author = { + let iroh = Node::persistent(iroh_root) + .await + .unwrap() + .enable_docs() + .spawn() + .await + .unwrap(); + let author = iroh.authors().create().await.unwrap(); + iroh.authors().set_default(author).await.unwrap(); + assert_eq!(iroh.authors().default().await.unwrap(), author); + iroh.shutdown().await.unwrap(); + author + }; + { + let iroh = Node::persistent(iroh_root) + .await + .unwrap() + .enable_docs() + .spawn() + .await + .unwrap(); + assert_eq!(iroh.authors().default().await.unwrap(), default_author); + iroh.shutdown().await.unwrap(); + } + + Ok(()) +} diff --git a/tests/gc.rs b/tests/gc.rs index 02e5f91..76908b6 100644 --- a/tests/gc.rs +++ b/tests/gc.rs @@ -176,7 +176,6 @@ async fn gc_hashseq_impl() -> Result<()> { Ok(()) } -#[cfg(feature = "fs-store")] mod file { use std::{io, path::PathBuf}; @@ -235,7 +234,7 @@ mod file { let (node, _) = wrap_in_node(bao_store.clone(), Duration::from_secs(10)).await; let client = node.client(); let doc = client.docs().create().await?; - let author = client.authors().create().await?; + let author = client.docs().author_create().await?; let temp_path = dir.join("temp"); tokio::fs::create_dir_all(&temp_path).await?; let mut to_import = Vec::new(); From 7fc22a088eccaed180630a6b78a3fa91a89d8a4e Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 12:01:15 +0200 Subject: [PATCH 14/26] Fix last remaining test --- tests/client.rs | 99 +++++++++++++++---------------------------------- tests/gc.rs | 59 +++++++++++++++++++---------- tests/sync.rs | 2 - tests/util.rs | 99 +++++++++++++++++++++++++++++++++++-------------- 4 files changed, 140 insertions(+), 119 deletions(-) diff --git a/tests/client.rs b/tests/client.rs index aaae906..83fdae0 100644 --- a/tests/client.rs +++ b/tests/client.rs @@ -1,3 +1,5 @@ +use std::time::Duration; + use anyhow::{Context, Result}; use futures_util::TryStreamExt; use iroh_blobs::{ @@ -6,6 +8,7 @@ use iroh_blobs::{ }; use iroh_docs::store::Query; use rand::RngCore; +use testresult::TestResult; use tokio::io::AsyncWriteExt; use util::Node; @@ -153,11 +156,8 @@ async fn test_default_author_memory() -> Result<()> { Ok(()) } -#[cfg(feature = "fs-store")] #[tokio::test] -async fn test_default_author_persist() -> Result<()> { - use crate::util::path::IrohPaths; - +async fn test_default_author_persist() -> TestResult<()> { let _guard = iroh_test::logging::setup(); let iroh_root_dir = tempfile::TempDir::new().unwrap(); @@ -165,53 +165,35 @@ async fn test_default_author_persist() -> Result<()> { // check that the default author exists and cannot be deleted. let default_author = { - let iroh = Node::persistent(iroh_root) - .await - .unwrap() - .enable_docs() - .spawn() - .await - .unwrap(); - let author = iroh.authors().default().await.unwrap(); - assert!(iroh.authors().export(author).await.unwrap().is_some()); - assert!(iroh.authors().delete(author).await.is_err()); + let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); + let author = iroh.docs().author_default().await.unwrap(); + assert!(iroh.docs().author_export(author).await.unwrap().is_some()); + assert!(iroh.docs().author_delete(author).await.is_err()); iroh.shutdown().await.unwrap(); author }; // check that the default author is persisted across restarts. { - let iroh = Node::persistent(iroh_root) - .await - .unwrap() - .enable_docs() - .spawn() - .await - .unwrap(); - let author = iroh.authors().default().await.unwrap(); + let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); + let author = iroh.docs().author_default().await.unwrap(); assert_eq!(author, default_author); - assert!(iroh.authors().export(author).await.unwrap().is_some()); - assert!(iroh.authors().delete(author).await.is_err()); + assert!(iroh.docs().author_export(author).await.unwrap().is_some()); + assert!(iroh.docs().author_delete(author).await.is_err()); iroh.shutdown().await.unwrap(); }; // check that a new default author is created if the default author file is deleted // manually. let default_author = { - tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) + tokio::fs::remove_file(iroh_root.join("default-author")) .await .unwrap(); - let iroh = Node::persistent(iroh_root) - .await - .unwrap() - .enable_docs() - .spawn() - .await - .unwrap(); - let author = iroh.authors().default().await.unwrap(); + let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); + let author = iroh.docs().author_default().await.unwrap(); assert!(author != default_author); - assert!(iroh.authors().export(author).await.unwrap().is_some()); - assert!(iroh.authors().delete(author).await.is_err()); + assert!(iroh.docs().author_export(author).await.unwrap().is_some()); + assert!(iroh.docs().author_delete(author).await.is_err()); iroh.shutdown().await.unwrap(); author }; @@ -219,17 +201,11 @@ async fn test_default_author_persist() -> Result<()> { // check that the node fails to start if the default author is missing from the docs store. { let mut docs_store = - iroh_docs::store::fs::Store::persistent(IrohPaths::DocsDatabase.with_root(iroh_root)) - .unwrap(); + iroh_docs::store::fs::Store::persistent(iroh_root.join("docs.redb")).unwrap(); docs_store.delete_author(default_author).unwrap(); docs_store.flush().unwrap(); drop(docs_store); - let iroh = Node::persistent(iroh_root) - .await - .unwrap() - .enable_docs() - .spawn() - .await; + let iroh = Node::persistent(iroh_root).spawn().await; assert!(iroh.is_err()); // somehow the blob store is not shutdown correctly (yet?) on macos. @@ -237,44 +213,29 @@ async fn test_default_author_persist() -> Result<()> { #[cfg(target_os = "macos")] tokio::time::sleep(Duration::from_secs(1)).await; - tokio::fs::remove_file(IrohPaths::DefaultAuthor.with_root(iroh_root)) + tokio::fs::remove_file(iroh_root.join("default-author")) .await .unwrap(); drop(iroh); - let iroh = Node::persistent(iroh_root) - .await - .unwrap() - .enable_docs() - .spawn() - .await; - assert!(iroh.is_ok()); + let iroh = Node::persistent(iroh_root).spawn().await; + if let Err(cause) = iroh.as_ref() { + panic!("failed to start node: {:?}", cause); + } iroh.unwrap().shutdown().await.unwrap(); } // check that the default author can be set manually and is persisted. let default_author = { - let iroh = Node::persistent(iroh_root) - .await - .unwrap() - .enable_docs() - .spawn() - .await - .unwrap(); - let author = iroh.authors().create().await.unwrap(); - iroh.authors().set_default(author).await.unwrap(); - assert_eq!(iroh.authors().default().await.unwrap(), author); + let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); + let author = iroh.docs().author_create().await.unwrap(); + iroh.docs().author_set_default(author).await.unwrap(); + assert_eq!(iroh.docs().author_default().await.unwrap(), author); iroh.shutdown().await.unwrap(); author }; { - let iroh = Node::persistent(iroh_root) - .await - .unwrap() - .enable_docs() - .spawn() - .await - .unwrap(); - assert_eq!(iroh.authors().default().await.unwrap(), default_author); + let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); + assert_eq!(iroh.docs().author_default().await.unwrap(), default_author); iroh.shutdown().await.unwrap(); } diff --git a/tests/gc.rs b/tests/gc.rs index 76908b6..94857e3 100644 --- a/tests/gc.rs +++ b/tests/gc.rs @@ -1,6 +1,7 @@ #![cfg(feature = "rpc")] use std::{ io::{Cursor, Write}, + path::PathBuf, time::Duration, }; @@ -14,7 +15,7 @@ use iroh_blobs::{ BlobFormat, HashAndFormat, IROH_BLOCK_SIZE, }; use rand::RngCore; -use util::{Builder, Node}; +use util::Node; mod util; @@ -39,15 +40,34 @@ pub fn simulate_remote(data: &[u8]) -> (blake3::Hash, Cursor) { } /// Wrap a bao store in a node that has gc enabled. -async fn wrap_in_node( - bao_store: S, +async fn mem_node( gc_period: Duration, -) -> (Node, async_channel::Receiver<()>) -where - S: iroh_blobs::store::Store, -{ +) -> ( + Node, + async_channel::Receiver<()>, +) { let (gc_send, gc_recv) = async_channel::unbounded(); - let node = Builder::new(bao_store) + let node = Node::memory() + .gc_interval(Some(gc_period)) + .register_gc_done_cb(Box::new(move || { + gc_send.send_blocking(()).ok(); + })) + .spawn() + .await + .unwrap(); + (node, gc_recv) +} + +/// Wrap a bao store in a node that has gc enabled. +async fn persistent_node( + path: PathBuf, + gc_period: Duration, +) -> ( + Node, + async_channel::Receiver<()>, +) { + let (gc_send, gc_recv) = async_channel::unbounded(); + let node = Node::persistent(path) .gc_interval(Some(gc_period)) .register_gc_done_cb(Box::new(move || { gc_send.send_blocking(()).ok(); @@ -63,9 +83,9 @@ async fn gc_test_node() -> ( iroh_blobs::store::mem::Store, async_channel::Receiver<()>, ) { - let bao_store = iroh_blobs::store::mem::Store::new(); - let (node, gc_recv) = wrap_in_node(bao_store.clone(), Duration::from_millis(500)).await; - (node, bao_store, gc_recv) + let (node, gc_recv) = mem_node(Duration::from_millis(500)).await; + let store = node.blob_store().clone(); + (node, store, gc_recv) } async fn step(evs: &async_channel::Receiver<()>) { @@ -230,8 +250,8 @@ mod file { async fn redb_doc_import_stress() -> Result<()> { let _ = tracing_subscriber::fmt::try_init(); let dir = testdir!(); - let bao_store = iroh_blobs::store::fs::Store::load(dir.join("store")).await?; - let (node, _) = wrap_in_node(bao_store.clone(), Duration::from_secs(10)).await; + let (node, _) = persistent_node(dir.join("store"), Duration::from_secs(10)).await; + let bao_store = node.blob_store().clone(); let client = node.client(); let doc = client.docs().create().await?; let author = client.docs().author_create().await?; @@ -272,9 +292,8 @@ mod file { let dir = testdir!(); let path = data_path(dir.clone()); let outboard_path = outboard_path(dir.clone()); - - let bao_store = iroh_blobs::store::fs::Store::load(dir.clone()).await?; - let (node, evs) = wrap_in_node(bao_store.clone(), Duration::from_millis(100)).await; + let (node, evs) = persistent_node(dir.clone(), Duration::from_millis(100)).await; + let bao_store = node.blob_store().clone(); let data1 = create_test_data(10000000); let tt1 = bao_store .import_bytes(data1.clone(), BlobFormat::Raw) @@ -434,8 +453,8 @@ mod file { let path = data_path(dir.clone()); let outboard_path = outboard_path(dir.clone()); - let bao_store = iroh_blobs::store::fs::Store::load(dir.clone()).await?; - let (node, evs) = wrap_in_node(bao_store.clone(), Duration::from_millis(10)).await; + let (node, evs) = persistent_node(dir.clone(), Duration::from_millis(10)).await; + let bao_store = node.blob_store().clone(); let data1: Bytes = create_test_data(10000000); let (_entry, tt1) = simulate_download_partial(&bao_store, data1.clone()).await?; @@ -465,8 +484,8 @@ mod file { let _ = tracing_subscriber::fmt::try_init(); let dir = testdir!(); - let bao_store = iroh_blobs::store::fs::Store::load(dir.clone()).await?; - let (node, evs) = wrap_in_node(bao_store.clone(), Duration::from_secs(1)).await; + let (node, evs) = persistent_node(dir.clone(), Duration::from_secs(1)).await; + let bao_store = node.blob_store().clone(); let mut deleted = Vec::new(); let mut live = Vec::new(); diff --git a/tests/sync.rs b/tests/sync.rs index aa6bb59..3cc02b8 100644 --- a/tests/sync.rs +++ b/tests/sync.rs @@ -595,7 +595,6 @@ async fn sync_restart_node() -> Result<()> { let secret_key_1 = SecretKey::generate_with_rng(&mut rng); let node1 = Node::persistent(&node1_dir) - .await? .secret_key(secret_key_1.clone()) .insecure_skip_relay_cert_verify(true) .relay_mode(RelayMode::Custom(relay_map.clone())) @@ -662,7 +661,6 @@ async fn sync_restart_node() -> Result<()> { info!(me = id1.fmt_short(), "node1 respawn"); let node1 = Node::persistent(&node1_dir) - .await? .secret_key(secret_key_1.clone()) .insecure_skip_relay_cert_verify(true) .relay_mode(RelayMode::Custom(relay_map.clone())) diff --git a/tests/util.rs b/tests/util.rs index 1cf3ded..e42d277 100644 --- a/tests/util.rs +++ b/tests/util.rs @@ -2,9 +2,10 @@ #![allow(dead_code)] use std::{ collections::BTreeSet, + marker::PhantomData, net::{Ipv4Addr, Ipv6Addr, SocketAddrV4, SocketAddrV6}, ops::Deref, - path::Path, + path::{Path, PathBuf}, sync::Arc, }; @@ -35,10 +36,10 @@ pub const DEFAULT_BIND_ADDR_V6: SocketAddrV6 = pub struct Node { router: iroh_router::Router, client: Client, - _store: S, - _local_pool: LocalPool, - _rpc_task: AbortOnDropHandle<()>, - _gc_task: Option>, + store: S, + local_pool: LocalPool, + rpc_task: AbortOnDropHandle<()>, + gc_task: Option>, } impl Deref for Node { @@ -97,7 +98,7 @@ impl Client { /// An iroh node builder #[derive(derive_more::Debug)] pub struct Builder { - store: S, + path: Option, secret_key: Option, relay_mode: RelayMode, dns_resolver: Option, @@ -107,12 +108,12 @@ pub struct Builder { register_gc_done_cb: Option>, insecure_skip_relay_cert_verify: bool, bind_random_port: bool, + _p: PhantomData, } impl Builder { /// Spawns the node - pub async fn spawn(self) -> anyhow::Result> { - let store = self.store.clone(); + async fn spawn0(self, store: S) -> anyhow::Result> { let mut addr_v4 = DEFAULT_BIND_ADDR_V4; let mut addr_v6 = DEFAULT_BIND_ADDR_V6; if self.bind_random_port { @@ -151,16 +152,33 @@ impl Builder { Default::default(), &addr.info, ); - let docs = iroh_docs::engine::Engine::spawn( + let replica_store = match self.path { + Some(ref path) => iroh_docs::store::Store::persistent(path.join("docs.redb"))?, + None => iroh_docs::store::Store::memory(), + }; + let author_store = match self.path { + Some(ref path) => { + iroh_docs::engine::DefaultAuthorStorage::Persistent(path.join("default-author")) + } + None => iroh_docs::engine::DefaultAuthorStorage::Mem, + }; + let docs = match iroh_docs::engine::Engine::spawn( endpoint, gossip.clone(), - iroh_docs::store::Store::memory(), + replica_store, store.clone(), downloader, - iroh_docs::engine::DefaultAuthorStorage::Mem, + author_store, local_pool.handle().clone(), ) - .await?; + .await + { + Ok(docs) => docs, + Err(err) => { + store.shutdown().await; + return Err(err); + } + }; router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), blobs.clone()); router = router.accept(iroh_docs::net::DOCS_ALPN.to_vec(), Arc::new(docs.clone())); router = router.accept( @@ -209,7 +227,7 @@ impl Builder { let client = quic_rpc::RpcClient::new(controller); let _gc_task = if let Some(period) = self.gc_interval { - let store = self.store.clone(); + let store = store.clone(); let local_pool = local_pool.clone(); let docs = docs.clone(); let protected_cb = move || { @@ -254,10 +272,10 @@ impl Builder { Ok(Node { router, client, - _store: self.store, - _rpc_task: AbortOnDropHandle::new(rpc_task), - _gc_task, - _local_pool: local_pool, + store, + rpc_task: AbortOnDropHandle::new(rpc_task), + gc_task: _gc_task, + local_pool, }) } @@ -301,9 +319,9 @@ impl Builder { self } - pub fn new(store: S) -> Self { + fn new(path: Option) -> Self { Self { - store, + path, secret_key: None, relay_mode: RelayMode::Default, gc_interval: None, @@ -312,6 +330,7 @@ impl Builder { dns_resolver: None, node_discovery: None, register_gc_done_cb: None, + _p: PhantomData, } } } @@ -319,18 +338,31 @@ impl Builder { impl Node { /// Creates a new node with memory storage pub fn memory() -> Builder { - Builder::new(iroh_blobs::store::mem::Store::new()) + Builder::new(None) + } +} + +impl Builder { + /// Spawns the node + pub async fn spawn(self) -> anyhow::Result> { + let store = iroh_blobs::store::mem::Store::new(); + self.spawn0(store).await } } impl Node { /// Creates a new node with persistent storage - pub async fn persistent( - path: impl AsRef, - ) -> anyhow::Result> { - Ok(Builder::new( - iroh_blobs::store::fs::Store::load(path).await?, - )) + pub fn persistent(path: impl AsRef) -> Builder { + let path = Some(path.as_ref().to_owned()); + Builder::new(path) + } +} + +impl Builder { + /// Spawns the node + pub async fn spawn(self) -> anyhow::Result> { + let store = iroh_blobs::store::fs::Store::load(self.path.clone().unwrap()).await?; + self.spawn0(store).await } } @@ -340,9 +372,20 @@ impl Node { self.router.endpoint().node_id() } + /// Returns the blob store + pub fn blob_store(&self) -> &S { + &self.store + } + /// Shuts down the node - pub async fn shutdown(self) -> anyhow::Result<()> { - self.router.shutdown().await + pub async fn shutdown(mut self) -> anyhow::Result<()> { + self.router.shutdown().await?; + self.local_pool.shutdown().await; + self.rpc_task.abort(); + if let Some(mut task) = self.gc_task.take() { + task.abort(); + } + Ok(()) } /// Returns the client From dc9de49412e0f674b831f7d4547617f245946f6a Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 12:12:14 +0200 Subject: [PATCH 15/26] clippy --- tests/sync.rs | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/sync.rs b/tests/sync.rs index 3cc02b8..6b20f3b 100644 --- a/tests/sync.rs +++ b/tests/sync.rs @@ -88,7 +88,7 @@ async fn sync_simple() -> Result<()> { let hash0 = doc0 .set_bytes(author0, b"k1".to_vec(), b"v1".to_vec()) .await?; - assert_latest(&blobs0, &doc0, b"k1", b"v1").await; + assert_latest(blobs0, &doc0, b"k1", b"v1").await; let ticket = doc0 .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) .await?; @@ -113,7 +113,7 @@ async fn sync_simple() -> Result<()> { ], ) .await; - assert_latest(&blobs1, &doc1, b"k1", b"v1").await; + assert_latest(blobs1, &doc1, b"k1", b"v1").await; info!("node0: assert 2 events"); assert_next( @@ -271,7 +271,7 @@ async fn sync_full_basic() -> Result<()> { matches!(&e, LiveEvent::InsertLocal { entry } if entry.content_hash() == hash0), "expected LiveEvent::InsertLocal but got {e:?}", ); - assert_latest(&blobs0, &doc0, key0, value0).await; + assert_latest(blobs0, &doc0, key0, value0).await; let ticket = doc0 .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) .await?; @@ -316,7 +316,7 @@ async fn sync_full_basic() -> Result<()> { let hash1 = doc1 .set_bytes(author1, key1.to_vec(), value1.to_vec()) .await?; - assert_latest(&blobs1, &doc1, key1, value1).await; + assert_latest(blobs1, &doc1, key1, value1).await; info!("peer1: wait for 1 event (local insert, and pendingcontentready)"); assert_next( &mut events1, @@ -337,7 +337,7 @@ async fn sync_full_basic() -> Result<()> { Box::new(move |e| matches!(e, LiveEvent::ContentReady { hash } if *hash == hash1)), ], ).await; - assert_latest(&blobs0, &doc0, key1, value1).await; + assert_latest(blobs0, &doc0, key1, value1).await; // Note: If we could check gossip messages directly here (we can't easily), we would notice // that peer1 will receive a `Op::ContentReady` gossip message, broadcast @@ -389,8 +389,8 @@ async fn sync_full_basic() -> Result<()> { match_event!(LiveEvent::PendingContentReady), ] ).await; - assert_latest(&blobs2, &doc2, b"k1", b"v1").await; - assert_latest(&blobs2, &doc2, b"k2", b"v2").await; + assert_latest(blobs2, &doc2, b"k1", b"v1").await; + assert_latest(blobs2, &doc2, b"k2", b"v2").await; info!("peer0: wait for 2 events (join & accept sync finished from peer2)"); assert_next( @@ -606,7 +606,7 @@ async fn sync_restart_node() -> Result<()> { // create doc & ticket on node1 let doc1 = node1.docs().create().await?; - let blobs1 = node1.blobs().clone(); + let blobs1 = node1.blobs(); let mut events1 = doc1.subscribe().await?; let ticket = doc1 .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) @@ -629,7 +629,7 @@ async fn sync_restart_node() -> Result<()> { info!("node2 set a"); let hash_a = doc2.set_bytes(author2, "n2/a", "a").await?; - assert_latest(&blobs2, &doc2, b"n2/a", b"a").await; + assert_latest(blobs2, &doc2, b"n2/a", b"a").await; assert_next_unordered_with_optionals( &mut events1, @@ -647,7 +647,7 @@ async fn sync_restart_node() -> Result<()> { ], ) .await; - assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; + assert_latest(blobs1, &doc1, b"n2/a", b"a").await; info!(me = id1.fmt_short(), "node1 start shutdown"); node1.shutdown().await?; @@ -673,7 +673,7 @@ async fn sync_restart_node() -> Result<()> { let doc1 = node1.docs().open(doc1.id()).await?.expect("doc to exist"); let blobs1 = node1.blobs(); let mut events1 = doc1.subscribe().await?; - assert_latest(&blobs1, &doc1, b"n2/a", b"a").await; + assert_latest(blobs1, &doc1, b"n2/a", b"a").await; // check that initial resync is working doc1.start_sync(vec![]).await?; @@ -691,7 +691,7 @@ async fn sync_restart_node() -> Result<()> { match_event!(LiveEvent::PendingContentReady), ] ).await; - assert_latest(&blobs1, &doc1, b"n2/b", b"b").await; + assert_latest(blobs1, &doc1, b"n2/b", b"b").await; // check that live conn is working info!(me = id2.fmt_short(), "node2 set c"); @@ -711,7 +711,7 @@ async fn sync_restart_node() -> Result<()> { ] ).await; - assert_latest(&blobs1, &doc1, b"n2/c", b"c").await; + assert_latest(blobs1, &doc1, b"n2/c", b"c").await; Ok(()) } @@ -916,7 +916,7 @@ async fn sync_big() -> Result<()> { // assert initial data for (i, doc) in docs.iter().enumerate() { let blobs = nodes[i].blobs(); - let entries = get_all_with_content(&blobs, doc).await?; + let entries = get_all_with_content(blobs, doc).await?; let mut expected = expected .iter() .filter(|e| e.author == authors[i]) @@ -1169,7 +1169,7 @@ async fn doc_delete() -> Result<()> { let hash = doc .set_bytes(author, b"foo".to_vec(), b"hi".to_vec()) .await?; - assert_latest(&blobs, &doc, b"foo", b"hi").await; + assert_latest(blobs, &doc, b"foo", b"hi").await; let deleted = doc.del(author, b"foo".to_vec()).await?; assert_eq!(deleted, 1); From bf8338c33a97b1f94993897fbe5583a997f05d05 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 12:25:00 +0200 Subject: [PATCH 16/26] use TestResult --- tests/client.rs | 61 +++++++++++++++++++++++-------------------------- tests/sync.rs | 4 ++-- 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/tests/client.rs b/tests/client.rs index 83fdae0..7b68be2 100644 --- a/tests/client.rs +++ b/tests/client.rs @@ -39,7 +39,7 @@ async fn test_doc_close() -> Result<()> { } #[tokio::test] -async fn test_doc_import_export() -> Result<()> { +async fn test_doc_import_export() -> TestResult<()> { let _guard = iroh_test::logging::setup(); let node = Node::memory().spawn().await?; @@ -160,50 +160,47 @@ async fn test_default_author_memory() -> Result<()> { async fn test_default_author_persist() -> TestResult<()> { let _guard = iroh_test::logging::setup(); - let iroh_root_dir = tempfile::TempDir::new().unwrap(); + let iroh_root_dir = tempfile::TempDir::new()?; let iroh_root = iroh_root_dir.path(); // check that the default author exists and cannot be deleted. let default_author = { - let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); - let author = iroh.docs().author_default().await.unwrap(); - assert!(iroh.docs().author_export(author).await.unwrap().is_some()); + let iroh = Node::persistent(iroh_root).spawn().await?; + let author = iroh.docs().author_default().await?; + assert!(iroh.docs().author_export(author).await?.is_some()); assert!(iroh.docs().author_delete(author).await.is_err()); - iroh.shutdown().await.unwrap(); + iroh.shutdown().await?; author }; // check that the default author is persisted across restarts. { - let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); - let author = iroh.docs().author_default().await.unwrap(); + let iroh = Node::persistent(iroh_root).spawn().await?; + let author = iroh.docs().author_default().await?; assert_eq!(author, default_author); - assert!(iroh.docs().author_export(author).await.unwrap().is_some()); + assert!(iroh.docs().author_export(author).await?.is_some()); assert!(iroh.docs().author_delete(author).await.is_err()); - iroh.shutdown().await.unwrap(); + iroh.shutdown().await?; }; // check that a new default author is created if the default author file is deleted // manually. let default_author = { - tokio::fs::remove_file(iroh_root.join("default-author")) - .await - .unwrap(); - let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); - let author = iroh.docs().author_default().await.unwrap(); + tokio::fs::remove_file(iroh_root.join("default-author")).await?; + let iroh = Node::persistent(iroh_root).spawn().await?; + let author = iroh.docs().author_default().await?; assert!(author != default_author); - assert!(iroh.docs().author_export(author).await.unwrap().is_some()); + assert!(iroh.docs().author_export(author).await?.is_some()); assert!(iroh.docs().author_delete(author).await.is_err()); - iroh.shutdown().await.unwrap(); + iroh.shutdown().await?; author }; // check that the node fails to start if the default author is missing from the docs store. { - let mut docs_store = - iroh_docs::store::fs::Store::persistent(iroh_root.join("docs.redb")).unwrap(); - docs_store.delete_author(default_author).unwrap(); - docs_store.flush().unwrap(); + let mut docs_store = iroh_docs::store::fs::Store::persistent(iroh_root.join("docs.redb"))?; + docs_store.delete_author(default_author)?; + docs_store.flush()?; drop(docs_store); let iroh = Node::persistent(iroh_root).spawn().await; assert!(iroh.is_err()); @@ -213,30 +210,28 @@ async fn test_default_author_persist() -> TestResult<()> { #[cfg(target_os = "macos")] tokio::time::sleep(Duration::from_secs(1)).await; - tokio::fs::remove_file(iroh_root.join("default-author")) - .await - .unwrap(); + tokio::fs::remove_file(iroh_root.join("default-author")).await?; drop(iroh); let iroh = Node::persistent(iroh_root).spawn().await; if let Err(cause) = iroh.as_ref() { panic!("failed to start node: {:?}", cause); } - iroh.unwrap().shutdown().await.unwrap(); + iroh?.shutdown().await?; } // check that the default author can be set manually and is persisted. let default_author = { - let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); - let author = iroh.docs().author_create().await.unwrap(); - iroh.docs().author_set_default(author).await.unwrap(); - assert_eq!(iroh.docs().author_default().await.unwrap(), author); - iroh.shutdown().await.unwrap(); + let iroh = Node::persistent(iroh_root).spawn().await?; + let author = iroh.docs().author_create().await?; + iroh.docs().author_set_default(author).await?; + assert_eq!(iroh.docs().author_default().await?, author); + iroh.shutdown().await?; author }; { - let iroh = Node::persistent(iroh_root).spawn().await.unwrap(); - assert_eq!(iroh.docs().author_default().await.unwrap(), default_author); - iroh.shutdown().await.unwrap(); + let iroh = Node::persistent(iroh_root).spawn().await?; + assert_eq!(iroh.docs().author_default().await?, default_author); + iroh.shutdown().await?; } Ok(()) diff --git a/tests/sync.rs b/tests/sync.rs index 6b20f3b..83c88c6 100644 --- a/tests/sync.rs +++ b/tests/sync.rs @@ -244,7 +244,7 @@ async fn sync_gossip_bulk() -> Result<()> { /// This tests basic sync and gossip with 3 peers. #[tokio::test] #[ignore = "flaky"] -async fn sync_full_basic() -> Result<()> { +async fn sync_full_basic() -> testresult::TestResult<()> { let mut rng = test_rng(b"sync_full_basic"); setup_logging(); let mut nodes = spawn_nodes(2, &mut rng).await?; @@ -984,7 +984,7 @@ async fn sync_big() -> Result<()> { #[tokio::test] #[cfg(feature = "test-utils")] -async fn test_list_docs_stream() -> Result<()> { +async fn test_list_docs_stream() -> testresult::TestResult<()> { let node = Node::memory() .relay_mode(RelayMode::Disabled) .spawn() From e87f0a93683fd95958356fb9254707a7d6d76bb0 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 12:32:15 +0200 Subject: [PATCH 17/26] fix unused warning --- tests/client.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/client.rs b/tests/client.rs index 7b68be2..ce5c049 100644 --- a/tests/client.rs +++ b/tests/client.rs @@ -1,5 +1,3 @@ -use std::time::Duration; - use anyhow::{Context, Result}; use futures_util::TryStreamExt; use iroh_blobs::{ @@ -208,7 +206,7 @@ async fn test_default_author_persist() -> TestResult<()> { // somehow the blob store is not shutdown correctly (yet?) on macos. // so we give it some time until we find a proper fix. #[cfg(target_os = "macos")] - tokio::time::sleep(Duration::from_secs(1)).await; + tokio::time::sleep(std::time::Duration::from_secs(1)).await; tokio::fs::remove_file(iroh_root.join("default-author")).await?; drop(iroh); From 38e2e7fe6fde942b850974bbd0df1e4f7cc654f2 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 12:37:14 +0200 Subject: [PATCH 18/26] WIP --- tests/client.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/client.rs b/tests/client.rs index ce5c049..16a4389 100644 --- a/tests/client.rs +++ b/tests/client.rs @@ -1,3 +1,4 @@ +#![cfg(feature = "rpc")] use anyhow::{Context, Result}; use futures_util::TryStreamExt; use iroh_blobs::{ From e9aa12830087b2412cd08e287c88488f8d5e874e Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 12:46:11 +0200 Subject: [PATCH 19/26] Accept unmaintained instant crate --- deny.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/deny.toml b/deny.toml index c5e36d1..577bdba 100644 --- a/deny.toml +++ b/deny.toml @@ -34,6 +34,7 @@ license-files = [ [advisories] ignore = [ "RUSTSEC-2024-0370", # unmaintained, no upgrade available + "RUSTSEC-2024-0384", # unmaintained, no upgrade available ] [sources] From 6d32cdc2ae86bf3703016e019574be9716404cef Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 17:08:56 +0200 Subject: [PATCH 20/26] Go back to authors and docs split it is not better or worse than the new one, but it breaks less code --- src/rpc/client.rs | 942 +-------------------------------- src/rpc/client/authors.rs | 113 ++++ src/rpc/client/docs.rs | 868 ++++++++++++++++++++++++++++++ src/rpc/docs_handle_request.rs | 14 +- src/rpc/proto.rs | 2 +- tests/client.rs | 62 +-- tests/gc.rs | 2 +- tests/sync.rs | 28 +- tests/util.rs | 12 +- 9 files changed, 1048 insertions(+), 995 deletions(-) create mode 100644 src/rpc/client/authors.rs create mode 100644 src/rpc/client/docs.rs diff --git a/src/rpc/client.rs b/src/rpc/client.rs index 92a2e43..f9b2d7d 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -1,939 +1,3 @@ -//! API for document management. -//! -//! The main entry point is the [`Client`]. -use std::{ - path::{Path, PathBuf}, - pin::Pin, - sync::Arc, - task::{Context, Poll}, -}; - -use anyhow::{anyhow, Context as _, Result}; -use bytes::Bytes; -use derive_more::{Display, FromStr}; -use futures_lite::{Stream, StreamExt}; -use iroh_base::{key::PublicKey, node_addr::AddrInfoOptions}; -use iroh_blobs::{export::ExportProgress, store::ExportMode, Hash}; -use iroh_net::NodeAddr; -use portable_atomic::{AtomicBool, Ordering}; -use quic_rpc::{client::BoxedConnector, message::RpcMsg, Connector}; -use serde::{Deserialize, Serialize}; - -use super::proto::{ - AuthorCreateRequest, AuthorDeleteRequest, AuthorExportRequest, AuthorGetDefaultRequest, - AuthorImportRequest, AuthorListRequest, AuthorSetDefaultRequest, CloseRequest, CreateRequest, - DelRequest, DelResponse, DocListRequest, DocSubscribeRequest, DropRequest, ExportFileRequest, - GetDownloadPolicyRequest, GetExactRequest, GetManyRequest, GetSyncPeersRequest, - ImportFileRequest, ImportRequest, LeaveRequest, OpenRequest, RpcService, - SetDownloadPolicyRequest, SetHashRequest, SetRequest, ShareRequest, StartSyncRequest, - StatusRequest, -}; -#[doc(inline)] -pub use crate::engine::{Origin, SyncEvent, SyncReason}; -use crate::{ - actor::OpenState, - store::{DownloadPolicy, Query}, - Author, AuthorId, Capability, CapabilityKind, ContentStatus, DocTicket, NamespaceId, - PeerIdBytes, RecordIdentifier, -}; - -/// Iroh docs client. -#[derive(Debug, Clone)] -pub struct Client> { - pub(super) rpc: quic_rpc::RpcClient, -} - -impl> Client { - /// Creates a new docs client. - pub fn new(rpc: quic_rpc::RpcClient) -> Self { - Self { rpc } - } - - /// Creates a client. - pub async fn create(&self) -> Result> { - let res = self.rpc.rpc(CreateRequest {}).await??; - let doc = Doc::new(self.rpc.clone(), res.id); - Ok(doc) - } - - /// Deletes a document from the local node. - /// - /// This is a destructive operation. Both the document secret key and all entries in the - /// document will be permanently deleted from the node's storage. Content blobs will be deleted - /// through garbage collection unless they are referenced from another document or tag. - pub async fn drop_doc(&self, doc_id: NamespaceId) -> Result<()> { - self.rpc.rpc(DropRequest { doc_id }).await??; - Ok(()) - } - - /// Imports a document from a namespace capability. - /// - /// This does not start sync automatically. Use [`Doc::start_sync`] to start sync. - pub async fn import_namespace(&self, capability: Capability) -> Result> { - let res = self.rpc.rpc(ImportRequest { capability }).await??; - let doc = Doc::new(self.rpc.clone(), res.doc_id); - Ok(doc) - } - - /// Imports a document from a ticket and joins all peers in the ticket. - pub async fn import(&self, ticket: DocTicket) -> Result> { - let DocTicket { capability, nodes } = ticket; - let doc = self.import_namespace(capability).await?; - doc.start_sync(nodes).await?; - Ok(doc) - } - - /// Imports a document from a ticket, creates a subscription stream and joins all peers in the ticket. - /// - /// Returns the [`Doc`] and a [`Stream`] of [`LiveEvent`]s. - /// - /// The subscription stream is created before the sync is started, so the first call to this - /// method after starting the node is guaranteed to not miss any sync events. - pub async fn import_and_subscribe( - &self, - ticket: DocTicket, - ) -> Result<(Doc, impl Stream>)> { - let DocTicket { capability, nodes } = ticket; - let res = self.rpc.rpc(ImportRequest { capability }).await??; - let doc = Doc::new(self.rpc.clone(), res.doc_id); - let events = doc.subscribe().await?; - doc.start_sync(nodes).await?; - Ok((doc, events)) - } - - /// Lists all documents. - pub async fn list(&self) -> Result>> { - let stream = self.rpc.server_streaming(DocListRequest {}).await?; - Ok(flatten(stream).map(|res| res.map(|res| (res.id, res.capability)))) - } - - /// Returns a [`Doc`] client for a single document. - /// - /// Returns None if the document cannot be found. - pub async fn open(&self, id: NamespaceId) -> Result>> { - self.rpc.rpc(OpenRequest { doc_id: id }).await??; - let doc = Doc::new(self.rpc.clone(), id); - Ok(Some(doc)) - } - - /// Creates a new document author. - /// - /// You likely want to save the returned [`AuthorId`] somewhere so that you can use this author - /// again. - /// - /// If you need only a single author, use [`Self::author_default`]. - pub async fn author_create(&self) -> Result { - let res = self.rpc.rpc(AuthorCreateRequest).await??; - Ok(res.author_id) - } - - /// Returns the default document author of this node. - /// - /// On persistent nodes, the author is created on first start and its public key is saved - /// in the data directory. - /// - /// The default author can be set with [`Self::author_set_default`]. - pub async fn author_default(&self) -> Result { - let res = self.rpc.rpc(AuthorGetDefaultRequest).await??; - Ok(res.author_id) - } - - /// Sets the node-wide default author. - /// - /// If the author does not exist, an error is returned. - /// - /// On a persistent node, the author id will be saved to a file in the data directory and - /// reloaded after a restart. - pub async fn author_set_default(&self, author_id: AuthorId) -> Result<()> { - self.rpc - .rpc(AuthorSetDefaultRequest { author_id }) - .await??; - Ok(()) - } - - /// Lists document authors for which we have a secret key. - /// - /// It's only possible to create writes from authors that we have the secret key of. - pub async fn author_list(&self) -> Result>> { - let stream = self.rpc.server_streaming(AuthorListRequest {}).await?; - Ok(flatten(stream).map(|res| res.map(|res| res.author_id))) - } - - /// Exports the given author. - /// - /// Warning: The [`Author`] struct contains sensitive data. - pub async fn author_export(&self, author: AuthorId) -> Result> { - let res = self.rpc.rpc(AuthorExportRequest { author }).await??; - Ok(res.author) - } - - /// Imports the given author. - /// - /// Warning: The [`Author`] struct contains sensitive data. - pub async fn author_import(&self, author: Author) -> Result<()> { - self.rpc.rpc(AuthorImportRequest { author }).await??; - Ok(()) - } - - /// Deletes the given author by id. - /// - /// Warning: This permanently removes this author. - /// - /// Returns an error if attempting to delete the default author. - pub async fn author_delete(&self, author: AuthorId) -> Result<()> { - self.rpc.rpc(AuthorDeleteRequest { author }).await??; - Ok(()) - } -} - -/// Document handle -#[derive(Debug, Clone)] -pub struct Doc = BoxedConnector>(Arc>) -where - C: quic_rpc::Connector; - -impl> PartialEq for Doc { - fn eq(&self, other: &Self) -> bool { - self.0.id == other.0.id - } -} - -impl> Eq for Doc {} - -#[derive(Debug)] -struct DocInner = BoxedConnector> { - id: NamespaceId, - rpc: quic_rpc::RpcClient, - closed: AtomicBool, - rt: tokio::runtime::Handle, -} - -impl Drop for DocInner -where - C: quic_rpc::Connector, -{ - fn drop(&mut self) { - let doc_id = self.id; - let rpc = self.rpc.clone(); - if !self.closed.swap(true, Ordering::Relaxed) { - self.rt.spawn(async move { - rpc.rpc(CloseRequest { doc_id }).await.ok(); - }); - } - } -} - -impl> Doc { - fn new(rpc: quic_rpc::RpcClient, id: NamespaceId) -> Self { - Self(Arc::new(DocInner { - rpc, - id, - closed: AtomicBool::new(false), - rt: tokio::runtime::Handle::current(), - })) - } - - async fn rpc(&self, msg: M) -> Result - where - M: RpcMsg, - { - let res = self.0.rpc.rpc(msg).await?; - Ok(res) - } - - /// Returns the document id of this doc. - pub fn id(&self) -> NamespaceId { - self.0.id - } - - /// Closes the document. - pub async fn close(&self) -> Result<()> { - if !self.0.closed.swap(true, Ordering::Relaxed) { - self.rpc(CloseRequest { doc_id: self.id() }).await??; - } - Ok(()) - } - - fn ensure_open(&self) -> Result<()> { - if self.0.closed.load(Ordering::Relaxed) { - Err(anyhow!("document is closed")) - } else { - Ok(()) - } - } - - /// Sets the content of a key to a byte array. - pub async fn set_bytes( - &self, - author_id: AuthorId, - key: impl Into, - value: impl Into, - ) -> Result { - self.ensure_open()?; - let res = self - .rpc(SetRequest { - doc_id: self.id(), - author_id, - key: key.into(), - value: value.into(), - }) - .await??; - Ok(res.entry.content_hash()) - } - - /// Sets an entries on the doc via its key, hash, and size. - pub async fn set_hash( - &self, - author_id: AuthorId, - key: impl Into, - hash: Hash, - size: u64, - ) -> Result<()> { - self.ensure_open()?; - self.rpc(SetHashRequest { - doc_id: self.id(), - author_id, - key: key.into(), - hash, - size, - }) - .await??; - Ok(()) - } - - /// Adds an entry from an absolute file path - pub async fn import_file( - &self, - author: AuthorId, - key: Bytes, - path: impl AsRef, - in_place: bool, - ) -> Result { - self.ensure_open()?; - let stream = self - .0 - .rpc - .server_streaming(ImportFileRequest { - doc_id: self.id(), - author_id: author, - path: path.as_ref().into(), - key, - in_place, - }) - .await?; - Ok(ImportFileProgress::new(stream)) - } - - /// Exports an entry as a file to a given absolute path. - pub async fn export_file( - &self, - entry: Entry, - path: impl AsRef, - mode: ExportMode, - ) -> Result { - self.ensure_open()?; - let stream = self - .0 - .rpc - .server_streaming(ExportFileRequest { - entry: entry.0, - path: path.as_ref().into(), - mode, - }) - .await?; - Ok(ExportFileProgress::new(stream)) - } - - /// Deletes entries that match the given `author` and key `prefix`. - /// - /// This inserts an empty entry with the key set to `prefix`, effectively clearing all other - /// entries whose key starts with or is equal to the given `prefix`. - /// - /// Returns the number of entries deleted. - pub async fn del(&self, author_id: AuthorId, prefix: impl Into) -> Result { - self.ensure_open()?; - let res = self - .rpc(DelRequest { - doc_id: self.id(), - author_id, - prefix: prefix.into(), - }) - .await??; - let DelResponse { removed } = res; - Ok(removed) - } - - /// Returns an entry for a key and author. - /// - /// Optionally also returns the entry unless it is empty (i.e. a deletion marker). - pub async fn get_exact( - &self, - author: AuthorId, - key: impl AsRef<[u8]>, - include_empty: bool, - ) -> Result> { - self.ensure_open()?; - let res = self - .rpc(GetExactRequest { - author, - key: key.as_ref().to_vec().into(), - doc_id: self.id(), - include_empty, - }) - .await??; - Ok(res.entry.map(|entry| entry.into())) - } - - /// Returns all entries matching the query. - pub async fn get_many( - &self, - query: impl Into, - ) -> Result>> { - self.ensure_open()?; - let stream = self - .0 - .rpc - .server_streaming(GetManyRequest { - doc_id: self.id(), - query: query.into(), - }) - .await?; - Ok(flatten(stream).map(|res| res.map(|res| res.entry.into()))) - } - - /// Returns a single entry. - pub async fn get_one(&self, query: impl Into) -> Result> { - self.get_many(query).await?.next().await.transpose() - } - - /// Shares this document with peers over a ticket. - pub async fn share( - &self, - mode: ShareMode, - addr_options: AddrInfoOptions, - ) -> anyhow::Result { - self.ensure_open()?; - let res = self - .rpc(ShareRequest { - doc_id: self.id(), - mode, - addr_options, - }) - .await??; - Ok(res.0) - } - - /// Starts to sync this document with a list of peers. - pub async fn start_sync(&self, peers: Vec) -> Result<()> { - self.ensure_open()?; - let _res = self - .rpc(StartSyncRequest { - doc_id: self.id(), - peers, - }) - .await??; - Ok(()) - } - - /// Stops the live sync for this document. - pub async fn leave(&self) -> Result<()> { - self.ensure_open()?; - let _res = self.rpc(LeaveRequest { doc_id: self.id() }).await??; - Ok(()) - } - - /// Subscribes to events for this document. - pub async fn subscribe(&self) -> anyhow::Result>> { - self.ensure_open()?; - let stream = self - .0 - .rpc - .try_server_streaming(DocSubscribeRequest { doc_id: self.id() }) - .await?; - Ok(stream.map(|res| match res { - Ok(res) => Ok(res.event.into()), - Err(err) => Err(err.into()), - })) - } - - /// Returns status info for this document - pub async fn status(&self) -> anyhow::Result { - self.ensure_open()?; - let res = self.rpc(StatusRequest { doc_id: self.id() }).await??; - Ok(res.status) - } - - /// Sets the download policy for this document - pub async fn set_download_policy(&self, policy: DownloadPolicy) -> Result<()> { - self.rpc(SetDownloadPolicyRequest { - doc_id: self.id(), - policy, - }) - .await??; - Ok(()) - } - - /// Returns the download policy for this document - pub async fn get_download_policy(&self) -> Result { - let res = self - .rpc(GetDownloadPolicyRequest { doc_id: self.id() }) - .await??; - Ok(res.policy) - } - - /// Returns sync peers for this document - pub async fn get_sync_peers(&self) -> Result>> { - let res = self - .rpc(GetSyncPeersRequest { doc_id: self.id() }) - .await??; - Ok(res.peers) - } -} - -impl<'a, C> From<&'a Doc> for &'a quic_rpc::RpcClient -where - C: quic_rpc::Connector, -{ - fn from(doc: &'a Doc) -> &'a quic_rpc::RpcClient { - &doc.0.rpc - } -} - -/// A single entry in a [`Doc`]. -#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq)] -pub struct Entry(crate::Entry); - -impl From for Entry { - fn from(value: crate::Entry) -> Self { - Self(value) - } -} - -impl From for Entry { - fn from(value: crate::SignedEntry) -> Self { - Self(value.into()) - } -} - -impl Entry { - /// Returns the [`RecordIdentifier`] for this entry. - pub fn id(&self) -> &RecordIdentifier { - self.0.id() - } - - /// Returns the [`AuthorId`] of this entry. - pub fn author(&self) -> AuthorId { - self.0.author() - } - - /// Returns the [`struct@Hash`] of the content data of this record. - pub fn content_hash(&self) -> Hash { - self.0.content_hash() - } - - /// Returns the length of the data addressed by this record's content hash. - pub fn content_len(&self) -> u64 { - self.0.content_len() - } - - /// Returns the key of this entry. - pub fn key(&self) -> &[u8] { - self.0.key() - } - - /// Returns the timestamp of this entry. - pub fn timestamp(&self) -> u64 { - self.0.timestamp() - } -} - -/// Progress messages for an doc import operation -/// -/// An import operation involves computing the outboard of a file, and then -/// either copying or moving the file into the database, then setting the author, hash, size, and tag of that -/// file as an entry in the doc. -#[derive(Debug, Serialize, Deserialize)] -pub enum ImportProgress { - /// An item was found with name `name`, from now on referred to via `id`. - Found { - /// A new unique id for this entry. - id: u64, - /// The name of the entry. - name: String, - /// The size of the entry in bytes. - size: u64, - }, - /// We got progress ingesting item `id`. - Progress { - /// The unique id of the entry. - id: u64, - /// The offset of the progress, in bytes. - offset: u64, - }, - /// We are done adding `id` to the data store and the hash is `hash`. - IngestDone { - /// The unique id of the entry. - id: u64, - /// The hash of the entry. - hash: Hash, - }, - /// We are done setting the entry to the doc. - AllDone { - /// The key of the entry - key: Bytes, - }, - /// We got an error and need to abort. - /// - /// This will be the last message in the stream. - Abort(serde_error::Error), -} - -/// Intended capability for document share tickets -#[derive(Serialize, Deserialize, Debug, Clone, Display, FromStr)] -pub enum ShareMode { - /// Read-only access - Read, - /// Write access - Write, -} - -/// Events informing about actions of the live sync progress. -#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, strum::Display)] -pub enum LiveEvent { - /// A local insertion. - InsertLocal { - /// The inserted entry. - entry: Entry, - }, - /// Received a remote insert. - InsertRemote { - /// The peer that sent us the entry. - from: PublicKey, - /// The inserted entry. - entry: Entry, - /// If the content is available at the local node - content_status: ContentStatus, - }, - /// The content of an entry was downloaded and is now available at the local node - ContentReady { - /// The content hash of the newly available entry content - hash: Hash, - }, - /// We have a new neighbor in the swarm. - NeighborUp(PublicKey), - /// We lost a neighbor in the swarm. - NeighborDown(PublicKey), - /// A set-reconciliation sync finished. - SyncFinished(SyncEvent), - /// All pending content is now ready. - /// - /// This event signals that all queued content downloads from the last sync run have either - /// completed or failed. - /// - /// It will only be emitted after a [`Self::SyncFinished`] event, never before. - /// - /// Receiving this event does not guarantee that all content in the document is available. If - /// blobs failed to download, this event will still be emitted after all operations completed. - PendingContentReady, -} - -impl From for LiveEvent { - fn from(event: crate::engine::LiveEvent) -> LiveEvent { - match event { - crate::engine::LiveEvent::InsertLocal { entry } => Self::InsertLocal { - entry: entry.into(), - }, - crate::engine::LiveEvent::InsertRemote { - from, - entry, - content_status, - } => Self::InsertRemote { - from, - content_status, - entry: entry.into(), - }, - crate::engine::LiveEvent::ContentReady { hash } => Self::ContentReady { hash }, - crate::engine::LiveEvent::NeighborUp(node) => Self::NeighborUp(node), - crate::engine::LiveEvent::NeighborDown(node) => Self::NeighborDown(node), - crate::engine::LiveEvent::SyncFinished(details) => Self::SyncFinished(details), - crate::engine::LiveEvent::PendingContentReady => Self::PendingContentReady, - } - } -} - -/// Progress stream for [`Doc::import_file`]. -#[derive(derive_more::Debug)] -#[must_use = "streams do nothing unless polled"] -pub struct ImportFileProgress { - #[debug(skip)] - stream: Pin> + Send + Unpin + 'static>>, -} - -impl ImportFileProgress { - fn new( - stream: (impl Stream, impl Into>> - + Send - + Unpin - + 'static), - ) -> Self { - let stream = stream.map(|item| match item { - Ok(item) => Ok(item.into()), - Err(err) => Err(err.into()), - }); - Self { - stream: Box::pin(stream), - } - } - - /// Finishes writing the stream, ignoring all intermediate progress events. - /// - /// Returns a [`ImportFileOutcome`] which contains a tag, key, and hash and the size of the - /// content. - pub async fn finish(mut self) -> Result { - let mut entry_size = 0; - let mut entry_hash = None; - while let Some(msg) = self.next().await { - match msg? { - ImportProgress::Found { size, .. } => { - entry_size = size; - } - ImportProgress::AllDone { key } => { - let hash = entry_hash - .context("expected DocImportProgress::IngestDone event to occur")?; - let outcome = ImportFileOutcome { - hash, - key, - size: entry_size, - }; - return Ok(outcome); - } - ImportProgress::Abort(err) => return Err(err.into()), - ImportProgress::Progress { .. } => {} - ImportProgress::IngestDone { hash, .. } => { - entry_hash = Some(hash); - } - } - } - Err(anyhow!("Response stream ended prematurely")) - } -} - -/// Outcome of a [`Doc::import_file`] operation -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct ImportFileOutcome { - /// The hash of the entry's content - pub hash: Hash, - /// The size of the entry - pub size: u64, - /// The key of the entry - pub key: Bytes, -} - -impl Stream for ImportFileProgress { - type Item = Result; - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(&mut self.stream).poll_next(cx) - } -} - -/// Progress stream for [`Doc::export_file`]. -#[derive(derive_more::Debug)] -pub struct ExportFileProgress { - #[debug(skip)] - stream: Pin> + Send + Unpin + 'static>>, -} -impl ExportFileProgress { - fn new( - stream: (impl Stream, impl Into>> - + Send - + Unpin - + 'static), - ) -> Self { - let stream = stream.map(|item| match item { - Ok(item) => Ok(item.into()), - Err(err) => Err(err.into()), - }); - Self { - stream: Box::pin(stream), - } - } - - /// Iterates through the export progress stream, returning when the stream has completed. - /// - /// Returns a [`ExportFileOutcome`] which contains a file path the data was written to and the size of the content. - pub async fn finish(mut self) -> Result { - let mut total_size = 0; - let mut path = None; - while let Some(msg) = self.next().await { - match msg? { - ExportProgress::Found { size, outpath, .. } => { - total_size = size.value(); - path = Some(outpath); - } - ExportProgress::AllDone => { - let path = path.context("expected ExportProgress::Found event to occur")?; - let outcome = ExportFileOutcome { - size: total_size, - path, - }; - return Ok(outcome); - } - ExportProgress::Done { .. } => {} - ExportProgress::Abort(err) => return Err(anyhow!(err)), - ExportProgress::Progress { .. } => {} - } - } - Err(anyhow!("Response stream ended prematurely")) - } -} - -/// Outcome of a [`Doc::export_file`] operation -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct ExportFileOutcome { - /// The size of the entry - pub size: u64, - /// The path to which the entry was saved - pub path: PathBuf, -} - -impl Stream for ExportFileProgress { - type Item = Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - Pin::new(&mut self.stream).poll_next(cx) - } -} - -fn flatten( - s: impl Stream, E2>>, -) -> impl Stream> -where - E1: std::error::Error + Send + Sync + 'static, - E2: std::error::Error + Send + Sync + 'static, -{ - s.map(|res| match res { - Ok(Ok(res)) => Ok(res), - Ok(Err(err)) => Err(err.into()), - Err(err) => Err(err.into()), - }) -} - -#[cfg(test)] -mod tests { - use iroh_blobs::util::local_pool::LocalPool; - use iroh_gossip::net::GOSSIP_ALPN; - // use rand::RngCore; - // use tokio::io::AsyncWriteExt; - use tracing::warn; - - use super::*; - use crate::{ - engine::{DefaultAuthorStorage, Engine}, - net::DOCS_ALPN, - }; - - async fn setup_router() -> Result<( - Client, - iroh_router::Router, - tokio::task::JoinHandle>, - )> { - let endpoint = iroh_net::Endpoint::builder().bind().await?; - let local_pool = LocalPool::single(); - let mut router = iroh_router::Router::builder(endpoint.clone()); - - // Setup gossip - let my_addr = endpoint.node_addr().await?; - let gossip = iroh_gossip::net::Gossip::from_endpoint( - endpoint.clone(), - Default::default(), - &my_addr.info, - ); - router = router.accept(GOSSIP_ALPN.to_vec(), Arc::new(gossip.clone())); - - // Setup blobs - - let bao_store = iroh_blobs::store::mem::Store::new(); - let downloader = iroh_blobs::downloader::Downloader::new( - bao_store.clone(), - endpoint.clone(), - local_pool.handle().clone(), - ); - let blobs = iroh_blobs::net_protocol::Blobs::new_with_events( - bao_store.clone(), - local_pool.handle().clone(), - Default::default(), - downloader.clone(), - endpoint.clone(), - ); - router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), Arc::new(blobs)); - - // Setup docs - - let replica_store = crate::store::Store::memory(); - let engine = Engine::spawn( - endpoint.clone(), - gossip, - replica_store, - bao_store, - downloader, - DefaultAuthorStorage::Mem, - local_pool.handle().clone(), - ) - .await?; - - router = router.accept(DOCS_ALPN.to_vec(), Arc::new(engine.clone())); - - // Build the router - - let router = router.spawn().await?; - - // Setup RPC - let (internal_rpc, controller) = quic_rpc::transport::flume::channel(32); - let controller = quic_rpc::transport::boxed::BoxedConnector::new(controller); - let internal_rpc = quic_rpc::transport::boxed::BoxedListener::new(internal_rpc); - let internal_rpc = quic_rpc::RpcServer::new(internal_rpc); - - let rpc_server_task = tokio::task::spawn(async move { - loop { - let request = internal_rpc.accept().await; - match request { - Ok(accepting) => { - let engine = engine.clone(); - tokio::task::spawn(async move { - let (msg, chan) = accepting.read_first().await.unwrap(); - engine.handle_rpc_request(msg, chan).await.unwrap(); - }); - } - Err(err) => { - warn!("rpc error: {:?}", err); - } - } - } - }); - - let docs_client = Client::new(quic_rpc::RpcClient::new(controller.clone())); - - Ok((docs_client, router, rpc_server_task)) - } - - #[tokio::test] - async fn test_drop_doc_client_sync() -> Result<()> { - let _guard = iroh_test::logging::setup(); - - let (docs_client, router, rpc_server_task) = setup_router().await?; - - let doc = docs_client.create().await?; - - let res = std::thread::spawn(move || { - drop(doc); - }); - - tokio::task::spawn_blocking(move || res.join().map_err(|e| anyhow::anyhow!("{:?}", e))) - .await??; - - rpc_server_task.abort(); - router.shutdown().await?; - - Ok(()) - } -} +//! RPC Client for docs and authors +pub mod authors; +pub mod docs; diff --git a/src/rpc/client/authors.rs b/src/rpc/client/authors.rs new file mode 100644 index 0000000..64c85ea --- /dev/null +++ b/src/rpc/client/authors.rs @@ -0,0 +1,113 @@ +//! API for document management. +//! +//! The main entry point is the [`Client`]. + +use anyhow::Result; +use futures_lite::{Stream, StreamExt}; +use quic_rpc::{client::BoxedConnector, Connector}; + +#[doc(inline)] +pub use crate::engine::{Origin, SyncEvent, SyncReason}; +use crate::{ + rpc::proto::{ + AuthorCreateRequest, AuthorDeleteRequest, AuthorExportRequest, AuthorGetDefaultRequest, + AuthorImportRequest, AuthorListRequest, AuthorSetDefaultRequest, RpcService, + }, + Author, AuthorId, +}; + +/// Iroh docs client. +#[derive(Debug, Clone)] +pub struct Client> { + pub(super) rpc: quic_rpc::RpcClient, +} + +impl> Client { + /// Creates a new docs client. + pub fn new(rpc: quic_rpc::RpcClient) -> Self { + Self { rpc } + } + + /// Creates a new document author. + /// + /// You likely want to save the returned [`AuthorId`] somewhere so that you can use this author + /// again. + /// + /// If you need only a single author, use [`Self::author_default`]. + pub async fn create(&self) -> Result { + let res = self.rpc.rpc(AuthorCreateRequest).await??; + Ok(res.author_id) + } + + /// Returns the default document author of this node. + /// + /// On persistent nodes, the author is created on first start and its public key is saved + /// in the data directory. + /// + /// The default author can be set with [`Self::author_set_default`]. + pub async fn default(&self) -> Result { + let res = self.rpc.rpc(AuthorGetDefaultRequest).await??; + Ok(res.author_id) + } + + /// Sets the node-wide default author. + /// + /// If the author does not exist, an error is returned. + /// + /// On a persistent node, the author id will be saved to a file in the data directory and + /// reloaded after a restart. + pub async fn set_default(&self, author_id: AuthorId) -> Result<()> { + self.rpc + .rpc(AuthorSetDefaultRequest { author_id }) + .await??; + Ok(()) + } + + /// Lists document authors for which we have a secret key. + /// + /// It's only possible to create writes from authors that we have the secret key of. + pub async fn list(&self) -> Result>> { + let stream = self.rpc.server_streaming(AuthorListRequest {}).await?; + Ok(flatten(stream).map(|res| res.map(|res| res.author_id))) + } + + /// Exports the given author. + /// + /// Warning: The [`Author`] struct contains sensitive data. + pub async fn export(&self, author: AuthorId) -> Result> { + let res = self.rpc.rpc(AuthorExportRequest { author }).await??; + Ok(res.author) + } + + /// Imports the given author. + /// + /// Warning: The [`Author`] struct contains sensitive data. + pub async fn import(&self, author: Author) -> Result<()> { + self.rpc.rpc(AuthorImportRequest { author }).await??; + Ok(()) + } + + /// Deletes the given author by id. + /// + /// Warning: This permanently removes this author. + /// + /// Returns an error if attempting to delete the default author. + pub async fn delete(&self, author: AuthorId) -> Result<()> { + self.rpc.rpc(AuthorDeleteRequest { author }).await??; + Ok(()) + } +} + +fn flatten( + s: impl Stream, E2>>, +) -> impl Stream> +where + E1: std::error::Error + Send + Sync + 'static, + E2: std::error::Error + Send + Sync + 'static, +{ + s.map(|res| match res { + Ok(Ok(res)) => Ok(res), + Ok(Err(err)) => Err(err.into()), + Err(err) => Err(err.into()), + }) +} diff --git a/src/rpc/client/docs.rs b/src/rpc/client/docs.rs new file mode 100644 index 0000000..0b1d43c --- /dev/null +++ b/src/rpc/client/docs.rs @@ -0,0 +1,868 @@ +//! API for document management. +//! +//! The main entry point is the [`Client`]. +use std::{ + path::{Path, PathBuf}, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; + +use anyhow::{anyhow, Context as _, Result}; +use bytes::Bytes; +use derive_more::{Display, FromStr}; +use futures_lite::{Stream, StreamExt}; +use iroh_base::{key::PublicKey, node_addr::AddrInfoOptions}; +use iroh_blobs::{export::ExportProgress, store::ExportMode, Hash}; +use iroh_net::NodeAddr; +use portable_atomic::{AtomicBool, Ordering}; +use quic_rpc::{client::BoxedConnector, message::RpcMsg, Connector}; +use serde::{Deserialize, Serialize}; + +#[doc(inline)] +pub use crate::engine::{Origin, SyncEvent, SyncReason}; +use crate::{ + actor::OpenState, + rpc::proto::{ + CloseRequest, CreateRequest, DelRequest, DelResponse, DocListRequest, DocSubscribeRequest, + DropRequest, ExportFileRequest, GetDownloadPolicyRequest, GetExactRequest, GetManyRequest, + GetSyncPeersRequest, ImportFileRequest, ImportRequest, LeaveRequest, OpenRequest, + RpcService, SetDownloadPolicyRequest, SetHashRequest, SetRequest, ShareRequest, + StartSyncRequest, StatusRequest, + }, + store::{DownloadPolicy, Query}, + AuthorId, Capability, CapabilityKind, ContentStatus, DocTicket, NamespaceId, PeerIdBytes, + RecordIdentifier, +}; + +/// Iroh docs client. +#[derive(Debug, Clone)] +pub struct Client> { + pub(super) rpc: quic_rpc::RpcClient, +} + +impl> Client { + /// Creates a new docs client. + pub fn new(rpc: quic_rpc::RpcClient) -> Self { + Self { rpc } + } + + /// Creates a client. + pub async fn create(&self) -> Result> { + let res = self.rpc.rpc(CreateRequest {}).await??; + let doc = Doc::new(self.rpc.clone(), res.id); + Ok(doc) + } + + /// Deletes a document from the local node. + /// + /// This is a destructive operation. Both the document secret key and all entries in the + /// document will be permanently deleted from the node's storage. Content blobs will be deleted + /// through garbage collection unless they are referenced from another document or tag. + pub async fn drop_doc(&self, doc_id: NamespaceId) -> Result<()> { + self.rpc.rpc(DropRequest { doc_id }).await??; + Ok(()) + } + + /// Imports a document from a namespace capability. + /// + /// This does not start sync automatically. Use [`Doc::start_sync`] to start sync. + pub async fn import_namespace(&self, capability: Capability) -> Result> { + let res = self.rpc.rpc(ImportRequest { capability }).await??; + let doc = Doc::new(self.rpc.clone(), res.doc_id); + Ok(doc) + } + + /// Imports a document from a ticket and joins all peers in the ticket. + pub async fn import(&self, ticket: DocTicket) -> Result> { + let DocTicket { capability, nodes } = ticket; + let doc = self.import_namespace(capability).await?; + doc.start_sync(nodes).await?; + Ok(doc) + } + + /// Imports a document from a ticket, creates a subscription stream and joins all peers in the ticket. + /// + /// Returns the [`Doc`] and a [`Stream`] of [`LiveEvent`]s. + /// + /// The subscription stream is created before the sync is started, so the first call to this + /// method after starting the node is guaranteed to not miss any sync events. + pub async fn import_and_subscribe( + &self, + ticket: DocTicket, + ) -> Result<(Doc, impl Stream>)> { + let DocTicket { capability, nodes } = ticket; + let res = self.rpc.rpc(ImportRequest { capability }).await??; + let doc = Doc::new(self.rpc.clone(), res.doc_id); + let events = doc.subscribe().await?; + doc.start_sync(nodes).await?; + Ok((doc, events)) + } + + /// Lists all documents. + pub async fn list(&self) -> Result>> { + let stream = self.rpc.server_streaming(DocListRequest {}).await?; + Ok(flatten(stream).map(|res| res.map(|res| (res.id, res.capability)))) + } + + /// Returns a [`Doc`] client for a single document. + /// + /// Returns None if the document cannot be found. + pub async fn open(&self, id: NamespaceId) -> Result>> { + self.rpc.rpc(OpenRequest { doc_id: id }).await??; + let doc = Doc::new(self.rpc.clone(), id); + Ok(Some(doc)) + } +} + +/// Document handle +#[derive(Debug, Clone)] +pub struct Doc = BoxedConnector>(Arc>) +where + C: quic_rpc::Connector; + +impl> PartialEq for Doc { + fn eq(&self, other: &Self) -> bool { + self.0.id == other.0.id + } +} + +impl> Eq for Doc {} + +#[derive(Debug)] +struct DocInner = BoxedConnector> { + id: NamespaceId, + rpc: quic_rpc::RpcClient, + closed: AtomicBool, + rt: tokio::runtime::Handle, +} + +impl Drop for DocInner +where + C: quic_rpc::Connector, +{ + fn drop(&mut self) { + let doc_id = self.id; + let rpc = self.rpc.clone(); + if !self.closed.swap(true, Ordering::Relaxed) { + self.rt.spawn(async move { + rpc.rpc(CloseRequest { doc_id }).await.ok(); + }); + } + } +} + +impl> Doc { + fn new(rpc: quic_rpc::RpcClient, id: NamespaceId) -> Self { + Self(Arc::new(DocInner { + rpc, + id, + closed: AtomicBool::new(false), + rt: tokio::runtime::Handle::current(), + })) + } + + async fn rpc(&self, msg: M) -> Result + where + M: RpcMsg, + { + let res = self.0.rpc.rpc(msg).await?; + Ok(res) + } + + /// Returns the document id of this doc. + pub fn id(&self) -> NamespaceId { + self.0.id + } + + /// Closes the document. + pub async fn close(&self) -> Result<()> { + if !self.0.closed.swap(true, Ordering::Relaxed) { + self.rpc(CloseRequest { doc_id: self.id() }).await??; + } + Ok(()) + } + + fn ensure_open(&self) -> Result<()> { + if self.0.closed.load(Ordering::Relaxed) { + Err(anyhow!("document is closed")) + } else { + Ok(()) + } + } + + /// Sets the content of a key to a byte array. + pub async fn set_bytes( + &self, + author_id: AuthorId, + key: impl Into, + value: impl Into, + ) -> Result { + self.ensure_open()?; + let res = self + .rpc(SetRequest { + doc_id: self.id(), + author_id, + key: key.into(), + value: value.into(), + }) + .await??; + Ok(res.entry.content_hash()) + } + + /// Sets an entries on the doc via its key, hash, and size. + pub async fn set_hash( + &self, + author_id: AuthorId, + key: impl Into, + hash: Hash, + size: u64, + ) -> Result<()> { + self.ensure_open()?; + self.rpc(SetHashRequest { + doc_id: self.id(), + author_id, + key: key.into(), + hash, + size, + }) + .await??; + Ok(()) + } + + /// Adds an entry from an absolute file path + pub async fn import_file( + &self, + author: AuthorId, + key: Bytes, + path: impl AsRef, + in_place: bool, + ) -> Result { + self.ensure_open()?; + let stream = self + .0 + .rpc + .server_streaming(ImportFileRequest { + doc_id: self.id(), + author_id: author, + path: path.as_ref().into(), + key, + in_place, + }) + .await?; + Ok(ImportFileProgress::new(stream)) + } + + /// Exports an entry as a file to a given absolute path. + pub async fn export_file( + &self, + entry: Entry, + path: impl AsRef, + mode: ExportMode, + ) -> Result { + self.ensure_open()?; + let stream = self + .0 + .rpc + .server_streaming(ExportFileRequest { + entry: entry.0, + path: path.as_ref().into(), + mode, + }) + .await?; + Ok(ExportFileProgress::new(stream)) + } + + /// Deletes entries that match the given `author` and key `prefix`. + /// + /// This inserts an empty entry with the key set to `prefix`, effectively clearing all other + /// entries whose key starts with or is equal to the given `prefix`. + /// + /// Returns the number of entries deleted. + pub async fn del(&self, author_id: AuthorId, prefix: impl Into) -> Result { + self.ensure_open()?; + let res = self + .rpc(DelRequest { + doc_id: self.id(), + author_id, + prefix: prefix.into(), + }) + .await??; + let DelResponse { removed } = res; + Ok(removed) + } + + /// Returns an entry for a key and author. + /// + /// Optionally also returns the entry unless it is empty (i.e. a deletion marker). + pub async fn get_exact( + &self, + author: AuthorId, + key: impl AsRef<[u8]>, + include_empty: bool, + ) -> Result> { + self.ensure_open()?; + let res = self + .rpc(GetExactRequest { + author, + key: key.as_ref().to_vec().into(), + doc_id: self.id(), + include_empty, + }) + .await??; + Ok(res.entry.map(|entry| entry.into())) + } + + /// Returns all entries matching the query. + pub async fn get_many( + &self, + query: impl Into, + ) -> Result>> { + self.ensure_open()?; + let stream = self + .0 + .rpc + .server_streaming(GetManyRequest { + doc_id: self.id(), + query: query.into(), + }) + .await?; + Ok(flatten(stream).map(|res| res.map(|res| res.entry.into()))) + } + + /// Returns a single entry. + pub async fn get_one(&self, query: impl Into) -> Result> { + self.get_many(query).await?.next().await.transpose() + } + + /// Shares this document with peers over a ticket. + pub async fn share( + &self, + mode: ShareMode, + addr_options: AddrInfoOptions, + ) -> anyhow::Result { + self.ensure_open()?; + let res = self + .rpc(ShareRequest { + doc_id: self.id(), + mode, + addr_options, + }) + .await??; + Ok(res.0) + } + + /// Starts to sync this document with a list of peers. + pub async fn start_sync(&self, peers: Vec) -> Result<()> { + self.ensure_open()?; + let _res = self + .rpc(StartSyncRequest { + doc_id: self.id(), + peers, + }) + .await??; + Ok(()) + } + + /// Stops the live sync for this document. + pub async fn leave(&self) -> Result<()> { + self.ensure_open()?; + let _res = self.rpc(LeaveRequest { doc_id: self.id() }).await??; + Ok(()) + } + + /// Subscribes to events for this document. + pub async fn subscribe(&self) -> anyhow::Result>> { + self.ensure_open()?; + let stream = self + .0 + .rpc + .try_server_streaming(DocSubscribeRequest { doc_id: self.id() }) + .await?; + Ok(stream.map(|res| match res { + Ok(res) => Ok(res.event.into()), + Err(err) => Err(err.into()), + })) + } + + /// Returns status info for this document + pub async fn status(&self) -> anyhow::Result { + self.ensure_open()?; + let res = self.rpc(StatusRequest { doc_id: self.id() }).await??; + Ok(res.status) + } + + /// Sets the download policy for this document + pub async fn set_download_policy(&self, policy: DownloadPolicy) -> Result<()> { + self.rpc(SetDownloadPolicyRequest { + doc_id: self.id(), + policy, + }) + .await??; + Ok(()) + } + + /// Returns the download policy for this document + pub async fn get_download_policy(&self) -> Result { + let res = self + .rpc(GetDownloadPolicyRequest { doc_id: self.id() }) + .await??; + Ok(res.policy) + } + + /// Returns sync peers for this document + pub async fn get_sync_peers(&self) -> Result>> { + let res = self + .rpc(GetSyncPeersRequest { doc_id: self.id() }) + .await??; + Ok(res.peers) + } +} + +impl<'a, C> From<&'a Doc> for &'a quic_rpc::RpcClient +where + C: quic_rpc::Connector, +{ + fn from(doc: &'a Doc) -> &'a quic_rpc::RpcClient { + &doc.0.rpc + } +} + +/// A single entry in a [`Doc`]. +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq)] +pub struct Entry(crate::Entry); + +impl From for Entry { + fn from(value: crate::Entry) -> Self { + Self(value) + } +} + +impl From for Entry { + fn from(value: crate::SignedEntry) -> Self { + Self(value.into()) + } +} + +impl Entry { + /// Returns the [`RecordIdentifier`] for this entry. + pub fn id(&self) -> &RecordIdentifier { + self.0.id() + } + + /// Returns the [`AuthorId`] of this entry. + pub fn author(&self) -> AuthorId { + self.0.author() + } + + /// Returns the [`struct@Hash`] of the content data of this record. + pub fn content_hash(&self) -> Hash { + self.0.content_hash() + } + + /// Returns the length of the data addressed by this record's content hash. + pub fn content_len(&self) -> u64 { + self.0.content_len() + } + + /// Returns the key of this entry. + pub fn key(&self) -> &[u8] { + self.0.key() + } + + /// Returns the timestamp of this entry. + pub fn timestamp(&self) -> u64 { + self.0.timestamp() + } +} + +/// Progress messages for an doc import operation +/// +/// An import operation involves computing the outboard of a file, and then +/// either copying or moving the file into the database, then setting the author, hash, size, and tag of that +/// file as an entry in the doc. +#[derive(Debug, Serialize, Deserialize)] +pub enum ImportProgress { + /// An item was found with name `name`, from now on referred to via `id`. + Found { + /// A new unique id for this entry. + id: u64, + /// The name of the entry. + name: String, + /// The size of the entry in bytes. + size: u64, + }, + /// We got progress ingesting item `id`. + Progress { + /// The unique id of the entry. + id: u64, + /// The offset of the progress, in bytes. + offset: u64, + }, + /// We are done adding `id` to the data store and the hash is `hash`. + IngestDone { + /// The unique id of the entry. + id: u64, + /// The hash of the entry. + hash: Hash, + }, + /// We are done setting the entry to the doc. + AllDone { + /// The key of the entry + key: Bytes, + }, + /// We got an error and need to abort. + /// + /// This will be the last message in the stream. + Abort(serde_error::Error), +} + +/// Intended capability for document share tickets +#[derive(Serialize, Deserialize, Debug, Clone, Display, FromStr)] +pub enum ShareMode { + /// Read-only access + Read, + /// Write access + Write, +} + +/// Events informing about actions of the live sync progress. +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, strum::Display)] +pub enum LiveEvent { + /// A local insertion. + InsertLocal { + /// The inserted entry. + entry: Entry, + }, + /// Received a remote insert. + InsertRemote { + /// The peer that sent us the entry. + from: PublicKey, + /// The inserted entry. + entry: Entry, + /// If the content is available at the local node + content_status: ContentStatus, + }, + /// The content of an entry was downloaded and is now available at the local node + ContentReady { + /// The content hash of the newly available entry content + hash: Hash, + }, + /// We have a new neighbor in the swarm. + NeighborUp(PublicKey), + /// We lost a neighbor in the swarm. + NeighborDown(PublicKey), + /// A set-reconciliation sync finished. + SyncFinished(SyncEvent), + /// All pending content is now ready. + /// + /// This event signals that all queued content downloads from the last sync run have either + /// completed or failed. + /// + /// It will only be emitted after a [`Self::SyncFinished`] event, never before. + /// + /// Receiving this event does not guarantee that all content in the document is available. If + /// blobs failed to download, this event will still be emitted after all operations completed. + PendingContentReady, +} + +impl From for LiveEvent { + fn from(event: crate::engine::LiveEvent) -> LiveEvent { + match event { + crate::engine::LiveEvent::InsertLocal { entry } => Self::InsertLocal { + entry: entry.into(), + }, + crate::engine::LiveEvent::InsertRemote { + from, + entry, + content_status, + } => Self::InsertRemote { + from, + content_status, + entry: entry.into(), + }, + crate::engine::LiveEvent::ContentReady { hash } => Self::ContentReady { hash }, + crate::engine::LiveEvent::NeighborUp(node) => Self::NeighborUp(node), + crate::engine::LiveEvent::NeighborDown(node) => Self::NeighborDown(node), + crate::engine::LiveEvent::SyncFinished(details) => Self::SyncFinished(details), + crate::engine::LiveEvent::PendingContentReady => Self::PendingContentReady, + } + } +} + +/// Progress stream for [`Doc::import_file`]. +#[derive(derive_more::Debug)] +#[must_use = "streams do nothing unless polled"] +pub struct ImportFileProgress { + #[debug(skip)] + stream: Pin> + Send + Unpin + 'static>>, +} + +impl ImportFileProgress { + fn new( + stream: (impl Stream, impl Into>> + + Send + + Unpin + + 'static), + ) -> Self { + let stream = stream.map(|item| match item { + Ok(item) => Ok(item.into()), + Err(err) => Err(err.into()), + }); + Self { + stream: Box::pin(stream), + } + } + + /// Finishes writing the stream, ignoring all intermediate progress events. + /// + /// Returns a [`ImportFileOutcome`] which contains a tag, key, and hash and the size of the + /// content. + pub async fn finish(mut self) -> Result { + let mut entry_size = 0; + let mut entry_hash = None; + while let Some(msg) = self.next().await { + match msg? { + ImportProgress::Found { size, .. } => { + entry_size = size; + } + ImportProgress::AllDone { key } => { + let hash = entry_hash + .context("expected DocImportProgress::IngestDone event to occur")?; + let outcome = ImportFileOutcome { + hash, + key, + size: entry_size, + }; + return Ok(outcome); + } + ImportProgress::Abort(err) => return Err(err.into()), + ImportProgress::Progress { .. } => {} + ImportProgress::IngestDone { hash, .. } => { + entry_hash = Some(hash); + } + } + } + Err(anyhow!("Response stream ended prematurely")) + } +} + +/// Outcome of a [`Doc::import_file`] operation +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ImportFileOutcome { + /// The hash of the entry's content + pub hash: Hash, + /// The size of the entry + pub size: u64, + /// The key of the entry + pub key: Bytes, +} + +impl Stream for ImportFileProgress { + type Item = Result; + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut self.stream).poll_next(cx) + } +} + +/// Progress stream for [`Doc::export_file`]. +#[derive(derive_more::Debug)] +pub struct ExportFileProgress { + #[debug(skip)] + stream: Pin> + Send + Unpin + 'static>>, +} +impl ExportFileProgress { + fn new( + stream: (impl Stream, impl Into>> + + Send + + Unpin + + 'static), + ) -> Self { + let stream = stream.map(|item| match item { + Ok(item) => Ok(item.into()), + Err(err) => Err(err.into()), + }); + Self { + stream: Box::pin(stream), + } + } + + /// Iterates through the export progress stream, returning when the stream has completed. + /// + /// Returns a [`ExportFileOutcome`] which contains a file path the data was written to and the size of the content. + pub async fn finish(mut self) -> Result { + let mut total_size = 0; + let mut path = None; + while let Some(msg) = self.next().await { + match msg? { + ExportProgress::Found { size, outpath, .. } => { + total_size = size.value(); + path = Some(outpath); + } + ExportProgress::AllDone => { + let path = path.context("expected ExportProgress::Found event to occur")?; + let outcome = ExportFileOutcome { + size: total_size, + path, + }; + return Ok(outcome); + } + ExportProgress::Done { .. } => {} + ExportProgress::Abort(err) => return Err(anyhow!(err)), + ExportProgress::Progress { .. } => {} + } + } + Err(anyhow!("Response stream ended prematurely")) + } +} + +/// Outcome of a [`Doc::export_file`] operation +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ExportFileOutcome { + /// The size of the entry + pub size: u64, + /// The path to which the entry was saved + pub path: PathBuf, +} + +impl Stream for ExportFileProgress { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + Pin::new(&mut self.stream).poll_next(cx) + } +} + +fn flatten( + s: impl Stream, E2>>, +) -> impl Stream> +where + E1: std::error::Error + Send + Sync + 'static, + E2: std::error::Error + Send + Sync + 'static, +{ + s.map(|res| match res { + Ok(Ok(res)) => Ok(res), + Ok(Err(err)) => Err(err.into()), + Err(err) => Err(err.into()), + }) +} + +#[cfg(test)] +mod tests { + use iroh_blobs::util::local_pool::LocalPool; + use iroh_gossip::net::GOSSIP_ALPN; + // use rand::RngCore; + // use tokio::io::AsyncWriteExt; + use tracing::warn; + + use super::*; + use crate::{ + engine::{DefaultAuthorStorage, Engine}, + net::DOCS_ALPN, + }; + + async fn setup_router() -> Result<( + Client, + iroh_router::Router, + tokio::task::JoinHandle>, + )> { + let endpoint = iroh_net::Endpoint::builder().bind().await?; + let local_pool = LocalPool::single(); + let mut router = iroh_router::Router::builder(endpoint.clone()); + + // Setup gossip + let my_addr = endpoint.node_addr().await?; + let gossip = iroh_gossip::net::Gossip::from_endpoint( + endpoint.clone(), + Default::default(), + &my_addr.info, + ); + router = router.accept(GOSSIP_ALPN.to_vec(), Arc::new(gossip.clone())); + + // Setup blobs + + let bao_store = iroh_blobs::store::mem::Store::new(); + let downloader = iroh_blobs::downloader::Downloader::new( + bao_store.clone(), + endpoint.clone(), + local_pool.handle().clone(), + ); + let blobs = iroh_blobs::net_protocol::Blobs::new_with_events( + bao_store.clone(), + local_pool.handle().clone(), + Default::default(), + downloader.clone(), + endpoint.clone(), + ); + router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), Arc::new(blobs)); + + // Setup docs + + let replica_store = crate::store::Store::memory(); + let engine = Engine::spawn( + endpoint.clone(), + gossip, + replica_store, + bao_store, + downloader, + DefaultAuthorStorage::Mem, + local_pool.handle().clone(), + ) + .await?; + + router = router.accept(DOCS_ALPN.to_vec(), Arc::new(engine.clone())); + + // Build the router + + let router = router.spawn().await?; + + // Setup RPC + let (internal_rpc, controller) = quic_rpc::transport::flume::channel(32); + let controller = quic_rpc::transport::boxed::BoxedConnector::new(controller); + let internal_rpc = quic_rpc::transport::boxed::BoxedListener::new(internal_rpc); + let internal_rpc = quic_rpc::RpcServer::new(internal_rpc); + + let rpc_server_task = tokio::task::spawn(async move { + loop { + let request = internal_rpc.accept().await; + match request { + Ok(accepting) => { + let engine = engine.clone(); + tokio::task::spawn(async move { + let (msg, chan) = accepting.read_first().await.unwrap(); + engine.handle_rpc_request(msg, chan).await.unwrap(); + }); + } + Err(err) => { + warn!("rpc error: {:?}", err); + } + } + } + }); + + let docs_client = Client::new(quic_rpc::RpcClient::new(controller.clone())); + + Ok((docs_client, router, rpc_server_task)) + } + + #[tokio::test] + async fn test_drop_doc_client_sync() -> Result<()> { + let _guard = iroh_test::logging::setup(); + + let (docs_client, router, rpc_server_task) = setup_router().await?; + + let doc = docs_client.create().await?; + + let res = std::thread::spawn(move || { + drop(doc); + }); + + tokio::task::spawn_blocking(move || res.join().map_err(|e| anyhow::anyhow!("{:?}", e))) + .await??; + + rpc_server_task.abort(); + router.shutdown().await?; + + Ok(()) + } +} diff --git a/src/rpc/docs_handle_request.rs b/src/rpc/docs_handle_request.rs index 6938c32..2c147ff 100644 --- a/src/rpc/docs_handle_request.rs +++ b/src/rpc/docs_handle_request.rs @@ -10,7 +10,7 @@ use iroh_blobs::{ }; use super::{ - client::ShareMode, + client::docs::ShareMode, proto::{ AuthorCreateRequest, AuthorCreateResponse, AuthorDeleteRequest, AuthorDeleteResponse, AuthorExportRequest, AuthorExportResponse, AuthorGetDefaultRequest, @@ -419,9 +419,11 @@ impl Engine { let this = self.clone(); self.local_pool_handle().spawn_detached(|| async move { if let Err(e) = this.doc_import_file0(msg, tx).await { - tx2.send(super::client::ImportProgress::Abort(RpcError::new(&*e))) - .await - .ok(); + tx2.send(super::client::docs::ImportProgress::Abort(RpcError::new( + &*e, + ))) + .await + .ok(); } }); rx.map(ImportFileResponse) @@ -430,13 +432,13 @@ impl Engine { async fn doc_import_file0( self, msg: ImportFileRequest, - progress: async_channel::Sender, + progress: async_channel::Sender, ) -> anyhow::Result<()> { use std::collections::BTreeMap; use iroh_blobs::store::ImportMode; - use super::client::ImportProgress as DocImportProgress; + use super::client::docs::ImportProgress as DocImportProgress; let progress = AsyncChannelProgressSender::new(progress); let names = Arc::new(Mutex::new(BTreeMap::new())); diff --git a/src/rpc/proto.rs b/src/rpc/proto.rs index 4b0581a..fc379fe 100644 --- a/src/rpc/proto.rs +++ b/src/rpc/proto.rs @@ -12,7 +12,7 @@ use quic_rpc_derive::rpc_requests; use serde::{Deserialize, Serialize}; use super::{ - client::{ImportProgress, ShareMode}, + client::docs::{ImportProgress, ShareMode}, RpcError, RpcResult, }; use crate::{ diff --git a/tests/client.rs b/tests/client.rs index 16a4389..6aacd05 100644 --- a/tests/client.rs +++ b/tests/client.rs @@ -19,7 +19,7 @@ async fn test_doc_close() -> Result<()> { let _guard = iroh_test::logging::setup(); let node = Node::memory().spawn().await?; - let author = node.docs().author_default().await?; + let author = node.authors().default().await?; // open doc two times let doc1 = node.docs().create().await?; let doc2 = node.docs().open(doc1.id()).await?.expect("doc to exist"); @@ -67,7 +67,7 @@ async fn test_doc_import_export() -> TestResult<()> { let client = node.client(); let docs_client = client.docs(); let doc = docs_client.create().await.context("doc create")?; - let author = docs_client.author_create().await.context("author create")?; + let author = client.authors().create().await.context("author create")?; // import file let import_outcome = doc @@ -115,33 +115,33 @@ async fn test_authors() -> Result<()> { let node = Node::memory().spawn().await?; // default author always exists - let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + let authors: Vec<_> = node.authors().list().await?.try_collect().await?; assert_eq!(authors.len(), 1); - let default_author = node.docs().author_default().await?; + let default_author = node.authors().default().await?; assert_eq!(authors, vec![default_author]); - let author_id = node.docs().author_create().await?; + let author_id = node.authors().create().await?; - let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + let authors: Vec<_> = node.authors().list().await?.try_collect().await?; assert_eq!(authors.len(), 2); let author = node - .docs() - .author_export(author_id) + .authors() + .export(author_id) .await? .expect("should have author"); - node.docs().author_delete(author_id).await?; - let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + node.authors().delete(author_id).await?; + let authors: Vec<_> = node.authors().list().await?.try_collect().await?; assert_eq!(authors.len(), 1); - node.docs().author_import(author).await?; + node.authors().import(author).await?; - let authors: Vec<_> = node.docs().author_list().await?.try_collect().await?; + let authors: Vec<_> = node.authors().list().await?.try_collect().await?; assert_eq!(authors.len(), 2); - assert!(node.docs().author_default().await? != author_id); - node.docs().author_set_default(author_id).await?; - assert_eq!(node.docs().author_default().await?, author_id); + assert!(node.authors().default().await? != author_id); + node.authors().set_default(author_id).await?; + assert_eq!(node.authors().default().await?, author_id); Ok(()) } @@ -149,9 +149,9 @@ async fn test_authors() -> Result<()> { #[tokio::test] async fn test_default_author_memory() -> Result<()> { let iroh = Node::memory().spawn().await?; - let author = iroh.docs().author_default().await?; - assert!(iroh.docs().author_export(author).await?.is_some()); - assert!(iroh.docs().author_delete(author).await.is_err()); + let author = iroh.authors().default().await?; + assert!(iroh.authors().export(author).await?.is_some()); + assert!(iroh.authors().delete(author).await.is_err()); Ok(()) } @@ -165,9 +165,9 @@ async fn test_default_author_persist() -> TestResult<()> { // check that the default author exists and cannot be deleted. let default_author = { let iroh = Node::persistent(iroh_root).spawn().await?; - let author = iroh.docs().author_default().await?; - assert!(iroh.docs().author_export(author).await?.is_some()); - assert!(iroh.docs().author_delete(author).await.is_err()); + let author = iroh.authors().default().await?; + assert!(iroh.authors().export(author).await?.is_some()); + assert!(iroh.authors().delete(author).await.is_err()); iroh.shutdown().await?; author }; @@ -175,10 +175,10 @@ async fn test_default_author_persist() -> TestResult<()> { // check that the default author is persisted across restarts. { let iroh = Node::persistent(iroh_root).spawn().await?; - let author = iroh.docs().author_default().await?; + let author = iroh.authors().default().await?; assert_eq!(author, default_author); - assert!(iroh.docs().author_export(author).await?.is_some()); - assert!(iroh.docs().author_delete(author).await.is_err()); + assert!(iroh.authors().export(author).await?.is_some()); + assert!(iroh.authors().delete(author).await.is_err()); iroh.shutdown().await?; }; @@ -187,10 +187,10 @@ async fn test_default_author_persist() -> TestResult<()> { let default_author = { tokio::fs::remove_file(iroh_root.join("default-author")).await?; let iroh = Node::persistent(iroh_root).spawn().await?; - let author = iroh.docs().author_default().await?; + let author = iroh.authors().default().await?; assert!(author != default_author); - assert!(iroh.docs().author_export(author).await?.is_some()); - assert!(iroh.docs().author_delete(author).await.is_err()); + assert!(iroh.authors().export(author).await?.is_some()); + assert!(iroh.authors().delete(author).await.is_err()); iroh.shutdown().await?; author }; @@ -221,15 +221,15 @@ async fn test_default_author_persist() -> TestResult<()> { // check that the default author can be set manually and is persisted. let default_author = { let iroh = Node::persistent(iroh_root).spawn().await?; - let author = iroh.docs().author_create().await?; - iroh.docs().author_set_default(author).await?; - assert_eq!(iroh.docs().author_default().await?, author); + let author = iroh.authors().create().await?; + iroh.authors().set_default(author).await?; + assert_eq!(iroh.authors().default().await?, author); iroh.shutdown().await?; author }; { let iroh = Node::persistent(iroh_root).spawn().await?; - assert_eq!(iroh.docs().author_default().await?, default_author); + assert_eq!(iroh.authors().default().await?, default_author); iroh.shutdown().await?; } diff --git a/tests/gc.rs b/tests/gc.rs index 94857e3..09de6c3 100644 --- a/tests/gc.rs +++ b/tests/gc.rs @@ -254,7 +254,7 @@ mod file { let bao_store = node.blob_store().clone(); let client = node.client(); let doc = client.docs().create().await?; - let author = client.docs().author_create().await?; + let author = client.authors().create().await?; let temp_path = dir.join("temp"); tokio::fs::create_dir_all(&temp_path).await?; let mut to_import = Vec::new(); diff --git a/tests/sync.rs b/tests/sync.rs index 83c88c6..21d4875 100644 --- a/tests/sync.rs +++ b/tests/sync.rs @@ -13,7 +13,7 @@ use futures_util::{FutureExt, StreamExt, TryStreamExt}; use iroh_base::node_addr::AddrInfoOptions; use iroh_blobs::Hash; use iroh_docs::{ - rpc::client::{Doc, Entry, LiveEvent, ShareMode}, + rpc::client::docs::{Doc, Entry, LiveEvent, ShareMode}, store::{DownloadPolicy, FilterKind, Query}, AuthorId, ContentStatus, }; @@ -82,7 +82,7 @@ async fn sync_simple() -> Result<()> { // create doc on node0 let peer0 = nodes[0].node_id(); - let author0 = clients[0].docs().author_create().await?; + let author0 = clients[0].authors().create().await?; let doc0 = clients[0].docs().create().await?; let blobs0 = clients[0].blobs(); let hash0 = doc0 @@ -141,7 +141,7 @@ async fn sync_subscribe_no_sync() -> Result<()> { let client = node.client(); let doc = client.docs().create().await?; let mut sub = doc.subscribe().await?; - let author = client.docs().author_create().await?; + let author = client.authors().create().await?; doc.set_bytes(author, b"k".to_vec(), b"v".to_vec()).await?; let event = tokio::time::timeout(Duration::from_millis(100), sub.next()).await?; assert!( @@ -164,7 +164,7 @@ async fn sync_gossip_bulk() -> Result<()> { let clients = nodes.iter().map(|node| node.client()).collect::>(); let _peer0 = nodes[0].node_id(); - let author0 = clients[0].docs().author_create().await?; + let author0 = clients[0].authors().create().await?; let doc0 = clients[0].docs().create().await?; let mut ticket = doc0 .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) @@ -255,7 +255,7 @@ async fn sync_full_basic() -> testresult::TestResult<()> { // peer0: create doc and ticket let peer0 = nodes[0].node_id(); - let author0 = clients[0].docs().author_create().await?; + let author0 = clients[0].authors().create().await?; let doc0 = clients[0].docs().create().await?; let blobs0 = clients[0].blobs(); let mut events0 = doc0.subscribe().await?; @@ -278,7 +278,7 @@ async fn sync_full_basic() -> testresult::TestResult<()> { info!("peer1: spawn"); let peer1 = nodes[1].node_id(); - let author1 = clients[1].docs().author_create().await?; + let author1 = clients[1].authors().create().await?; info!("peer1: join doc"); let doc1 = clients[1].docs().import(ticket.clone()).await?; let blobs1 = clients[1].blobs(); @@ -456,7 +456,7 @@ async fn sync_subscribe_stop_close() -> Result<()> { let client = node.client(); let doc = client.docs().create().await?; - let author = client.docs().author_create().await?; + let author = client.authors().create().await?; let status = doc.status().await?; assert_eq!(status.subscribers, 0); @@ -507,7 +507,7 @@ async fn test_sync_via_relay() -> Result<()> { .await?; let doc1 = node1.docs().create().await?; - let author1 = node1.docs().author_create().await?; + let author1 = node1.authors().create().await?; let inserted_hash = doc1 .set_bytes(author1, b"foo".to_vec(), b"bar".to_vec()) .await?; @@ -623,7 +623,7 @@ async fn sync_restart_node() -> Result<()> { .spawn() .await?; let id2 = node2.node_id(); - let author2 = node2.docs().author_create().await?; + let author2 = node2.authors().create().await?; let doc2 = node2.docs().import(ticket.clone()).await?; let blobs2 = node2.blobs(); @@ -758,13 +758,13 @@ async fn test_download_policies() -> Result<()> { let clients = nodes.iter().map(|node| node.client()).collect::>(); let doc_a = clients[0].docs().create().await?; - let author_a = clients[0].docs().author_create().await?; + let author_a = clients[0].authors().create().await?; let ticket = doc_a .share(ShareMode::Write, AddrInfoOptions::RelayAndAddresses) .await?; let doc_b = clients[1].docs().import(ticket).await?; - let author_b = clients[1].docs().author_create().await?; + let author_b = clients[1].authors().create().await?; doc_a.set_download_policy(policy_a).await?; doc_b.set_download_policy(policy_b).await?; @@ -880,7 +880,7 @@ async fn sync_big() -> Result<()> { let nodes = spawn_nodes(n_nodes, &mut rng).await?; let node_ids = nodes.iter().map(|node| node.node_id()).collect::>(); let clients = nodes.iter().map(|node| node.client()).collect::>(); - let authors = collect_futures(clients.iter().map(|c| c.docs().author_create())).await?; + let authors = collect_futures(clients.iter().map(|c| c.authors().create())).await?; let doc0 = clients[0].docs().create().await?; let mut ticket = doc0 @@ -1165,7 +1165,7 @@ async fn doc_delete() -> Result<()> { let client = node.client(); let doc = client.docs().create().await?; let blobs = client.blobs(); - let author = client.docs().author_create().await?; + let author = client.authors().create().await?; let hash = doc .set_bytes(author, b"foo".to_vec(), b"hi".to_vec()) .await?; @@ -1193,7 +1193,7 @@ async fn sync_drop_doc() -> Result<()> { let client = node.client(); let doc = client.docs().create().await?; - let author = client.docs().author_create().await?; + let author = client.authors().create().await?; let mut sub = doc.subscribe().await?; doc.set_bytes(author, b"foo".to_vec(), b"bar".to_vec()) diff --git a/tests/util.rs b/tests/util.rs index e42d277..ed21384 100644 --- a/tests/util.rs +++ b/tests/util.rs @@ -75,14 +75,16 @@ impl quic_rpc::Service for Service { #[derive(Debug, Clone)] pub struct Client { blobs: iroh_blobs::rpc::client::blobs::Client, - docs: iroh_docs::rpc::client::Client, + docs: iroh_docs::rpc::client::docs::Client, + authors: iroh_docs::rpc::client::authors::Client, } impl Client { fn new(client: quic_rpc::RpcClient) -> Self { Self { blobs: iroh_blobs::rpc::client::blobs::Client::new(client.clone().map().boxed()), - docs: iroh_docs::rpc::client::Client::new(client.map().boxed()), + docs: iroh_docs::rpc::client::docs::Client::new(client.clone().map().boxed()), + authors: iroh_docs::rpc::client::authors::Client::new(client.map().boxed()), } } @@ -90,9 +92,13 @@ impl Client { &self.blobs } - pub fn docs(&self) -> &iroh_docs::rpc::client::Client { + pub fn docs(&self) -> &iroh_docs::rpc::client::docs::Client { &self.docs } + + pub fn authors(&self) -> &iroh_docs::rpc::client::authors::Client { + &self.authors + } } /// An iroh node builder From ce59f9b0e4f285c72e98b7260d1285e2fe484eb4 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Tue, 12 Nov 2024 17:15:00 +0200 Subject: [PATCH 21/26] fix docs --- src/rpc/client/authors.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/rpc/client/authors.rs b/src/rpc/client/authors.rs index 64c85ea..be21f7b 100644 --- a/src/rpc/client/authors.rs +++ b/src/rpc/client/authors.rs @@ -33,7 +33,7 @@ impl> Client { /// You likely want to save the returned [`AuthorId`] somewhere so that you can use this author /// again. /// - /// If you need only a single author, use [`Self::author_default`]. + /// If you need only a single author, use [`Self::default`]. pub async fn create(&self) -> Result { let res = self.rpc.rpc(AuthorCreateRequest).await??; Ok(res.author_id) @@ -44,7 +44,7 @@ impl> Client { /// On persistent nodes, the author is created on first start and its public key is saved /// in the data directory. /// - /// The default author can be set with [`Self::author_set_default`]. + /// The default author can be set with [`Self::set_default`]. pub async fn default(&self) -> Result { let res = self.rpc.rpc(AuthorGetDefaultRequest).await??; Ok(res.author_id) From ac68d3966d55eb9e5cb5a1f0eaea1e4165247003 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Wed, 13 Nov 2024 10:58:33 +0200 Subject: [PATCH 22/26] dedup flatten --- src/rpc/client.rs | 17 +++++++++++++++++ src/rpc/client/authors.rs | 16 ++-------------- src/rpc/client/docs.rs | 16 ++-------------- 3 files changed, 21 insertions(+), 28 deletions(-) diff --git a/src/rpc/client.rs b/src/rpc/client.rs index f9b2d7d..90a0ad7 100644 --- a/src/rpc/client.rs +++ b/src/rpc/client.rs @@ -1,3 +1,20 @@ //! RPC Client for docs and authors +use anyhow::Result; +use futures_util::{Stream, StreamExt}; + pub mod authors; pub mod docs; + +fn flatten( + s: impl Stream, E2>>, +) -> impl Stream> +where + E1: std::error::Error + Send + Sync + 'static, + E2: std::error::Error + Send + Sync + 'static, +{ + s.map(|res| match res { + Ok(Ok(res)) => Ok(res), + Ok(Err(err)) => Err(err.into()), + Err(err) => Err(err.into()), + }) +} diff --git a/src/rpc/client/authors.rs b/src/rpc/client/authors.rs index be21f7b..611690e 100644 --- a/src/rpc/client/authors.rs +++ b/src/rpc/client/authors.rs @@ -16,6 +16,8 @@ use crate::{ Author, AuthorId, }; +use super::flatten; + /// Iroh docs client. #[derive(Debug, Clone)] pub struct Client> { @@ -97,17 +99,3 @@ impl> Client { Ok(()) } } - -fn flatten( - s: impl Stream, E2>>, -) -> impl Stream> -where - E1: std::error::Error + Send + Sync + 'static, - E2: std::error::Error + Send + Sync + 'static, -{ - s.map(|res| match res { - Ok(Ok(res)) => Ok(res), - Ok(Err(err)) => Err(err.into()), - Err(err) => Err(err.into()), - }) -} diff --git a/src/rpc/client/docs.rs b/src/rpc/client/docs.rs index 0b1d43c..57017b5 100644 --- a/src/rpc/client/docs.rs +++ b/src/rpc/client/docs.rs @@ -35,6 +35,8 @@ use crate::{ RecordIdentifier, }; +use super::flatten; + /// Iroh docs client. #[derive(Debug, Clone)] pub struct Client> { @@ -733,20 +735,6 @@ impl Stream for ExportFileProgress { } } -fn flatten( - s: impl Stream, E2>>, -) -> impl Stream> -where - E1: std::error::Error + Send + Sync + 'static, - E2: std::error::Error + Send + Sync + 'static, -{ - s.map(|res| match res { - Ok(Ok(res)) => Ok(res), - Ok(Err(err)) => Err(err.into()), - Err(err) => Err(err.into()), - }) -} - #[cfg(test)] mod tests { use iroh_blobs::util::local_pool::LocalPool; From ac9400ac76e6fefc6ef66f413c8a2573d2325e61 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Wed, 13 Nov 2024 11:07:43 +0200 Subject: [PATCH 23/26] Remove redundant tests --- src/rpc/client/authors.rs | 3 +- src/rpc/client/docs.rs | 123 +------------------------------------- 2 files changed, 2 insertions(+), 124 deletions(-) diff --git a/src/rpc/client/authors.rs b/src/rpc/client/authors.rs index 611690e..3529abb 100644 --- a/src/rpc/client/authors.rs +++ b/src/rpc/client/authors.rs @@ -6,6 +6,7 @@ use anyhow::Result; use futures_lite::{Stream, StreamExt}; use quic_rpc::{client::BoxedConnector, Connector}; +use super::flatten; #[doc(inline)] pub use crate::engine::{Origin, SyncEvent, SyncReason}; use crate::{ @@ -16,8 +17,6 @@ use crate::{ Author, AuthorId, }; -use super::flatten; - /// Iroh docs client. #[derive(Debug, Clone)] pub struct Client> { diff --git a/src/rpc/client/docs.rs b/src/rpc/client/docs.rs index 57017b5..333813d 100644 --- a/src/rpc/client/docs.rs +++ b/src/rpc/client/docs.rs @@ -19,6 +19,7 @@ use portable_atomic::{AtomicBool, Ordering}; use quic_rpc::{client::BoxedConnector, message::RpcMsg, Connector}; use serde::{Deserialize, Serialize}; +use super::flatten; #[doc(inline)] pub use crate::engine::{Origin, SyncEvent, SyncReason}; use crate::{ @@ -35,8 +36,6 @@ use crate::{ RecordIdentifier, }; -use super::flatten; - /// Iroh docs client. #[derive(Debug, Clone)] pub struct Client> { @@ -734,123 +733,3 @@ impl Stream for ExportFileProgress { Pin::new(&mut self.stream).poll_next(cx) } } - -#[cfg(test)] -mod tests { - use iroh_blobs::util::local_pool::LocalPool; - use iroh_gossip::net::GOSSIP_ALPN; - // use rand::RngCore; - // use tokio::io::AsyncWriteExt; - use tracing::warn; - - use super::*; - use crate::{ - engine::{DefaultAuthorStorage, Engine}, - net::DOCS_ALPN, - }; - - async fn setup_router() -> Result<( - Client, - iroh_router::Router, - tokio::task::JoinHandle>, - )> { - let endpoint = iroh_net::Endpoint::builder().bind().await?; - let local_pool = LocalPool::single(); - let mut router = iroh_router::Router::builder(endpoint.clone()); - - // Setup gossip - let my_addr = endpoint.node_addr().await?; - let gossip = iroh_gossip::net::Gossip::from_endpoint( - endpoint.clone(), - Default::default(), - &my_addr.info, - ); - router = router.accept(GOSSIP_ALPN.to_vec(), Arc::new(gossip.clone())); - - // Setup blobs - - let bao_store = iroh_blobs::store::mem::Store::new(); - let downloader = iroh_blobs::downloader::Downloader::new( - bao_store.clone(), - endpoint.clone(), - local_pool.handle().clone(), - ); - let blobs = iroh_blobs::net_protocol::Blobs::new_with_events( - bao_store.clone(), - local_pool.handle().clone(), - Default::default(), - downloader.clone(), - endpoint.clone(), - ); - router = router.accept(iroh_blobs::protocol::ALPN.to_vec(), Arc::new(blobs)); - - // Setup docs - - let replica_store = crate::store::Store::memory(); - let engine = Engine::spawn( - endpoint.clone(), - gossip, - replica_store, - bao_store, - downloader, - DefaultAuthorStorage::Mem, - local_pool.handle().clone(), - ) - .await?; - - router = router.accept(DOCS_ALPN.to_vec(), Arc::new(engine.clone())); - - // Build the router - - let router = router.spawn().await?; - - // Setup RPC - let (internal_rpc, controller) = quic_rpc::transport::flume::channel(32); - let controller = quic_rpc::transport::boxed::BoxedConnector::new(controller); - let internal_rpc = quic_rpc::transport::boxed::BoxedListener::new(internal_rpc); - let internal_rpc = quic_rpc::RpcServer::new(internal_rpc); - - let rpc_server_task = tokio::task::spawn(async move { - loop { - let request = internal_rpc.accept().await; - match request { - Ok(accepting) => { - let engine = engine.clone(); - tokio::task::spawn(async move { - let (msg, chan) = accepting.read_first().await.unwrap(); - engine.handle_rpc_request(msg, chan).await.unwrap(); - }); - } - Err(err) => { - warn!("rpc error: {:?}", err); - } - } - } - }); - - let docs_client = Client::new(quic_rpc::RpcClient::new(controller.clone())); - - Ok((docs_client, router, rpc_server_task)) - } - - #[tokio::test] - async fn test_drop_doc_client_sync() -> Result<()> { - let _guard = iroh_test::logging::setup(); - - let (docs_client, router, rpc_server_task) = setup_router().await?; - - let doc = docs_client.create().await?; - - let res = std::thread::spawn(move || { - drop(doc); - }); - - tokio::task::spawn_blocking(move || res.join().map_err(|e| anyhow::anyhow!("{:?}", e))) - .await??; - - rpc_server_task.abort(); - router.shutdown().await?; - - Ok(()) - } -} From c94a76160f44d2a135733b7289130790155b96fe Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Wed, 13 Nov 2024 11:32:44 +0200 Subject: [PATCH 24/26] Reexport LiveEvent instead of duplicating it --- src/lib.rs | 4 +-- src/rpc/client/docs.rs | 71 ++---------------------------------------- 2 files changed, 5 insertions(+), 70 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index 9ebbd23..a11a7d7 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -48,8 +48,8 @@ mod ticket; #[cfg(feature = "engine")] #[cfg_attr(iroh_docsrs, doc(cfg(feature = "engine")))] pub mod engine; -#[cfg(feature = "engine")] -#[cfg_attr(iroh_docsrs, doc(cfg(feature = "engine")))] +#[cfg(feature = "rpc")] +#[cfg_attr(iroh_docsrs, doc(cfg(feature = "rpc")))] pub mod rpc; pub mod actor; diff --git a/src/rpc/client/docs.rs b/src/rpc/client/docs.rs index 333813d..f68107e 100644 --- a/src/rpc/client/docs.rs +++ b/src/rpc/client/docs.rs @@ -12,7 +12,7 @@ use anyhow::{anyhow, Context as _, Result}; use bytes::Bytes; use derive_more::{Display, FromStr}; use futures_lite::{Stream, StreamExt}; -use iroh_base::{key::PublicKey, node_addr::AddrInfoOptions}; +use iroh_base::node_addr::AddrInfoOptions; use iroh_blobs::{export::ExportProgress, store::ExportMode, Hash}; use iroh_net::NodeAddr; use portable_atomic::{AtomicBool, Ordering}; @@ -21,7 +21,7 @@ use serde::{Deserialize, Serialize}; use super::flatten; #[doc(inline)] -pub use crate::engine::{Origin, SyncEvent, SyncReason}; +pub use crate::engine::{Origin, SyncEvent, SyncReason, LiveEvent}; use crate::{ actor::OpenState, rpc::proto::{ @@ -32,7 +32,7 @@ use crate::{ StartSyncRequest, StatusRequest, }, store::{DownloadPolicy, Query}, - AuthorId, Capability, CapabilityKind, ContentStatus, DocTicket, NamespaceId, PeerIdBytes, + AuthorId, Capability, CapabilityKind, DocTicket, NamespaceId, PeerIdBytes, RecordIdentifier, }; @@ -526,71 +526,6 @@ pub enum ShareMode { /// Write access Write, } - -/// Events informing about actions of the live sync progress. -#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq, strum::Display)] -pub enum LiveEvent { - /// A local insertion. - InsertLocal { - /// The inserted entry. - entry: Entry, - }, - /// Received a remote insert. - InsertRemote { - /// The peer that sent us the entry. - from: PublicKey, - /// The inserted entry. - entry: Entry, - /// If the content is available at the local node - content_status: ContentStatus, - }, - /// The content of an entry was downloaded and is now available at the local node - ContentReady { - /// The content hash of the newly available entry content - hash: Hash, - }, - /// We have a new neighbor in the swarm. - NeighborUp(PublicKey), - /// We lost a neighbor in the swarm. - NeighborDown(PublicKey), - /// A set-reconciliation sync finished. - SyncFinished(SyncEvent), - /// All pending content is now ready. - /// - /// This event signals that all queued content downloads from the last sync run have either - /// completed or failed. - /// - /// It will only be emitted after a [`Self::SyncFinished`] event, never before. - /// - /// Receiving this event does not guarantee that all content in the document is available. If - /// blobs failed to download, this event will still be emitted after all operations completed. - PendingContentReady, -} - -impl From for LiveEvent { - fn from(event: crate::engine::LiveEvent) -> LiveEvent { - match event { - crate::engine::LiveEvent::InsertLocal { entry } => Self::InsertLocal { - entry: entry.into(), - }, - crate::engine::LiveEvent::InsertRemote { - from, - entry, - content_status, - } => Self::InsertRemote { - from, - content_status, - entry: entry.into(), - }, - crate::engine::LiveEvent::ContentReady { hash } => Self::ContentReady { hash }, - crate::engine::LiveEvent::NeighborUp(node) => Self::NeighborUp(node), - crate::engine::LiveEvent::NeighborDown(node) => Self::NeighborDown(node), - crate::engine::LiveEvent::SyncFinished(details) => Self::SyncFinished(details), - crate::engine::LiveEvent::PendingContentReady => Self::PendingContentReady, - } - } -} - /// Progress stream for [`Doc::import_file`]. #[derive(derive_more::Debug)] #[must_use = "streams do nothing unless polled"] From 4bc65316aa271c9324e9e111bcea3fff3100da25 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Wed, 13 Nov 2024 11:42:08 +0200 Subject: [PATCH 25/26] fmt --- src/rpc/client/docs.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/rpc/client/docs.rs b/src/rpc/client/docs.rs index f68107e..e3605cd 100644 --- a/src/rpc/client/docs.rs +++ b/src/rpc/client/docs.rs @@ -21,7 +21,7 @@ use serde::{Deserialize, Serialize}; use super::flatten; #[doc(inline)] -pub use crate::engine::{Origin, SyncEvent, SyncReason, LiveEvent}; +pub use crate::engine::{LiveEvent, Origin, SyncEvent, SyncReason}; use crate::{ actor::OpenState, rpc::proto::{ @@ -32,8 +32,7 @@ use crate::{ StartSyncRequest, StatusRequest, }, store::{DownloadPolicy, Query}, - AuthorId, Capability, CapabilityKind, DocTicket, NamespaceId, PeerIdBytes, - RecordIdentifier, + AuthorId, Capability, CapabilityKind, DocTicket, NamespaceId, PeerIdBytes, RecordIdentifier, }; /// Iroh docs client. From 622d1c3feb8b2fe92d8b2ce6dc219d4de49c7c41 Mon Sep 17 00:00:00 2001 From: Ruediger Klaehn Date: Wed, 13 Nov 2024 11:42:48 +0200 Subject: [PATCH 26/26] clippy --- src/rpc/client/docs.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rpc/client/docs.rs b/src/rpc/client/docs.rs index e3605cd..166671e 100644 --- a/src/rpc/client/docs.rs +++ b/src/rpc/client/docs.rs @@ -380,7 +380,7 @@ impl> Doc { .try_server_streaming(DocSubscribeRequest { doc_id: self.id() }) .await?; Ok(stream.map(|res| match res { - Ok(res) => Ok(res.event.into()), + Ok(res) => Ok(res.event), Err(err) => Err(err.into()), })) }