From 6d77db9e52ece0b668587187c59f2862670db7cf Mon Sep 17 00:00:00 2001 From: David Chu Date: Thu, 16 Jan 2025 14:01:27 +0800 Subject: [PATCH 01/29] feat(hydro_std): send_partitioned operator and move decoupling (#1652) Allows specifying a distribution policy (for deciding which partition to send each message to) before networking. Designed to be as easy as possible to inject (so the distribution policy function definition takes in the cluster ID, for example, even though it doesn't need to, because this way we can avoid project->map->join) --- hydro_lang/src/stream.rs | 45 +-------- hydro_std/src/compartmentalize.rs | 121 +++++++++++++++++++++++ hydro_std/src/lib.rs | 1 + hydro_test/Cargo.toml | 1 + hydro_test/src/cluster/simple_cluster.rs | 73 ++++++++++++++ 5 files changed, 197 insertions(+), 44 deletions(-) create mode 100644 hydro_std/src/compartmentalize.rs diff --git a/hydro_lang/src/stream.rs b/hydro_lang/src/stream.rs index a031c2f3831..deffeb4fb35 100644 --- a/hydro_lang/src/stream.rs +++ b/hydro_lang/src/stream.rs @@ -15,14 +15,13 @@ use tokio::time::Instant; use crate::builder::FLOW_USED_MESSAGE; use crate::cycle::{CycleCollection, CycleComplete, DeferTick, ForwardRefMarker, TickCycleMarker}; use crate::ir::{DebugInstantiate, HydroLeaf, HydroNode, TeeNode}; -use crate::location::cluster::CLUSTER_SELF_ID; use crate::location::external_process::{ExternalBincodeStream, ExternalBytesPort}; use crate::location::tick::{NoTimestamp, Timestamped}; use crate::location::{ check_matching_location, CanSend, ExternalProcess, Location, LocationId, NoTick, Tick, }; use crate::staging_util::get_this_crate; -use crate::{Bounded, Cluster, ClusterId, Optional, Process, Singleton, Unbounded}; +use crate::{Bounded, Cluster, ClusterId, Optional, Singleton, Unbounded}; /// Marks the stream as being totally ordered, which means that there are /// no sources of non-determinism (other than intentional ones) that will @@ -1536,49 +1535,7 @@ pub(super) fn deserialize_bincode(tagged: Option } } -impl<'a, T, C1, B, Order> Stream, B, Order> { - pub fn decouple_cluster( - self, - other: &Cluster<'a, C2>, - ) -> Stream, Unbounded, Order> - where - Cluster<'a, C1>: Location<'a, Root = Cluster<'a, C1>>, - Cluster<'a, C1>: - CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, T)>, - T: Clone + Serialize + DeserializeOwned, - Order: - MinOrder< as CanSend<'a, Cluster<'a, C2>>>::OutStrongestOrder>, - { - let sent = self - .map(q!(move |b| ( - ClusterId::from_raw(CLUSTER_SELF_ID.raw_id), - b.clone() - ))) - .send_bincode_interleaved(other); - - unsafe { - // SAFETY: this is safe because we are mapping clusters 1:1 - sent.assume_ordering() - } - } -} - impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { - pub fn decouple_process( - self, - other: &Process<'a, P2>, - ) -> Stream, Unbounded, Order> - where - L::Root: CanSend<'a, Process<'a, P2>, In = T, Out = T>, - T: Clone + Serialize + DeserializeOwned, - Order: MinOrder< - >>::OutStrongestOrder, - Min = Order, - >, - { - self.send_bincode::, T>(other) - } - pub fn send_bincode, CoreType>( self, other: &L2, diff --git a/hydro_std/src/compartmentalize.rs b/hydro_std/src/compartmentalize.rs new file mode 100644 index 00000000000..83be82d239f --- /dev/null +++ b/hydro_std/src/compartmentalize.rs @@ -0,0 +1,121 @@ +use dfir_rs::serde::de::DeserializeOwned; +use dfir_rs::serde::Serialize; +use hydro_lang::*; +use location::{CanSend, NoTick}; +use stageleft::IntoQuotedMut; +use stream::MinOrder; + +pub trait PartitionStream<'a, T, C1, C2, Order> { + fn send_partitioned, T)) -> (ClusterId, T) + 'a>( + self, + other: &Cluster<'a, C2>, + dist_policy: impl IntoQuotedMut<'a, F, Cluster<'a, C1>>, + ) -> Stream, Unbounded, NoOrder> + where + Cluster<'a, C1>: Location<'a, Root = Cluster<'a, C1>>, + Cluster<'a, C1>: + CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, T)>, + T: Clone + Serialize + DeserializeOwned, + Order: MinOrder< + as CanSend<'a, Cluster<'a, C2>>>::OutStrongestOrder, + Min = NoOrder, + >; +} + +impl<'a, T, C1, C2, Order> PartitionStream<'a, T, C1, C2, Order> + for Stream<(ClusterId, T), Cluster<'a, C1>, Unbounded, Order> +{ + fn send_partitioned, T)) -> (ClusterId, T) + 'a>( + self, + other: &Cluster<'a, C2>, + dist_policy: impl IntoQuotedMut<'a, F, Cluster<'a, C1>>, + ) -> Stream, Unbounded, NoOrder> + where + Cluster<'a, C1>: Location<'a, Root = Cluster<'a, C1>>, + Cluster<'a, C1>: + CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, T)>, + T: Clone + Serialize + DeserializeOwned, + Order: MinOrder< + as CanSend<'a, Cluster<'a, C2>>>::OutStrongestOrder, + Min = NoOrder, + >, + { + self.map(dist_policy).send_bincode_interleaved(other) + } +} + +pub trait DecoupleClusterStream<'a, T, C1, B, Order> { + fn decouple_cluster( + self, + other: &Cluster<'a, C2>, + ) -> Stream, Unbounded, Order> + where + Cluster<'a, C1>: Location<'a, Root = Cluster<'a, C1>>, + Cluster<'a, C1>: + CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, T)>, + T: Clone + Serialize + DeserializeOwned, + Order: + MinOrder< as CanSend<'a, Cluster<'a, C2>>>::OutStrongestOrder>; +} + +impl<'a, T, C1, B, Order> DecoupleClusterStream<'a, T, C1, B, Order> + for Stream, B, Order> +{ + fn decouple_cluster( + self, + other: &Cluster<'a, C2>, + ) -> Stream, Unbounded, Order> + where + Cluster<'a, C1>: Location<'a, Root = Cluster<'a, C1>>, + Cluster<'a, C1>: + CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, T)>, + T: Clone + Serialize + DeserializeOwned, + Order: + MinOrder< as CanSend<'a, Cluster<'a, C2>>>::OutStrongestOrder>, + { + let sent = self + .map(q!(move |b| ( + ClusterId::from_raw(CLUSTER_SELF_ID.raw_id), + b.clone() + ))) + .send_bincode_interleaved(other); + + unsafe { + // SAFETY: this is safe because we are mapping clusters 1:1 + sent.assume_ordering() + } + } +} + +pub trait DecoupleProcessStream<'a, T, L: Location<'a> + NoTick, B, Order> { + fn decouple_process( + self, + other: &Process<'a, P2>, + ) -> Stream, Unbounded, Order> + where + L::Root: CanSend<'a, Process<'a, P2>, In = T, Out = T>, + T: Clone + Serialize + DeserializeOwned, + Order: MinOrder< + >>::OutStrongestOrder, + Min = Order, + >; +} + +impl<'a, T, L: Location<'a> + NoTick, B, Order> DecoupleProcessStream<'a, T, L, B, Order> + for Stream +{ + fn decouple_process( + self, + other: &Process<'a, P2>, + ) -> Stream, Unbounded, Order> + where + L::Root: CanSend<'a, Process<'a, P2>, In = T, Out = T>, + T: Clone + Serialize + DeserializeOwned, + Order: MinOrder< + >>::OutStrongestOrder, + Min = Order, + >, + { + self.send_bincode::, T>(other) + } +} diff --git a/hydro_std/src/lib.rs b/hydro_std/src/lib.rs index bec45446438..f0c90f8210e 100644 --- a/hydro_std/src/lib.rs +++ b/hydro_std/src/lib.rs @@ -1,5 +1,6 @@ stageleft::stageleft_no_entry_crate!(); +pub mod compartmentalize; pub mod quorum; pub mod request_response; diff --git a/hydro_test/Cargo.toml b/hydro_test/Cargo.toml index 8e860bba966..8e43bdec962 100644 --- a/hydro_test/Cargo.toml +++ b/hydro_test/Cargo.toml @@ -18,6 +18,7 @@ tokio = { version = "1.29.0", features = [ "full" ] } stageleft = { path = "../stageleft", version = "^0.6.0" } rand = "0.8.0" serde = { version = "1.0.197", features = [ "derive" ] } +ctor = "0.2.9" [build-dependencies] stageleft_tool = { path = "../stageleft_tool", version = "^0.5.0" } diff --git a/hydro_test/src/cluster/simple_cluster.rs b/hydro_test/src/cluster/simple_cluster.rs index 51339ee72ee..7ebb717929e 100644 --- a/hydro_test/src/cluster/simple_cluster.rs +++ b/hydro_test/src/cluster/simple_cluster.rs @@ -1,4 +1,25 @@ use hydro_lang::*; +use hydro_std::compartmentalize::{DecoupleClusterStream, DecoupleProcessStream, PartitionStream}; +use stageleft::IntoQuotedMut; + +pub fn partition<'a, F: Fn((ClusterId<()>, String)) -> (ClusterId<()>, String) + 'a>( + cluster1: Cluster<'a, ()>, + cluster2: Cluster<'a, ()>, + dist_policy: impl IntoQuotedMut<'a, F, Cluster<'a, ()>>, +) -> (Cluster<'a, ()>, Cluster<'a, ()>) { + cluster1 + .source_iter(q!(vec!(CLUSTER_SELF_ID))) + .map(q!(move |id| ( + ClusterId::<()>::from_raw(id.raw_id), + format!("Hello from {}", id.raw_id) + ))) + .send_partitioned(&cluster2, dist_policy) + .for_each(q!(move |message| println!( + "My self id is {}, my message is {:?}", + CLUSTER_SELF_ID.raw_id, message + ))); + (cluster1, cluster2) +} pub fn decouple_cluster<'a>(flow: &FlowBuilder<'a>) -> (Cluster<'a, ()>, Cluster<'a, ()>) { let cluster1 = flow.cluster(); @@ -49,6 +70,8 @@ pub fn simple_cluster<'a>(flow: &FlowBuilder<'a>) -> (Process<'a, ()>, Cluster<' mod tests { use hydro_deploy::Deployment; use hydro_lang::deploy::DeployCrateWrapper; + use hydro_lang::ClusterId; + use stageleft::q; #[tokio::test] async fn simple_cluster() { @@ -165,4 +188,54 @@ mod tests { } } } + + #[tokio::test] + async fn partition() { + let mut deployment = Deployment::new(); + + let num_nodes = 3; + let num_partitions = 2; + let builder = hydro_lang::FlowBuilder::new(); + let (cluster1, cluster2) = super::partition( + builder.cluster::<()>(), + builder.cluster::<()>(), + q!(move |(id, msg)| ( + ClusterId::<()>::from_raw(id.raw_id * num_partitions as u32), + msg + )), + ); + let built = builder.with_default_optimize(); + + let nodes = built + .with_cluster(&cluster1, (0..num_nodes).map(|_| deployment.Localhost())) + .with_cluster( + &cluster2, + (0..num_nodes * num_partitions).map(|_| deployment.Localhost()), + ) + .deploy(&mut deployment); + + deployment.deploy().await.unwrap(); + + let cluster2_stdouts = futures::future::join_all( + nodes + .get_cluster(&cluster2) + .members() + .iter() + .map(|node| node.stdout()), + ) + .await; + + deployment.start().await.unwrap(); + + for (cluster2_id, mut stdout) in cluster2_stdouts.into_iter().enumerate() { + if cluster2_id % num_partitions == 0 { + let expected_message = format!( + r#"My self id is {}, my message is "Hello from {}""#, + cluster2_id, + cluster2_id / num_partitions + ); + assert_eq!(stdout.recv().await.unwrap(), expected_message); + } + } + } } From b5cb01f43a79ec9b7da0ffc540bfb17722959058 Mon Sep 17 00:00:00 2001 From: Mingwei Samuel Date: Thu, 16 Jan 2025 10:08:45 -0800 Subject: [PATCH 02/29] chore: update `.vscode/settings.json` (#1660) --- .vscode/settings.json | 25 ++++++++++++------------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/.vscode/settings.json b/.vscode/settings.json index b2db3df9b47..cb24f188940 100644 --- a/.vscode/settings.json +++ b/.vscode/settings.json @@ -1,17 +1,16 @@ { - "rust-analyzer.runnableEnv": [ - { - "env": { - // Stack backtraces. - "RUST_BACKTRACE": "full", - // Set output levels for `tracing` logging. - "RUST_LOG": "debug,hydroflow=trace", - // Make sure all snapshots are written instead of just the first failure. - "INSTA_FORCE_PASS": "1", - "INSTA_UPDATE": "always", - "TRYBUILD": "overwrite", - } - } + "rust-analyzer.runnables.extraEnv": { + // Stack backtraces. + "RUST_BACKTRACE": "full", + // Set output levels for `tracing` logging. + "RUST_LOG": "debug,dfir_rs=trace", + // Make sure all snapshots are written instead of just the first failure. + "INSTA_FORCE_PASS": "1", + "INSTA_UPDATE": "always", + "TRYBUILD": "overwrite", + }, + "rust-analyzer.runnables.extraTestBinaryArgs": [ + "--nocapture" ], "rust-analyzer.cargo.features": ["deploy"], "editor.semanticTokenColorCustomizations": { From 146d10a1347aa23e1e42500abef86201851bacfd Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Thu, 16 Jan 2025 10:28:42 -0800 Subject: [PATCH 03/29] docs(hydro_lang)!: rename `singleton_first_tick` to `optional_first_tick` and add example of doctest (#1659) --- hydro_lang/src/location/tick.rs | 2 +- hydro_lang/src/optional.rs | 18 ++++++++++++++++++ hydro_test/src/cluster/bench_client.rs | 2 +- 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/hydro_lang/src/location/tick.rs b/hydro_lang/src/location/tick.rs index f161fc170b4..2841db80646 100644 --- a/hydro_lang/src/location/tick.rs +++ b/hydro_lang/src/location/tick.rs @@ -123,7 +123,7 @@ impl<'a, L: Location<'a>> Tick { } } - pub fn singleton_first_tick( + pub fn optional_first_tick( &self, e: impl QuotedWithContext<'a, T, Tick>, ) -> Optional diff --git a/hydro_lang/src/optional.rs b/hydro_lang/src/optional.rs index f686c3927b7..dbfa591e0b3 100644 --- a/hydro_lang/src/optional.rs +++ b/hydro_lang/src/optional.rs @@ -197,6 +197,24 @@ impl<'a, T: Clone, L: Location<'a>, B> Clone for Optional { } impl<'a, T, L: Location<'a>, B> Optional { + /// Transforms the optional value by applying a function `f` to it, + /// continuously as the input is updated. + /// + /// Whenever the optional is empty, the output optional is also empty. + /// + /// # Example + /// ```rust + /// # use hydro_lang::*; + /// # use dfir_rs::futures::StreamExt; + /// # tokio_test::block_on(test_util::stream_transform_test(|process| { + /// let tick = process.tick(); + /// let optional = tick.optional_first_tick(q!(1)); + /// optional.map(q!(|v| v + 1)).all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 2 + /// # assert_eq!(stream.next().await.unwrap(), 2); + /// # })); + /// ``` pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F, L>) -> Optional { let f = f.splice_fn1_ctx(&self.location).into(); Optional::new( diff --git a/hydro_test/src/cluster/bench_client.rs b/hydro_test/src/cluster/bench_client.rs index 788a61c8da2..d573a9d2c1f 100644 --- a/hydro_test/src/cluster/bench_client.rs +++ b/hydro_test/src/cluster/bench_client.rs @@ -19,7 +19,7 @@ pub fn bench_client<'a>( // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); // Set up an initial set of payloads on the first tick - let start_this_tick = client_tick.singleton_first_tick(q!(())); + let start_this_tick = client_tick.optional_first_tick(q!(())); let c_new_payloads_on_start = start_this_tick.clone().flat_map_ordered(q!(move |_| (0 ..num_clients_per_node) From 99cfb68beb4f31563ca70031e8c02816f5afd9a9 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Thu, 16 Jan 2025 14:15:48 -0800 Subject: [PATCH 04/29] ci(hydro_cli): bump upload-artifact action to v4 (#1661) v3 now fails on GHA --- .github/workflows/build-cli.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/build-cli.yml b/.github/workflows/build-cli.yml index 7557be0b46f..acc5d01b919 100644 --- a/.github/workflows/build-cli.yml +++ b/.github/workflows/build-cli.yml @@ -49,7 +49,7 @@ jobs: run: | pip install hydro_deploy/hydro_cli/dist/${{ env.PACKAGE_NAME }}-*.whl --force-reinstall - name: "Upload wheels" - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: wheels path: hydro_deploy/hydro_cli/dist @@ -92,7 +92,7 @@ jobs: run: | pip install hydro_deploy/hydro_cli/dist/${{ env.PACKAGE_NAME }}-*.whl --force-reinstall - name: "Upload wheels" - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: wheels path: hydro_deploy/hydro_cli/dist @@ -121,7 +121,7 @@ jobs: manylinux: auto args: --release --out dist - name: "Upload wheels" - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: wheels path: hydro_deploy/hydro_cli/dist @@ -158,7 +158,7 @@ jobs: run: | python -m pip install hydro_deploy/hydro_cli/dist/${{ env.PACKAGE_NAME }}-*.whl --force-reinstall - name: "Upload wheels" - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: wheels path: hydro_deploy/hydro_cli/dist From 720feb9aa9494b252893bec347557c9703751abb Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Thu, 16 Jan 2025 17:54:52 -0800 Subject: [PATCH 05/29] ci(hydro_cli): fix matrix clobbering the same artifact path (#1664) Breaking change from v3 to v4 --- .github/workflows/build-cli.yml | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build-cli.yml b/.github/workflows/build-cli.yml index acc5d01b919..2fd11fdaabf 100644 --- a/.github/workflows/build-cli.yml +++ b/.github/workflows/build-cli.yml @@ -51,7 +51,7 @@ jobs: - name: "Upload wheels" uses: actions/upload-artifact@v4 with: - name: wheels + name: wheels-${{ matrix.target }} path: hydro_deploy/hydro_cli/dist linux: @@ -94,7 +94,7 @@ jobs: - name: "Upload wheels" uses: actions/upload-artifact@v4 with: - name: wheels + name: wheels-${{ matrix.target }} path: hydro_deploy/hydro_cli/dist linux-cross: @@ -123,7 +123,7 @@ jobs: - name: "Upload wheels" uses: actions/upload-artifact@v4 with: - name: wheels + name: wheels-${{ matrix.target }} path: hydro_deploy/hydro_cli/dist windows: @@ -160,7 +160,7 @@ jobs: - name: "Upload wheels" uses: actions/upload-artifact@v4 with: - name: wheels + name: wheels-${{ matrix.target }} path: hydro_deploy/hydro_cli/dist release: @@ -175,7 +175,9 @@ jobs: steps: - uses: actions/download-artifact@v3 with: - name: wheels + pattern: wheels-* + path: wheels + merge-multiple: true - uses: actions/setup-python@v5 - name: "Publish to PyPi" env: From d069c1f9e861f708f89f7b0de75d26c63f29b028 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Thu, 16 Jan 2025 17:55:03 -0800 Subject: [PATCH 06/29] docs: fix badges in README (#1665) --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 7efcd2d2307..b06d2238ab6 100644 --- a/README.md +++ b/README.md @@ -2,8 +2,8 @@ "hf"

- Crates.io - Docs.rs + Crates.io + Docs.rs

Hydro is a novel distributed programming library for standard Rust. Hydro allows developers to build distributed systems that are efficient, scalable, and correct. From 496ac2b8a7ba502f20199ed7decd021f3b54b35a Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Thu, 16 Jan 2025 20:08:16 -0800 Subject: [PATCH 07/29] fixup!: ci(hydro_cli): fix matrix clobbering the same artifact path Some of the matrices nest `target` under `platform`. --- .github/workflows/build-cli.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build-cli.yml b/.github/workflows/build-cli.yml index 2fd11fdaabf..8918bfb43fa 100644 --- a/.github/workflows/build-cli.yml +++ b/.github/workflows/build-cli.yml @@ -123,7 +123,7 @@ jobs: - name: "Upload wheels" uses: actions/upload-artifact@v4 with: - name: wheels-${{ matrix.target }} + name: wheels-${{ matrix.platform.target }} path: hydro_deploy/hydro_cli/dist windows: @@ -160,7 +160,7 @@ jobs: - name: "Upload wheels" uses: actions/upload-artifact@v4 with: - name: wheels-${{ matrix.target }} + name: wheels-${{ matrix.platform.target }} path: hydro_deploy/hydro_cli/dist release: From e43e7850121e4dfd1d2bfb2d76b1de3294ca2d48 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Thu, 23 Jan 2025 16:08:44 -0800 Subject: [PATCH 08/29] docs(hydro_lang): initial website docs on core Hydro concepts (#1668) --- Cargo.lock | 1 + docs/docs/hydro/correctness.md | 16 +++++++ .../hydro/live-collections/_category_.json | 8 ++++ .../live-collections/bounded-unbounded.md | 48 +++++++++++++++++++ .../determinism.md} | 17 +++---- docs/docs/hydro/live-collections/index.md | 4 ++ .../live-collections/singletons-optionals.md | 10 ++++ docs/docs/hydro/live-collections/streams.md | 10 ++++ docs/docs/hydro/locations/_category_.json | 8 ++++ docs/docs/hydro/locations/clusters.md | 10 ++++ docs/docs/hydro/locations/external-clients.md | 10 ++++ docs/docs/hydro/locations/index.md | 41 ++++++++++++++++ docs/docs/hydro/locations/processes.md | 10 ++++ docs/docs/hydro/stageleft.mdx | 2 +- .../hydro/ticks-atomicity/_category_.json | 8 ++++ docs/docs/hydro/ticks-atomicity/atomicity.md | 10 ++++ .../ticks-atomicity/batching-and-emitting.md | 10 ++++ .../hydro/ticks-atomicity/execution-model.md | 10 ++++ docs/docs/hydro/ticks-atomicity/index.md | 12 +++++ .../hydro/ticks-atomicity/stateful-loops.md | 10 ++++ hydro_lang/src/location/cluster/mod.rs | 22 +++++++-- hydro_lang/src/location/process.rs | 14 ++++++ hydro_lang/src/stream.rs | 10 ++++ hydro_lang/src/test_util.rs | 20 ++++++++ hydro_test/Cargo.toml | 1 + 25 files changed, 306 insertions(+), 16 deletions(-) create mode 100644 docs/docs/hydro/correctness.md create mode 100644 docs/docs/hydro/live-collections/_category_.json create mode 100644 docs/docs/hydro/live-collections/bounded-unbounded.md rename docs/docs/hydro/{consistency.md => live-collections/determinism.md} (68%) create mode 100644 docs/docs/hydro/live-collections/index.md create mode 100644 docs/docs/hydro/live-collections/singletons-optionals.md create mode 100644 docs/docs/hydro/live-collections/streams.md create mode 100644 docs/docs/hydro/locations/_category_.json create mode 100644 docs/docs/hydro/locations/clusters.md create mode 100644 docs/docs/hydro/locations/external-clients.md create mode 100644 docs/docs/hydro/locations/index.md create mode 100644 docs/docs/hydro/locations/processes.md create mode 100644 docs/docs/hydro/ticks-atomicity/_category_.json create mode 100644 docs/docs/hydro/ticks-atomicity/atomicity.md create mode 100644 docs/docs/hydro/ticks-atomicity/batching-and-emitting.md create mode 100644 docs/docs/hydro/ticks-atomicity/execution-model.md create mode 100644 docs/docs/hydro/ticks-atomicity/index.md create mode 100644 docs/docs/hydro/ticks-atomicity/stateful-loops.md diff --git a/Cargo.lock b/Cargo.lock index bd721e82b1e..88943c1c77d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1648,6 +1648,7 @@ dependencies = [ "stageleft", "stageleft_tool", "tokio", + "tokio-test", ] [[package]] diff --git a/docs/docs/hydro/correctness.md b/docs/docs/hydro/correctness.md new file mode 100644 index 00000000000..69a9981a2a6 --- /dev/null +++ b/docs/docs/hydro/correctness.md @@ -0,0 +1,16 @@ +--- +sidebar_position: 3 +--- + +# Safety and Correctness +Just like Rust's type system helps you avoid memory safety bugs, Hydro helps you ensure **distributed safety**. Hydro's type systems helps you avoid many kinds of distributed systems bugs, including: +- Non-determinism due to message delays (which reorder arrival) or retries (which result in duplicates) + - See [Live Collections / Eventual Determinism](./live-collections/determinism.md) +- Using mismatched serialization and deserialization formats across services + - See [Locations and Networking](./locations/index.md) +- Misusing node identifiers across logically independent clusters of machines + - See [Locations / Clusters](./locations/clusters.md) +- Relying on non-determinstic clocks for batching events + - See [Ticks and Atomicity / Batching and Emitting Streams](./ticks-atomicity/batching-and-emitting.md) + +These safety guarantees are surfaced through the Rust type system, so you can catch these bugs at compile time rather than in production. And when it is necessary to bypass these checks for advanced distributed logic, you can use the same `unsafe` keyword as in Rust as an escape hatch. diff --git a/docs/docs/hydro/live-collections/_category_.json b/docs/docs/hydro/live-collections/_category_.json new file mode 100644 index 00000000000..e392c1ef73e --- /dev/null +++ b/docs/docs/hydro/live-collections/_category_.json @@ -0,0 +1,8 @@ +{ + "label": "Live Collections", + "position": 4, + "link": { + "type": "doc", + "id": "hydro/live-collections/index" + } +} diff --git a/docs/docs/hydro/live-collections/bounded-unbounded.md b/docs/docs/hydro/live-collections/bounded-unbounded.md new file mode 100644 index 00000000000..a60180fc3d2 --- /dev/null +++ b/docs/docs/hydro/live-collections/bounded-unbounded.md @@ -0,0 +1,48 @@ +--- +sidebar_position: 0 +--- + +# Bounded and Unbounded Types +Although live collections can be continually updated, some collection types also support **termination**, after which no additional changes can be made. For example, a live collection created by reading integers from an in-memory `Vec` will become terminated once all the elements of the `Vec` have been loaded. But other live collections, such as one being updated by the network, may never become terminated. + +In Hydro, certain APIs are restricted to only work on collections that are **guaranteed to terminate** (**bounded** collections). All live collections in Hydro have a type parameter (typically named `B`), which tracks whether the collection is bounded (has the type `Bounded`) or unbounded (has the type `Unbounded`). These types are used in the signature of many Hydro APIs to ensure that the API is only called on the appropriate type of collection. + +## Converting Boundedness +In some cases, you may need to convert between bounded and unbounded collections. Converting from a bounded collection **to an unbounded collection** is always allowed and safe, since it relaxes the guarantees on the collection. This can be done by calling `.into()` on the collection. + +```rust,no_run +# use hydro_lang::*; +# use dfir_rs::futures::StreamExt; +# let flow = FlowBuilder::new(); +# let process = flow.process::<()>(); +# let tick = process.tick(); +# let numbers = process.source_iter(q!(vec![1, 2, 3, 4])); +let input: Stream<_, _, Bounded> = // ... +# unsafe { numbers.timestamped(&tick).tick_batch() }; +let unbounded: Stream<_, _, Unbounded> = input.into(); +``` + +```rust,no_run +# use hydro_lang::*; +# use dfir_rs::futures::StreamExt; +# let flow = FlowBuilder::new(); +# let process = flow.process::<()>(); +# let tick = process.tick(); +let input: Singleton<_, _, Bounded> = tick.singleton(q!(0)); +let unbounded: Singleton<_, _, Unbounded> = input.into(); +``` + +Converting from an unbounded collection **to a bounded collection**, however is more complex. This requires cutting off the unbounded collection at a specific point in time, which may not be possible to do deterministically. For example, the most common way to convert an unbounded `Stream` to a bounded one is to batch its elements non-deterministically using `.tick_batch()`. + +```rust,no_run +# use hydro_lang::*; +# use dfir_rs::futures::StreamExt; +# let flow = FlowBuilder::new(); +# let process = flow.process::<()>(); +let unbounded_input = // ... +# process.source_iter(q!(vec![1, 2, 3, 4])); +let tick = process.tick(); +let batch: Stream<_, _, Bounded> = unsafe { + unbounded_input.timestamped(&tick).tick_batch() +}; +``` diff --git a/docs/docs/hydro/consistency.md b/docs/docs/hydro/live-collections/determinism.md similarity index 68% rename from docs/docs/hydro/consistency.md rename to docs/docs/hydro/live-collections/determinism.md index e22702cad46..f632f777214 100644 --- a/docs/docs/hydro/consistency.md +++ b/docs/docs/hydro/live-collections/determinism.md @@ -1,9 +1,9 @@ --- -sidebar_position: 3 +sidebar_position: 1 --- -# Consistency and Safety -A key feature of Hydro is its integration with the Rust type system to highlight possible sources of inconsistent distributed behavior due to sources of non-determinism such as batching, timeouts, and message reordering. In this section, we'll walk through the consistency guarantees in Hydro and how to use the **`unsafe`** keyword as an escape hatch when introducing sources of non-determinism. +# Eventual Determinism +Most programs are strong guarantees on **determinism**, the property that when provided the same inputs, the outputs of the program are always the same. Even when the inputs and outputs are live collections, we can focus on the _eventual_ state of the collection (as if we froze the input and waited until the output stops changing). :::info @@ -11,12 +11,7 @@ Our consistency and safety model is based on the POPL'25 paper [Flo: A Semantic ::: -## Eventual Determinism -Hydro provides strong guarantees on **determinism**, the property that when provided the same inputs, the outputs of the program are always the same. Even when the inputs and outputs are streaming, we can use this property by looking at the **aggregate collection** (i.e. the result of collecting the elements of the stream into a finite collection). This makes it easy to build composable blocks of code without having to worry about runtime behavior such as batching or network delays. - -Because Hydro programs can involve network delay, we guarantee **eventual determinism**: given a set of streaming inputs which have arrived, the outputs of the program (which continuously change as inputs arrive) will **eventually** have the same _aggregate_ value. - -Again, by focusing on the _aggregate_ value rather than individual outputs, Hydro programs can involve concepts such as retractions (for incremental computation) while still guaranteeing determinism because the _resolved_ output (after processing retractions) will eventually be the same. +Hydro thus guarantees **eventual determinism**: given a set of streaming inputs which have arrived, the outputs of the program will **eventually** have the same _final_ value. This makes it easy to build composable blocks of code without having to worry about runtime behavior such as batching or network delays. :::note @@ -54,9 +49,9 @@ use std::fmt::Debug; use std::time::Duration; /// ... -/// +/// /// # Safety -/// This function will non-deterministically print elements +/// This function will non-deterministically print elements /// from the stream according to a timer. unsafe fn print_samples( stream: Stream, Unbounded> diff --git a/docs/docs/hydro/live-collections/index.md b/docs/docs/hydro/live-collections/index.md new file mode 100644 index 00000000000..cc57a73fcbe --- /dev/null +++ b/docs/docs/hydro/live-collections/index.md @@ -0,0 +1,4 @@ +# Live Collections +Traditional programs (like those in Rust) typically manipulate **collections** of data elements, such as those stored in a `Vec` or `HashMap`. These collections are **fixed** in the sense that any transformations applied to them such as `map` are immediately executed on a snapshot of the collection. This means that the output will not be updated when the input collection is modified. + +In Hydro, programs instead work with **live collections** which are expected to dynamically change over time as new elements are added or removed (in response to API requests, streaming ingestion, etc). Applying a transformation like `map` to a live collection results in another live collection that will dynamically change over time. diff --git a/docs/docs/hydro/live-collections/singletons-optionals.md b/docs/docs/hydro/live-collections/singletons-optionals.md new file mode 100644 index 00000000000..1679e6f536f --- /dev/null +++ b/docs/docs/hydro/live-collections/singletons-optionals.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 3 +--- + +# Singletons and Optionals +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/live-collections/streams.md b/docs/docs/hydro/live-collections/streams.md new file mode 100644 index 00000000000..599999e0d88 --- /dev/null +++ b/docs/docs/hydro/live-collections/streams.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 2 +--- + +# Streams +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/locations/_category_.json b/docs/docs/hydro/locations/_category_.json new file mode 100644 index 00000000000..fc22d7a7c83 --- /dev/null +++ b/docs/docs/hydro/locations/_category_.json @@ -0,0 +1,8 @@ +{ + "label": "Locations and Networking", + "position": 5, + "link": { + "type": "doc", + "id": "hydro/locations/index" + } +} diff --git a/docs/docs/hydro/locations/clusters.md b/docs/docs/hydro/locations/clusters.md new file mode 100644 index 00000000000..c97fe8bb071 --- /dev/null +++ b/docs/docs/hydro/locations/clusters.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 1 +--- + +# Clusters +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/locations/external-clients.md b/docs/docs/hydro/locations/external-clients.md new file mode 100644 index 00000000000..d53ea41f0b9 --- /dev/null +++ b/docs/docs/hydro/locations/external-clients.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 2 +--- + +# External Clients +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/locations/index.md b/docs/docs/hydro/locations/index.md new file mode 100644 index 00000000000..7bfcaadcd65 --- /dev/null +++ b/docs/docs/hydro/locations/index.md @@ -0,0 +1,41 @@ +# Locations and Networking +Hydro is a **global**, **distributed** programming model. This means that the data and computation in a Hydro program can be spread across multiple machines, data centers, and even continents. To achieve this, Hydro uses the concept of **locations** to keep track of _where_ data is stored and computation is executed. + +Each live collection type (`Stream`, `Singleton`, etc.) has a type parameter `L` which will always be a type that implements the `Location` trait (e.g. `Process` and `Cluster`, documented in this section). Most Hydro APIs that transform live collections will emit a new live collection with the same location type as the input, and APIs that consume multiple live collections will require them all to have the same location type. + +To create distributed programs, live collections can be sent over the network using a variety of APIs. For example, `Stream`s can be sent from a process to another process using `.send_bincode(&loc2)` (which uses [bincode](https://docs.rs/bincode/latest/bincode/) as a serialization format). The sections for each location type discuss the networking APIs in further detail. + +## Creating Locations +Locations can be created by calling the appropriate method on the global `FlowBuilder` (e.g. `flow.process()` or `flow.cluster()`). These methods will return a handle to the location that can be used to create live collections and run computations. + +:::caution + +It is possible to create **different** locations that still have the same type, for example: + +```rust +# use hydro_lang::*; +let flow = FlowBuilder::new(); +let process1: Process<()> = flow.process::<()>(); +let process2: Process<()> = flow.process::<()>(); + +assert_ne!(process1, process2); +# let _ = flow.compile_no_network::(); +``` + +These locations will not be unified and may be deployed to separate machines. When deploying a Hydro program, additional runtime checks will be performed to ensure that input locations match. + +```rust +# use hydro_lang::*; +let flow = FlowBuilder::new(); +let process1: Process<()> = flow.process::<()>(); +let process2: Process<()> = flow.process::<()>(); + +# test_util::assert_panics_with_message(|| { +process1.source_iter(q!([1, 2, 3])) + .cross_product(process2.source_iter(q!([1, 2, 3]))); +// PANIC: assertion `left == right` failed: locations do not match +# }, "assertion `left == right` failed: locations do not match"); +# let _ = flow.compile_no_network::(); +``` + +::: diff --git a/docs/docs/hydro/locations/processes.md b/docs/docs/hydro/locations/processes.md new file mode 100644 index 00000000000..44359005905 --- /dev/null +++ b/docs/docs/hydro/locations/processes.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 0 +--- + +# Processes +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/stageleft.mdx b/docs/docs/hydro/stageleft.mdx index 54c119394d5..a4d50c394b2 100644 --- a/docs/docs/hydro/stageleft.mdx +++ b/docs/docs/hydro/stageleft.mdx @@ -1,6 +1,6 @@ --- title: Stageleft -sidebar_position: 4 +sidebar_position: 7 --- import StageleftDocs from '../../../stageleft/README.md' diff --git a/docs/docs/hydro/ticks-atomicity/_category_.json b/docs/docs/hydro/ticks-atomicity/_category_.json new file mode 100644 index 00000000000..0b51084aab0 --- /dev/null +++ b/docs/docs/hydro/ticks-atomicity/_category_.json @@ -0,0 +1,8 @@ +{ + "label": "Ticks and Atomicity", + "position": 6, + "link": { + "type": "doc", + "id": "hydro/ticks-atomicity/index" + } +} diff --git a/docs/docs/hydro/ticks-atomicity/atomicity.md b/docs/docs/hydro/ticks-atomicity/atomicity.md new file mode 100644 index 00000000000..3c45ab26f99 --- /dev/null +++ b/docs/docs/hydro/ticks-atomicity/atomicity.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 3 +--- + +# Atomicity and Timestamps +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/ticks-atomicity/batching-and-emitting.md b/docs/docs/hydro/ticks-atomicity/batching-and-emitting.md new file mode 100644 index 00000000000..a9549c51161 --- /dev/null +++ b/docs/docs/hydro/ticks-atomicity/batching-and-emitting.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 1 +--- + +# Batching and Emitting Streams +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/ticks-atomicity/execution-model.md b/docs/docs/hydro/ticks-atomicity/execution-model.md new file mode 100644 index 00000000000..db2dc3d05c9 --- /dev/null +++ b/docs/docs/hydro/ticks-atomicity/execution-model.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 0 +--- + +# The Tick Execution Model +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/docs/docs/hydro/ticks-atomicity/index.md b/docs/docs/hydro/ticks-atomicity/index.md new file mode 100644 index 00000000000..8f0ac9a4bfe --- /dev/null +++ b/docs/docs/hydro/ticks-atomicity/index.md @@ -0,0 +1,12 @@ +# Ticks and Atomicity +By default, all live collections in Hydro are transformed **asynchronously**, which means that there may be arbitrary delays between when a live collection is updated and when downstream transformations see the updates. This is because Hydro is designed to work in a distributed setting where messages may be delayed. But for some programs, it is necessary to define local iterative loops where transformations are applied atomically; this is achieved with **ticks**. + +## Loops +In some programs, you may want to process batches or snapshots of a live collection in an iterative manner. For example, in a map-reduce program, it may be helpful to compute aggregations on small local batches of data before sending those intermediate results to a reducer. + +To create such iterative loops, Hydro provides the concept of **ticks**. A **tick** captures the body of an infinite loop running locally to the machine (importantly, this means that ticks define a **logical time** which is not comparable across machines). Ticks are non-deterministically generated, so batching data into ticks is an **unsafe** operation that requires special attention. + +## Atomicity +In other programs, it is necessary to define an atomic section where a set of transformations are guaranteed to be executed **all at once**. For example, in a transaction processing program, it is important that the transaction is applied **before** an acknowledgment is sent to the client. + +In Hydro, this can be achieved by placing the transaction and acknowledgment in the same atomic **tick**. Hydro guarantees that all the outputs of a tick will be computed before any are released. Importantly, atomic ticks cannot span several locations, since that would require a locking mechanism that has significant performance implications. diff --git a/docs/docs/hydro/ticks-atomicity/stateful-loops.md b/docs/docs/hydro/ticks-atomicity/stateful-loops.md new file mode 100644 index 00000000000..7f3c56673e6 --- /dev/null +++ b/docs/docs/hydro/ticks-atomicity/stateful-loops.md @@ -0,0 +1,10 @@ +--- +sidebar_position: 2 +--- + +# Stateful Loops +:::caution + +The Hydro documentation is currently under active development! This page is a placeholder for future content. + +::: diff --git a/hydro_lang/src/location/cluster/mod.rs b/hydro_lang/src/location/cluster/mod.rs index 7e9ee8e76ed..5517612654a 100644 --- a/hydro_lang/src/location/cluster/mod.rs +++ b/hydro_lang/src/location/cluster/mod.rs @@ -1,3 +1,4 @@ +use std::fmt::{Debug, Formatter}; use std::marker::PhantomData; use proc_macro2::{Span, TokenStream}; @@ -18,12 +19,17 @@ pub struct Cluster<'a, C> { pub(crate) _phantom: Invariant<'a, C>, } -pub trait IsCluster { - type Tag; +impl Debug for Cluster<'_, C> { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Cluster({})", self.id) + } } -impl IsCluster for Cluster<'_, C> { - type Tag = C; +impl Eq for Cluster<'_, C> {} +impl PartialEq for Cluster<'_, C> { + fn eq(&self, other: &Self) -> bool { + self.id == other.id && self.flow_state.as_ptr() == other.flow_state.as_ptr() + } } impl<'a, C> Cluster<'a, C> { @@ -102,6 +108,14 @@ impl<'a, C: 'a, Ctx> FreeVariableWithContext for ClusterIds<'a, C> { impl<'a, C, Ctx> QuotedWithContext<'a, &'a Vec>, Ctx> for ClusterIds<'a, C> {} +pub trait IsCluster { + type Tag; +} + +impl IsCluster for Cluster<'_, C> { + type Tag = C; +} + /// A free variable representing the cluster's own ID. When spliced in /// a quoted snippet that will run on a cluster, this turns into a [`ClusterId`]. pub static CLUSTER_SELF_ID: ClusterSelfId = ClusterSelfId { _private: &() }; diff --git a/hydro_lang/src/location/process.rs b/hydro_lang/src/location/process.rs index 417cc92165c..b3987648984 100644 --- a/hydro_lang/src/location/process.rs +++ b/hydro_lang/src/location/process.rs @@ -1,3 +1,4 @@ +use std::fmt::{Debug, Formatter}; use std::marker::PhantomData; use super::{Location, LocationId}; @@ -10,6 +11,19 @@ pub struct Process<'a, P = ()> { pub(crate) _phantom: Invariant<'a, P>, } +impl

Debug for Process<'_, P> { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "Process({})", self.id) + } +} + +impl

Eq for Process<'_, P> {} +impl

PartialEq for Process<'_, P> { + fn eq(&self, other: &Self) -> bool { + self.id == other.id && self.flow_state.as_ptr() == other.flow_state.as_ptr() + } +} + impl

Clone for Process<'_, P> { fn clone(&self) -> Self { Process { diff --git a/hydro_lang/src/stream.rs b/hydro_lang/src/stream.rs index deffeb4fb35..db14cfe40c5 100644 --- a/hydro_lang/src/stream.rs +++ b/hydro_lang/src/stream.rs @@ -73,6 +73,16 @@ pub struct Stream { _phantom: PhantomData<(T, L, B, Order)>, } +impl<'a, T, L: Location<'a>, O> From> for Stream { + fn from(stream: Stream) -> Stream { + Stream { + location: stream.location, + ir_node: stream.ir_node, + _phantom: PhantomData, + } + } +} + impl<'a, T, L: Location<'a>, B> From> for Stream { fn from(stream: Stream) -> Stream { Stream { diff --git a/hydro_lang/src/test_util.rs b/hydro_lang/src/test_util.rs index 42d0cabc979..f1712848ea5 100644 --- a/hydro_lang/src/test_util.rs +++ b/hydro_lang/src/test_util.rs @@ -1,4 +1,5 @@ use std::future::Future; +use std::panic::{catch_unwind, AssertUnwindSafe}; use std::pin::Pin; use serde::de::DeserializeOwned; @@ -33,3 +34,22 @@ pub async fn stream_transform_test< check(external_out).await; } + +// from https://users.rust-lang.org/t/how-to-write-doctest-that-panic-with-an-expected-message/58650 +pub fn assert_panics_with_message(func: impl FnOnce(), msg: &'static str) { + let err = catch_unwind(AssertUnwindSafe(func)).expect_err("Didn't panic!"); + + let chk = |panic_msg: &'_ str| { + if !panic_msg.contains(msg) { + panic!( + "Expected a panic message containing `{}`; got: `{}`.", + msg, panic_msg + ); + } + }; + + err.downcast::() + .map(|s| chk(&s)) + .or_else(|err| err.downcast::<&'static str>().map(|s| chk(*s))) + .expect("Unexpected panic type!"); +} diff --git a/hydro_test/Cargo.toml b/hydro_test/Cargo.toml index 8e43bdec962..75e82d67d86 100644 --- a/hydro_test/Cargo.toml +++ b/hydro_test/Cargo.toml @@ -31,3 +31,4 @@ hydro_lang = { path = "../hydro_lang", version = "^0.11.0", features = [ "deploy futures = "0.3.0" async-ssh2-lite = { version = "0.5.0", features = ["vendored-openssl"] } dfir_macro = { path = "../dfir_macro", version = "^0.11.0" } +tokio-test = "0.4.4" From 90b3d8ae3c15080887180ee21b2e13c7f402adec Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Fri, 24 Jan 2025 13:29:10 -0800 Subject: [PATCH 09/29] docs(hydro_lang): add docs for processes (#1671) --- docs/docs/hydro/locations/processes.md | 37 ++++++++++++++++++++++++-- 1 file changed, 35 insertions(+), 2 deletions(-) diff --git a/docs/docs/hydro/locations/processes.md b/docs/docs/hydro/locations/processes.md index 44359005905..99e7b3883ca 100644 --- a/docs/docs/hydro/locations/processes.md +++ b/docs/docs/hydro/locations/processes.md @@ -3,8 +3,41 @@ sidebar_position: 0 --- # Processes -:::caution +The simplest type of location in Hydro is a **process**. A process represents a **single thread** running a piece of a Hydro program (with no internal concurrency). When creating a process, you can pass in a type parameter that will be used as a marker to distinguish that process from others (and will also be used to mark logs originating at that process). For example, you can create a process with a marker of `Leader` to represent a leader in a distributed system: -The Hydro documentation is currently under active development! This page is a placeholder for future content. +```rust,no_run +# use hydro_lang::*; +struct Leader {} + +let flow = FlowBuilder::new(); +let leader: Process = flow.process::(); +``` + +:::note + +Currently, each Hydro process is deployed as a **separate** operating system process. In the future, we plan to support running multiple Hydro processes in a single operating system process for more efficient resource sharing. ::: + +Once we have a process, we can create live collections on that process (see [Live Collections](../live-collections/index.md) for more details). For example, we can create a stream of integers on the leader process: + +```rust,no_run +# use hydro_lang::*; +# struct Leader {} +# let flow = FlowBuilder::new(); +# let leader: Process = flow.process::(); +let numbers = leader.source_iter(q!(vec![1, 2, 3, 4])); +``` + +## Networking +Because a process represents a single machine, it is straightforward to send data to and from a process. For example, we can send a stream of integers from the leader process to another process using the `send_bincode` method (which uses [bincode](https://docs.rs/bincode/latest/bincode/) as a serialization format). This automatically sets up network senders and receivers on the two processes. + +```rust,no_run +# use hydro_lang::*; +# struct Leader {} +# let flow = FlowBuilder::new(); +# let leader: Process = flow.process::(); +let numbers = leader.source_iter(q!(vec![1, 2, 3, 4])); +let process2: Process<()> = flow.process::<()>(); +let on_p2: Stream<_, Process<()>, _> = numbers.send_bincode(&process2); +``` From 316d700cf820fa448898ce8df95b5c6011c33cc0 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 28 Jan 2025 17:43:32 -0800 Subject: [PATCH 10/29] feat(hydro_lang): provide APIs for blanket-deploying locations (#1676) This makes it easy to implement patterns like deploying everything to localhost. --- hydro_lang/src/builder/built.rs | 55 ++++++++++++++++++----- hydro_lang/src/builder/deploy.rs | 56 +++++++++++++++++++++--- hydro_lang/src/builder/mod.rs | 50 +++++++++++++++++---- hydro_lang/src/deploy/in_memory_graph.rs | 8 ++++ hydro_lang/src/deploy/mod.rs | 4 ++ hydro_lang/src/ir.rs | 4 +- hydro_lang/src/test_util.rs | 32 ++++++++++++++ 7 files changed, 183 insertions(+), 26 deletions(-) diff --git a/hydro_lang/src/builder/built.rs b/hydro_lang/src/builder/built.rs index e9a8bdca3f9..42f84242439 100644 --- a/hydro_lang/src/builder/built.rs +++ b/hydro_lang/src/builder/built.rs @@ -12,8 +12,9 @@ use crate::staging_util::Invariant; pub struct BuiltFlow<'a> { pub(super) ir: Vec, - pub(super) processes: Vec, - pub(super) clusters: Vec, + pub(super) process_id_name: Vec<(usize, String)>, + pub(super) cluster_id_name: Vec<(usize, String)>, + pub(super) external_id_name: Vec<(usize, String)>, pub(super) used: bool, pub(super) _phantom: Invariant<'a>, @@ -54,8 +55,9 @@ impl<'a> BuiltFlow<'a> { self.used = true; BuiltFlow { ir: f(std::mem::take(&mut self.ir)), - processes: std::mem::take(&mut self.processes), - clusters: std::mem::take(&mut self.clusters), + process_id_name: std::mem::take(&mut self.process_id_name), + cluster_id_name: std::mem::take(&mut self.cluster_id_name), + external_id_name: std::mem::take(&mut self.external_id_name), used: false, _phantom: PhantomData, } @@ -69,18 +71,27 @@ impl<'a> BuiltFlow<'a> { fn into_deploy>(mut self) -> DeployFlow<'a, D> { self.used = true; let processes = if D::has_trivial_node() { - self.processes + self.process_id_name .iter() - .map(|id| (*id, D::trivial_process(*id))) + .map(|id| (id.0, D::trivial_process(id.0))) .collect() } else { HashMap::new() }; let clusters = if D::has_trivial_node() { - self.clusters + self.cluster_id_name .iter() - .map(|id| (*id, D::trivial_cluster(*id))) + .map(|id| (id.0, D::trivial_cluster(id.0))) + .collect() + } else { + HashMap::new() + }; + + let externals = if D::has_trivial_node() { + self.external_id_name + .iter() + .map(|id| (id.0, D::trivial_external(id.0))) .collect() } else { HashMap::new() @@ -88,9 +99,12 @@ impl<'a> BuiltFlow<'a> { DeployFlow { ir: std::mem::take(&mut self.ir), - nodes: processes, + processes, + process_id_name: std::mem::take(&mut self.process_id_name), clusters, - externals: HashMap::new(), + cluster_id_name: std::mem::take(&mut self.cluster_id_name), + externals, + external_id_name: std::mem::take(&mut self.external_id_name), used: false, _phantom: PhantomData, } @@ -104,6 +118,13 @@ impl<'a> BuiltFlow<'a> { self.into_deploy().with_process(process, spec) } + pub fn with_remaining_processes, S: IntoProcessSpec<'a, D> + 'a>( + self, + spec: impl Fn() -> S, + ) -> DeployFlow<'a, D> { + self.into_deploy().with_remaining_processes(spec) + } + pub fn with_external>( self, process: &ExternalProcess

, @@ -112,6 +133,13 @@ impl<'a> BuiltFlow<'a> { self.into_deploy().with_external(process, spec) } + pub fn with_remaining_externals, S: ExternalSpec<'a, D> + 'a>( + self, + spec: impl Fn() -> S, + ) -> DeployFlow<'a, D> { + self.into_deploy().with_remaining_externals(spec) + } + pub fn with_cluster>( self, cluster: &Cluster, @@ -120,6 +148,13 @@ impl<'a> BuiltFlow<'a> { self.into_deploy().with_cluster(cluster, spec) } + pub fn with_remaining_clusters, S: ClusterSpec<'a, D> + 'a>( + self, + spec: impl Fn() -> S, + ) -> DeployFlow<'a, D> { + self.into_deploy().with_remaining_clusters(spec) + } + pub fn compile>(self, env: &D::CompileEnv) -> CompiledFlow<'a, D::GraphId> { self.into_deploy::().compile(env) } diff --git a/hydro_lang/src/builder/deploy.rs b/hydro_lang/src/builder/deploy.rs index 0b3c71890e4..748dbf3b9df 100644 --- a/hydro_lang/src/builder/deploy.rs +++ b/hydro_lang/src/builder/deploy.rs @@ -25,9 +25,19 @@ use crate::staging_util::Invariant; pub struct DeployFlow<'a, D: LocalDeploy<'a>> { pub(super) ir: Vec, - pub(super) nodes: HashMap, + + /// Deployed instances of each process in the flow + pub(super) processes: HashMap, + + /// Lists all the processes that were created in the flow, same ID as `processes` + /// but with the type name of the tag. + pub(super) process_id_name: Vec<(usize, String)>, + pub(super) externals: HashMap, + pub(super) external_id_name: Vec<(usize, String)>, + pub(super) clusters: HashMap, + pub(super) cluster_id_name: Vec<(usize, String)>, pub(super) used: bool, pub(super) _phantom: Invariant<'a, D>, @@ -52,13 +62,25 @@ impl<'a, D: LocalDeploy<'a>> DeployFlow<'a, D> { spec: impl IntoProcessSpec<'a, D>, ) -> Self { let tag_name = std::any::type_name::

().to_string(); - self.nodes.insert( + self.processes.insert( process.id, spec.into_process_spec().build(process.id, &tag_name), ); self } + pub fn with_remaining_processes + 'a>( + mut self, + spec: impl Fn() -> S, + ) -> Self { + for (id, name) in &self.process_id_name { + self.processes + .insert(*id, spec().into_process_spec().build(*id, name)); + } + + self + } + pub fn with_external

( mut self, process: &ExternalProcess

, @@ -70,6 +92,17 @@ impl<'a, D: LocalDeploy<'a>> DeployFlow<'a, D> { self } + pub fn with_remaining_externals + 'a>( + mut self, + spec: impl Fn() -> S, + ) -> Self { + for (id, name) in &self.external_id_name { + self.externals.insert(*id, spec().build(*id, name)); + } + + self + } + pub fn with_cluster(mut self, cluster: &Cluster, spec: impl ClusterSpec<'a, D>) -> Self { let tag_name = std::any::type_name::().to_string(); self.clusters @@ -77,6 +110,17 @@ impl<'a, D: LocalDeploy<'a>> DeployFlow<'a, D> { self } + pub fn with_remaining_clusters + 'a>( + mut self, + spec: impl Fn() -> S, + ) -> Self { + for (id, name) in &self.cluster_id_name { + self.clusters.insert(*id, spec().build(*id, name)); + } + + self + } + pub fn compile_no_network(mut self) -> CompiledFlow<'a, D::GraphId> { self.used = true; @@ -99,7 +143,7 @@ impl<'a, D: Deploy<'a>> DeployFlow<'a, D> { leaf.compile_network::( env, &mut seen_tees, - &self.nodes, + &self.processes, &self.clusters, &self.externals, ) @@ -130,7 +174,7 @@ impl<'a, D: Deploy<'a>> DeployFlow<'a, D> { let #self_id_ident = #self_id_expr; }); - for other_location in self.nodes.keys().chain(self.clusters.keys()) { + for other_location in self.processes.keys().chain(self.clusters.keys()) { let other_id_ident = syn::Ident::new( &format!("__hydro_lang_cluster_ids_{}", c_id), Span::call_site(), @@ -160,7 +204,7 @@ impl<'a, D: Deploy<'a, CompileEnv = ()>> DeployFlow<'a, D> { leaf.compile_network::( &(), &mut seen_tees_instantiate, - &self.nodes, + &self.processes, &self.clusters, &self.externals, ) @@ -172,7 +216,7 @@ impl<'a, D: Deploy<'a, CompileEnv = ()>> DeployFlow<'a, D> { let mut meta = D::Meta::default(); let (mut processes, mut clusters, mut externals) = ( - std::mem::take(&mut self.nodes) + std::mem::take(&mut self.processes) .into_iter() .filter_map(|(node_id, node)| { if let Some(ir) = compiled.remove(&node_id) { diff --git a/hydro_lang/src/builder/mod.rs b/hydro_lang/src/builder/mod.rs index 894fb99a714..6aa6859cd86 100644 --- a/hydro_lang/src/builder/mod.rs +++ b/hydro_lang/src/builder/mod.rs @@ -1,3 +1,4 @@ +use std::any::type_name; use std::cell::RefCell; use std::collections::HashMap; use std::marker::PhantomData; @@ -44,8 +45,9 @@ pub const FLOW_USED_MESSAGE: &str = "Attempted to add a leaf to a flow that has pub struct FlowBuilder<'a> { flow_state: FlowState, - nodes: RefCell>, - clusters: RefCell>, + processes: RefCell>, + clusters: RefCell>, + externals: RefCell>, next_node_id: RefCell, @@ -87,8 +89,9 @@ impl<'a> FlowBuilder<'a> { cycle_counts: HashMap::new(), next_clock_id: 0, })), - nodes: RefCell::new(vec![]), + processes: RefCell::new(vec![]), clusters: RefCell::new(vec![]), + externals: RefCell::new(vec![]), next_node_id: RefCell::new(0), finalized: false, _phantom: PhantomData, @@ -101,8 +104,9 @@ impl<'a> FlowBuilder<'a> { built::BuiltFlow { ir: self.flow_state.borrow_mut().leaves.take().unwrap(), - processes: self.nodes.replace(vec![]), - clusters: self.clusters.replace(vec![]), + process_id_name: self.processes.replace(vec![]), + cluster_id_name: self.clusters.replace(vec![]), + external_id_name: self.externals.replace(vec![]), used: false, _phantom: PhantomData, } @@ -130,7 +134,9 @@ impl<'a> FlowBuilder<'a> { let id = *next_node_id; *next_node_id += 1; - self.nodes.borrow_mut().push(id); + self.processes + .borrow_mut() + .push((id, type_name::

().to_string())); Process { id, @@ -144,7 +150,9 @@ impl<'a> FlowBuilder<'a> { let id = *next_node_id; *next_node_id += 1; - self.nodes.borrow_mut().push(id); + self.externals + .borrow_mut() + .push((id, type_name::

().to_string())); ExternalProcess { id, @@ -158,7 +166,9 @@ impl<'a> FlowBuilder<'a> { let id = *next_node_id; *next_node_id += 1; - self.clusters.borrow_mut().push(id); + self.clusters + .borrow_mut() + .push((id, type_name::().to_string())); Cluster { id, @@ -176,6 +186,14 @@ impl<'a> FlowBuilder<'a> { self.with_default_optimize().with_process(process, spec) } + #[cfg(feature = "build")] + pub fn with_remaining_processes, S: IntoProcessSpec<'a, D> + 'a>( + self, + spec: impl Fn() -> S, + ) -> DeployFlow<'a, D> { + self.with_default_optimize().with_remaining_processes(spec) + } + #[cfg(feature = "build")] pub fn with_external>( self, @@ -185,6 +203,14 @@ impl<'a> FlowBuilder<'a> { self.with_default_optimize().with_external(process, spec) } + #[cfg(feature = "build")] + pub fn with_remaining_externals, S: ExternalSpec<'a, D> + 'a>( + self, + spec: impl Fn() -> S, + ) -> DeployFlow<'a, D> { + self.with_default_optimize().with_remaining_externals(spec) + } + #[cfg(feature = "build")] pub fn with_cluster>( self, @@ -194,6 +220,14 @@ impl<'a> FlowBuilder<'a> { self.with_default_optimize().with_cluster(cluster, spec) } + #[cfg(feature = "build")] + pub fn with_remaining_clusters, S: ClusterSpec<'a, D> + 'a>( + self, + spec: impl Fn() -> S, + ) -> DeployFlow<'a, D> { + self.with_default_optimize().with_remaining_clusters(spec) + } + #[cfg(feature = "build")] pub fn compile>(self, env: &D::CompileEnv) -> CompiledFlow<'a, D::GraphId> { self.with_default_optimize::().compile(env) diff --git a/hydro_lang/src/deploy/in_memory_graph.rs b/hydro_lang/src/deploy/in_memory_graph.rs index e9af6c9f9bc..b6e5d6b3ab9 100644 --- a/hydro_lang/src/deploy/in_memory_graph.rs +++ b/hydro_lang/src/deploy/in_memory_graph.rs @@ -22,6 +22,10 @@ impl LocalDeploy<'_> for SingleProcessGraph { fn trivial_cluster(_id: usize) -> Self::Cluster { SingleNode {} } + + fn trivial_external(_id: usize) -> Self::ExternalProcess { + SingleNode {} + } } impl ProcessSpec<'_, SingleProcessGraph> for () { @@ -74,6 +78,10 @@ impl LocalDeploy<'_> for MultiGraph { fn trivial_cluster(_id: usize) -> Self::Cluster { MultiNode {} } + + fn trivial_external(_id: usize) -> Self::ExternalProcess { + MultiNode {} + } } impl ProcessSpec<'_, MultiGraph> for () { diff --git a/hydro_lang/src/deploy/mod.rs b/hydro_lang/src/deploy/mod.rs index 3e76af6ffa5..5ec1e175c83 100644 --- a/hydro_lang/src/deploy/mod.rs +++ b/hydro_lang/src/deploy/mod.rs @@ -50,6 +50,10 @@ pub trait LocalDeploy<'a> { fn trivial_cluster(_id: usize) -> Self::Cluster { panic!("No trivial cluster") } + + fn trivial_external(_id: usize) -> Self::ExternalProcess { + panic!("No trivial external") + } } pub trait Deploy<'a> { diff --git a/hydro_lang/src/ir.rs b/hydro_lang/src/ir.rs index b1a68f19960..2ffd432d0f2 100644 --- a/hydro_lang/src/ir.rs +++ b/hydro_lang/src/ir.rs @@ -95,13 +95,13 @@ impl HydroLeaf { self, compile_env: &D::CompileEnv, seen_tees: &mut SeenTees, - nodes: &HashMap, + processes: &HashMap, clusters: &HashMap, externals: &HashMap, ) -> HydroLeaf { self.transform_children( |n, s| { - n.compile_network::(compile_env, s, nodes, clusters, externals); + n.compile_network::(compile_env, s, processes, clusters, externals); }, seen_tees, ) diff --git a/hydro_lang/src/test_util.rs b/hydro_lang/src/test_util.rs index f1712848ea5..e862e3bcc38 100644 --- a/hydro_lang/src/test_util.rs +++ b/hydro_lang/src/test_util.rs @@ -7,6 +7,38 @@ use serde::Serialize; use crate::{FlowBuilder, Process, Stream, Unbounded}; +pub async fn multi_location_test< + 'a, + O: Serialize + DeserializeOwned + 'static, + C: Future, + OutOrder, +>( + thunk: impl FnOnce( + &FlowBuilder<'a>, + &Process<'a, ()>, + ) -> Stream, Unbounded, OutOrder>, + check: impl FnOnce(Pin>>) -> C, +) { + let mut deployment = hydro_deploy::Deployment::new(); + let flow = FlowBuilder::new(); + let process = flow.process::<()>(); + let external = flow.external_process::<()>(); + let out = thunk(&flow, &process); + let out_port = out.send_bincode_external(&external); + let nodes = flow + .with_remaining_processes(|| deployment.Localhost()) + .with_remaining_clusters(|| vec![deployment.Localhost(); 4]) + .with_external(&external, deployment.Localhost()) + .deploy(&mut deployment); + + deployment.deploy().await.unwrap(); + + let external_out = nodes.connect_source_bincode(out_port).await; + deployment.start().await.unwrap(); + + check(external_out).await; +} + pub async fn stream_transform_test< 'a, O: Serialize + DeserializeOwned + 'static, From e043cedf1999670bd5a5a93960748c89ca092f2b Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 28 Jan 2025 17:44:36 -0800 Subject: [PATCH 11/29] feat(hydro_lang): make `Stream::cloned` work with borrowed elements (#1678) `Stream::cloned` is only really useful if the input has `&T` elements but we want `T`, so this updates the signature to do that. --- hydro_lang/src/stream.rs | 46 +++++++++++++++++++++------------------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/hydro_lang/src/stream.rs b/hydro_lang/src/stream.rs index db14cfe40c5..6ea29059329 100644 --- a/hydro_lang/src/stream.rs +++ b/hydro_lang/src/stream.rs @@ -252,28 +252,6 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } - /// Clone each element of the stream; akin to `map(q!(|d| d.clone()))`. - /// - /// # Example - /// ```rust - /// # use hydro_lang::*; - /// # use dfir_rs::futures::StreamExt; - /// # tokio_test::block_on(test_util::stream_transform_test(|process| { - /// process.source_iter(q!(vec![1..3])).cloned() - /// # }, |mut stream| async move { - /// // 1, 2, 3 - /// # for w in vec![1..3] { - /// # assert_eq!(stream.next().await.unwrap(), w); - /// # } - /// # })); - /// ``` - pub fn cloned(self) -> Stream - where - T: Clone, - { - self.map(q!(|d| d.clone())) - } - /// For each item `i` in the input stream, transform `i` using `f` and then treat the /// result as an [`Iterator`] to produce items one by one. The implementation for [`Iterator`] /// for the output type `U` must produce items in a **deterministic** order. @@ -614,6 +592,30 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { } } +impl<'a, T, L: Location<'a>, B, Order> Stream<&T, L, B, Order> { + /// Clone each element of the stream; akin to `map(q!(|d| d.clone()))`. + /// + /// # Example + /// ```rust + /// # use hydro_lang::*; + /// # use dfir_rs::futures::StreamExt; + /// # tokio_test::block_on(test_util::stream_transform_test(|process| { + /// process.source_iter(q!(&[1, 2, 3])).cloned() + /// # }, |mut stream| async move { + /// // 1, 2, 3 + /// # for w in vec![1, 2, 3] { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); + /// ``` + pub fn cloned(self) -> Stream + where + T: Clone, + { + self.map(q!(|d| d.clone())) + } +} + impl<'a, T, L: Location<'a>, B, Order> Stream where Order: MinOrder, From eee28d3a17ea542c69a2d7e535c38333f42d4398 Mon Sep 17 00:00:00 2001 From: David Chu Date: Wed, 29 Jan 2025 14:45:02 +0800 Subject: [PATCH 12/29] feat(hydro_lang): Add metadata field to HydroNode (#1632) --- hydro_lang/src/builder/mod.rs | 26 +- hydro_lang/src/ir.rs | 360 +- hydro_lang/src/location/external_process.rs | 58 +- hydro_lang/src/location/mod.rs | 68 +- hydro_lang/src/location/tick.rs | 11 +- hydro_lang/src/optional.rs | 199 +- hydro_lang/src/rewrites/persist_pullup.rs | 198 +- hydro_lang/src/rewrites/profiler.rs | 2 + ...p__tests__persist_pullup_behind_tee-2.snap | 76 +- ...lup__tests__persist_pullup_behind_tee.snap | 176 +- ...__tests__persist_pullup_through_map-2.snap | 16 + ...up__tests__persist_pullup_through_map.snap | 42 +- ...ts__profiler_wrapping_all_operators-2.snap | 32 + ...ests__profiler_wrapping_all_operators.snap | 42 +- ...properties__tests__property_optimized.snap | 24 + hydro_lang/src/singleton.rs | 142 +- hydro_lang/src/stream.rs | 252 +- hydro_std/Cargo.toml | 4 +- ...ter__compute_pi__tests__compute_pi_ir.snap | 126 +- ...er__many_to_many__tests__many_to_many.snap | 24 + ...ter__map_reduce__tests__map_reduce_ir.snap | 102 +- ...cluster__paxos_bench__tests__paxos_ir.snap | 3949 +++++++++++++++-- ...simple_cluster__tests__simple_cluster.snap | 102 +- ...rst_ten__tests__first_ten_distributed.snap | 40 + 24 files changed, 5310 insertions(+), 761 deletions(-) diff --git a/hydro_lang/src/builder/mod.rs b/hydro_lang/src/builder/mod.rs index 6aa6859cd86..78895e22bde 100644 --- a/hydro_lang/src/builder/mod.rs +++ b/hydro_lang/src/builder/mod.rs @@ -37,6 +37,9 @@ pub struct FlowStateInner { /// Counters for clock IDs. pub(crate) next_clock_id: usize, + + /// Counter for unique HydroNode IDs. + pub(crate) next_node_id: usize, } pub type FlowState = Rc>; @@ -49,7 +52,7 @@ pub struct FlowBuilder<'a> { clusters: RefCell>, externals: RefCell>, - next_node_id: RefCell, + next_location_id: RefCell, /// Tracks whether this flow has been finalized; it is an error to /// drop without finalizing. @@ -88,11 +91,12 @@ impl<'a> FlowBuilder<'a> { next_external_out: 0, cycle_counts: HashMap::new(), next_clock_id: 0, + next_node_id: 0, })), processes: RefCell::new(vec![]), clusters: RefCell::new(vec![]), externals: RefCell::new(vec![]), - next_node_id: RefCell::new(0), + next_location_id: RefCell::new(0), finalized: false, _phantom: PhantomData, } @@ -130,9 +134,9 @@ impl<'a> FlowBuilder<'a> { } pub fn process

(&self) -> Process<'a, P> { - let mut next_node_id = self.next_node_id.borrow_mut(); - let id = *next_node_id; - *next_node_id += 1; + let mut next_location_id = self.next_location_id.borrow_mut(); + let id = *next_location_id; + *next_location_id += 1; self.processes .borrow_mut() @@ -146,9 +150,9 @@ impl<'a> FlowBuilder<'a> { } pub fn external_process

(&self) -> ExternalProcess<'a, P> { - let mut next_node_id = self.next_node_id.borrow_mut(); - let id = *next_node_id; - *next_node_id += 1; + let mut next_location_id = self.next_location_id.borrow_mut(); + let id = *next_location_id; + *next_location_id += 1; self.externals .borrow_mut() @@ -162,9 +166,9 @@ impl<'a> FlowBuilder<'a> { } pub fn cluster(&self) -> Cluster<'a, C> { - let mut next_node_id = self.next_node_id.borrow_mut(); - let id = *next_node_id; - *next_node_id += 1; + let mut next_location_id = self.next_location_id.borrow_mut(); + let id = *next_location_id; + *next_location_id += 1; self.clusters .borrow_mut() diff --git a/hydro_lang/src/ir.rs b/hydro_lang/src/ir.rs index 2ffd432d0f2..adada879126 100644 --- a/hydro_lang/src/ir.rs +++ b/hydro_lang/src/ir.rs @@ -4,6 +4,7 @@ use std::cell::RefCell; use std::collections::BTreeMap; use std::collections::HashMap; use std::fmt::Debug; +use std::hash::{Hash, Hasher}; use std::ops::Deref; use std::rc::Rc; @@ -20,7 +21,7 @@ use syn::parse_quote; use crate::deploy::{Deploy, RegisterPort}; use crate::location::LocationId; -#[derive(Clone)] +#[derive(Clone, Hash)] pub struct DebugExpr(pub syn::Expr); impl From for DebugExpr { @@ -49,6 +50,35 @@ impl Debug for DebugExpr { } } +#[derive(Clone, Hash)] +pub struct DebugType(pub syn::Type); + +impl From for DebugType { + fn from(t: syn::Type) -> DebugType { + DebugType(t) + } +} + +impl Deref for DebugType { + type Target = syn::Type; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl ToTokens for DebugType { + fn to_tokens(&self, tokens: &mut TokenStream) { + self.0.to_tokens(tokens); + } +} + +impl Debug for DebugType { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.to_token_stream()) + } +} + pub enum DebugInstantiate { Building(), Finalized(syn::Expr, syn::Expr, Option>), @@ -60,8 +90,14 @@ impl Debug for DebugInstantiate { } } +impl Hash for DebugInstantiate { + fn hash(&self, _state: &mut H) { + // Do nothing + } +} + /// A source in a Hydro graph, where data enters the graph. -#[derive(Debug)] +#[derive(Debug, Hash)] pub enum HydroSource { Stream(DebugExpr), ExternalNetwork(), @@ -72,7 +108,7 @@ pub enum HydroSource { /// An leaf in a Hydro graph, which is an pipeline that doesn't emit /// any downstream values. Traversals over the dataflow graph and /// generating Hydroflow IR start from leaves. -#[derive(Debug)] +#[derive(Debug, Hash)] pub enum HydroLeaf { ForEach { f: DebugExpr, @@ -261,85 +297,159 @@ impl Debug for TeeNode { } } +impl Hash for TeeNode { + fn hash(&self, state: &mut H) { + self.0.borrow_mut().hash(state); + } +} + +#[derive(Debug, Clone, Hash)] +pub struct HydroNodeMetadata { + pub location_kind: LocationId, + pub output_type: Option, +} + /// An intermediate node in a Hydro graph, which consumes data /// from upstream nodes and emits data to downstream nodes. -#[derive(Debug)] +#[derive(Debug, Hash)] pub enum HydroNode { Placeholder, Source { source: HydroSource, location_kind: LocationId, + metadata: HydroNodeMetadata, }, CycleSource { ident: syn::Ident, location_kind: LocationId, + metadata: HydroNodeMetadata, }, Tee { inner: TeeNode, + metadata: HydroNodeMetadata, + }, + + Persist { + inner: Box, + metadata: HydroNodeMetadata, + }, + + Unpersist { + inner: Box, + metadata: HydroNodeMetadata, + }, + + Delta { + inner: Box, + metadata: HydroNodeMetadata, + }, + + Chain { + first: Box, + second: Box, + metadata: HydroNodeMetadata, }, - Persist(Box), - Unpersist(Box), - Delta(Box), + CrossProduct { + left: Box, + right: Box, + metadata: HydroNodeMetadata, + }, + + CrossSingleton { + left: Box, + right: Box, + metadata: HydroNodeMetadata, + }, + + Join { + left: Box, + right: Box, + metadata: HydroNodeMetadata, + }, + + Difference { + pos: Box, + neg: Box, + metadata: HydroNodeMetadata, + }, - Chain(Box, Box), - CrossProduct(Box, Box), - CrossSingleton(Box, Box), - Join(Box, Box), - Difference(Box, Box), - AntiJoin(Box, Box), + AntiJoin { + pos: Box, + neg: Box, + metadata: HydroNodeMetadata, + }, Map { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, FlatMap { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, Filter { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, FilterMap { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, - DeferTick(Box), + DeferTick { + input: Box, + metadata: HydroNodeMetadata, + }, Enumerate { is_static: bool, input: Box, + metadata: HydroNodeMetadata, }, Inspect { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, - Unique(Box), + Unique { + input: Box, + metadata: HydroNodeMetadata, + }, - Sort(Box), + Sort { + input: Box, + metadata: HydroNodeMetadata, + }, Fold { init: DebugExpr, acc: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, FoldKeyed { init: DebugExpr, acc: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, Reduce { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, ReduceKeyed { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, Network { @@ -351,6 +461,7 @@ pub enum HydroNode { instantiate_fn: DebugInstantiate, deserialize_fn: Option, input: Box, + metadata: HydroNodeMetadata, }, } @@ -438,7 +549,7 @@ impl<'a> HydroNode { HydroNode::CycleSource { .. } => {} - HydroNode::Tee { inner } => { + HydroNode::Tee { inner, .. } => { if let Some(transformed) = seen_tees.get(&(inner.0.as_ref() as *const RefCell)) { @@ -456,33 +567,33 @@ impl<'a> HydroNode { } } - HydroNode::Persist(inner) => transform(inner.as_mut(), seen_tees), - HydroNode::Unpersist(inner) => transform(inner.as_mut(), seen_tees), - HydroNode::Delta(inner) => transform(inner.as_mut(), seen_tees), + HydroNode::Persist { inner, .. } => transform(inner.as_mut(), seen_tees), + HydroNode::Unpersist { inner, .. } => transform(inner.as_mut(), seen_tees), + HydroNode::Delta { inner, .. } => transform(inner.as_mut(), seen_tees), - HydroNode::Chain(left, right) => { - transform(left.as_mut(), seen_tees); - transform(right.as_mut(), seen_tees); + HydroNode::Chain { first, second, .. } => { + transform(first.as_mut(), seen_tees); + transform(second.as_mut(), seen_tees); } - HydroNode::CrossProduct(left, right) => { + HydroNode::CrossProduct { left, right, .. } => { transform(left.as_mut(), seen_tees); transform(right.as_mut(), seen_tees); } - HydroNode::CrossSingleton(left, right) => { + HydroNode::CrossSingleton { left, right, .. } => { transform(left.as_mut(), seen_tees); transform(right.as_mut(), seen_tees); } - HydroNode::Join(left, right) => { + HydroNode::Join { left, right, .. } => { transform(left.as_mut(), seen_tees); transform(right.as_mut(), seen_tees); } - HydroNode::Difference(left, right) => { - transform(left.as_mut(), seen_tees); - transform(right.as_mut(), seen_tees); + HydroNode::Difference { pos, neg, .. } => { + transform(pos.as_mut(), seen_tees); + transform(neg.as_mut(), seen_tees); } - HydroNode::AntiJoin(left, right) => { - transform(left.as_mut(), seen_tees); - transform(right.as_mut(), seen_tees); + HydroNode::AntiJoin { pos, neg, .. } => { + transform(pos.as_mut(), seen_tees); + transform(neg.as_mut(), seen_tees); } HydroNode::Map { input, .. } => { @@ -497,10 +608,10 @@ impl<'a> HydroNode { HydroNode::FilterMap { input, .. } => { transform(input.as_mut(), seen_tees); } - HydroNode::Sort(input) => { + HydroNode::Sort { input, .. } => { transform(input.as_mut(), seen_tees); } - HydroNode::DeferTick(input) => { + HydroNode::DeferTick { input, .. } => { transform(input.as_mut(), seen_tees); } HydroNode::Enumerate { input, .. } => { @@ -510,7 +621,7 @@ impl<'a> HydroNode { transform(input.as_mut(), seen_tees); } - HydroNode::Unique(input) => { + HydroNode::Unique { input, .. } => { transform(input.as_mut(), seen_tees); } @@ -546,7 +657,7 @@ impl<'a> HydroNode { panic!() } - HydroNode::Persist(inner) => { + HydroNode::Persist { inner, .. } => { let (inner_ident, location) = inner.emit(graph_builders, built_tees, next_stmt_id); let persist_id = *next_stmt_id; @@ -563,11 +674,11 @@ impl<'a> HydroNode { (persist_ident, location) } - HydroNode::Unpersist(_) => { + HydroNode::Unpersist { .. } => { panic!("Unpersist is a marker node and should have been optimized away. This is likely a compiler bug.") } - HydroNode::Delta(inner) => { + HydroNode::Delta { inner, .. } => { let (inner_ident, location) = inner.emit(graph_builders, built_tees, next_stmt_id); let delta_id = *next_stmt_id; @@ -587,6 +698,7 @@ impl<'a> HydroNode { HydroNode::Source { source, location_kind, + .. } => { let location_id = match location_kind { LocationId::Process(id) => id, @@ -640,6 +752,7 @@ impl<'a> HydroNode { HydroNode::CycleSource { ident, location_kind, + .. } => { let location_id = match location_kind.root() { LocationId::Process(id) => id, @@ -651,7 +764,7 @@ impl<'a> HydroNode { (ident.clone(), *location_id) } - HydroNode::Tee { inner } => { + HydroNode::Tee { inner, .. } => { if let Some(ret) = built_tees.get(&(inner.0.as_ref() as *const RefCell)) { ret.clone() @@ -682,14 +795,14 @@ impl<'a> HydroNode { } } - HydroNode::Chain(left, right) => { - let (left_ident, left_location_id) = - left.emit(graph_builders, built_tees, next_stmt_id); - let (right_ident, right_location_id) = - right.emit(graph_builders, built_tees, next_stmt_id); + HydroNode::Chain { first, second, .. } => { + let (first_ident, first_location_id) = + first.emit(graph_builders, built_tees, next_stmt_id); + let (second_ident, second_location_id) = + second.emit(graph_builders, built_tees, next_stmt_id); assert_eq!( - left_location_id, right_location_id, + first_location_id, second_location_id, "chain inputs must be in the same location" ); @@ -699,23 +812,23 @@ impl<'a> HydroNode { let chain_ident = syn::Ident::new(&format!("stream_{}", union_id), Span::call_site()); - let builder = graph_builders.entry(left_location_id).or_default(); + let builder = graph_builders.entry(first_location_id).or_default(); builder.add_statement(parse_quote! { #chain_ident = chain(); }); builder.add_statement(parse_quote! { - #left_ident -> [0]#chain_ident; + #first_ident -> [0]#chain_ident; }); builder.add_statement(parse_quote! { - #right_ident -> [1]#chain_ident; + #second_ident -> [1]#chain_ident; }); - (chain_ident, left_location_id) + (chain_ident, first_location_id) } - HydroNode::CrossSingleton(left, right) => { + HydroNode::CrossSingleton { left, right, .. } => { let (left_ident, left_location_id) = left.emit(graph_builders, built_tees, next_stmt_id); let (right_ident, right_location_id) = @@ -748,27 +861,28 @@ impl<'a> HydroNode { (cross_ident, left_location_id) } - HydroNode::CrossProduct(..) | HydroNode::Join(..) => { - let operator: syn::Ident = if matches!(self, HydroNode::CrossProduct(..)) { + HydroNode::CrossProduct { .. } | HydroNode::Join { .. } => { + let operator: syn::Ident = if matches!(self, HydroNode::CrossProduct { .. }) { parse_quote!(cross_join_multiset) } else { parse_quote!(join_multiset) }; - let (HydroNode::CrossProduct(left, right) | HydroNode::Join(left, right)) = self + let (HydroNode::CrossProduct { left, right, .. } + | HydroNode::Join { left, right, .. }) = self else { unreachable!() }; - let (left_inner, left_was_persist) = if let HydroNode::Persist(left) = left.as_ref() - { - (left, true) - } else { - (left, false) - }; + let (left_inner, left_was_persist) = + if let HydroNode::Persist { inner: left, .. } = left.as_ref() { + (left, true) + } else { + (left, false) + }; let (right_inner, right_was_persist) = - if let HydroNode::Persist(right) = right.as_ref() { + if let HydroNode::Persist { inner: right, .. } = right.as_ref() { (right, true) } else { (right, false) @@ -826,31 +940,33 @@ impl<'a> HydroNode { (stream_ident, left_location_id) } - HydroNode::Difference(..) | HydroNode::AntiJoin(..) => { - let operator: syn::Ident = if matches!(self, HydroNode::Difference(..)) { + HydroNode::Difference { .. } | HydroNode::AntiJoin { .. } => { + let operator: syn::Ident = if matches!(self, HydroNode::Difference { .. }) { parse_quote!(difference_multiset) } else { parse_quote!(anti_join_multiset) }; - let (HydroNode::Difference(left, right) | HydroNode::AntiJoin(left, right)) = self + let (HydroNode::Difference { pos, neg, .. } | HydroNode::AntiJoin { pos, neg, .. }) = + self else { unreachable!() }; - let (right, right_was_persist) = if let HydroNode::Persist(right) = right.as_ref() { - (right, true) - } else { - (right, false) - }; + let (neg, neg_was_persist) = + if let HydroNode::Persist { inner: neg, .. } = neg.as_ref() { + (neg, true) + } else { + (neg, false) + }; - let (left_ident, left_location_id) = - left.emit(graph_builders, built_tees, next_stmt_id); - let (right_ident, right_location_id) = - right.emit(graph_builders, built_tees, next_stmt_id); + let (pos_ident, pos_location_id) = + pos.emit(graph_builders, built_tees, next_stmt_id); + let (neg_ident, neg_location_id) = + neg.emit(graph_builders, built_tees, next_stmt_id); assert_eq!( - left_location_id, right_location_id, + pos_location_id, neg_location_id, "difference / anti join inputs must be in the same location" ); @@ -860,9 +976,9 @@ impl<'a> HydroNode { let stream_ident = syn::Ident::new(&format!("stream_{}", stream_id), Span::call_site()); - let builder = graph_builders.entry(left_location_id).or_default(); + let builder = graph_builders.entry(pos_location_id).or_default(); - if right_was_persist { + if neg_was_persist { builder.add_statement(parse_quote! { #stream_ident = #operator::<'tick, 'static>(); }); @@ -873,17 +989,17 @@ impl<'a> HydroNode { } builder.add_statement(parse_quote! { - #left_ident -> [pos]#stream_ident; + #pos_ident -> [pos]#stream_ident; }); builder.add_statement(parse_quote! { - #right_ident -> [neg]#stream_ident; + #neg_ident -> [neg]#stream_ident; }); - (stream_ident, left_location_id) + (stream_ident, pos_location_id) } - HydroNode::Map { f, input } => { + HydroNode::Map { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -900,7 +1016,7 @@ impl<'a> HydroNode { (map_ident, input_location_id) } - HydroNode::FlatMap { f, input } => { + HydroNode::FlatMap { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -918,7 +1034,7 @@ impl<'a> HydroNode { (flat_map_ident, input_location_id) } - HydroNode::Filter { f, input } => { + HydroNode::Filter { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -936,7 +1052,7 @@ impl<'a> HydroNode { (filter_ident, input_location_id) } - HydroNode::FilterMap { f, input } => { + HydroNode::FilterMap { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -954,7 +1070,7 @@ impl<'a> HydroNode { (filter_map_ident, input_location_id) } - HydroNode::Sort(input) => { + HydroNode::Sort { input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -971,7 +1087,7 @@ impl<'a> HydroNode { (sort_ident, input_location_id) } - HydroNode::DeferTick(input) => { + HydroNode::DeferTick { input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -989,7 +1105,9 @@ impl<'a> HydroNode { (defer_tick_ident, input_location_id) } - HydroNode::Enumerate { is_static, input } => { + HydroNode::Enumerate { + is_static, input, .. + } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1014,7 +1132,7 @@ impl<'a> HydroNode { (enumerate_ident, input_location_id) } - HydroNode::Inspect { f, input } => { + HydroNode::Inspect { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1032,7 +1150,7 @@ impl<'a> HydroNode { (inspect_ident, input_location_id) } - HydroNode::Unique(input) => { + HydroNode::Unique { input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1057,17 +1175,22 @@ impl<'a> HydroNode { parse_quote!(fold_keyed) }; - let (HydroNode::Fold { init, acc, input } - | HydroNode::FoldKeyed { init, acc, input }) = self + let (HydroNode::Fold { + init, acc, input, .. + } + | HydroNode::FoldKeyed { + init, acc, input, .. + }) = self else { unreachable!() }; - let (input, input_was_persist) = if let HydroNode::Persist(input) = input.as_ref() { - (input, true) - } else { - (input, false) - }; + let (input, input_was_persist) = + if let HydroNode::Persist { inner: input, .. } = input.as_ref() { + (input, true) + } else { + (input, false) + }; let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1099,16 +1222,18 @@ impl<'a> HydroNode { parse_quote!(reduce_keyed) }; - let (HydroNode::Reduce { f, input } | HydroNode::ReduceKeyed { f, input }) = self + let (HydroNode::Reduce { f, input, .. } | HydroNode::ReduceKeyed { f, input, .. }) = + self else { unreachable!() }; - let (input, input_was_persist) = if let HydroNode::Persist(input) = input.as_ref() { - (input, true) - } else { - (input, false) - }; + let (input, input_was_persist) = + if let HydroNode::Persist { inner: input, .. } = input.as_ref() { + (input, true) + } else { + (input, false) + }; let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1142,6 +1267,7 @@ impl<'a> HydroNode { instantiate_fn, deserialize_fn: deserialize_pipeline, input, + .. } => { let (sink_expr, source_expr, _connect_fn) = match instantiate_fn { DebugInstantiate::Building() => { @@ -1196,6 +1322,40 @@ impl<'a> HydroNode { } } } + + pub fn metadata(&self) -> &HydroNodeMetadata { + match self { + HydroNode::Placeholder => { + panic!() + } + HydroNode::Source { metadata, .. } => metadata, + HydroNode::CycleSource { metadata, .. } => metadata, + HydroNode::Tee { metadata, .. } => metadata, + HydroNode::Persist { metadata, .. } => metadata, + HydroNode::Unpersist { metadata, .. } => metadata, + HydroNode::Delta { metadata, .. } => metadata, + HydroNode::Chain { metadata, .. } => metadata, + HydroNode::CrossProduct { metadata, .. } => metadata, + HydroNode::CrossSingleton { metadata, .. } => metadata, + HydroNode::Join { metadata, .. } => metadata, + HydroNode::Difference { metadata, .. } => metadata, + HydroNode::AntiJoin { metadata, .. } => metadata, + HydroNode::Map { metadata, .. } => metadata, + HydroNode::FlatMap { metadata, .. } => metadata, + HydroNode::Filter { metadata, .. } => metadata, + HydroNode::FilterMap { metadata, .. } => metadata, + HydroNode::DeferTick { metadata, .. } => metadata, + HydroNode::Enumerate { metadata, .. } => metadata, + HydroNode::Inspect { metadata, .. } => metadata, + HydroNode::Unique { metadata, .. } => metadata, + HydroNode::Sort { metadata, .. } => metadata, + HydroNode::Fold { metadata, .. } => metadata, + HydroNode::FoldKeyed { metadata, .. } => metadata, + HydroNode::Reduce { metadata, .. } => metadata, + HydroNode::ReduceKeyed { metadata, .. } => metadata, + HydroNode::Network { metadata, .. } => metadata, + } + } } #[cfg(feature = "build")] diff --git a/hydro_lang/src/location/external_process.rs b/hydro_lang/src/location/external_process.rs index 926e38de670..e18c9e2f0b2 100644 --- a/hydro_lang/src/location/external_process.rs +++ b/hydro_lang/src/location/external_process.rs @@ -110,19 +110,24 @@ impl<'a, P> ExternalProcess<'a, P> { }, Stream::new( to.clone(), - HydroNode::Persist(Box::new(HydroNode::Network { - from_location: LocationId::ExternalProcess(self.id), - from_key: Some(next_external_port_id), - to_location: to.id(), - to_key: None, - serialize_fn: None, - instantiate_fn: crate::ir::DebugInstantiate::Building(), - deserialize_fn: Some(deser_expr.into()), - input: Box::new(HydroNode::Source { - source: HydroSource::ExternalNetwork(), - location_kind: LocationId::ExternalProcess(self.id), + HydroNode::Persist { + inner: Box::new(HydroNode::Network { + from_location: LocationId::ExternalProcess(self.id), + from_key: Some(next_external_port_id), + to_location: to.id(), + to_key: None, + serialize_fn: None, + instantiate_fn: crate::ir::DebugInstantiate::Building(), + deserialize_fn: Some(deser_expr.into()), + input: Box::new(HydroNode::Source { + source: HydroSource::ExternalNetwork(), + location_kind: LocationId::ExternalProcess(self.id), + metadata: self.new_node_metadata::(), + }), + metadata: to.new_node_metadata::(), }), - })), + metadata: to.new_node_metadata::(), + }, ), ) } @@ -146,19 +151,24 @@ impl<'a, P> ExternalProcess<'a, P> { }, Stream::new( to.clone(), - HydroNode::Persist(Box::new(HydroNode::Network { - from_location: LocationId::ExternalProcess(self.id), - from_key: Some(next_external_port_id), - to_location: to.id(), - to_key: None, - serialize_fn: None, - instantiate_fn: crate::ir::DebugInstantiate::Building(), - deserialize_fn: Some(crate::stream::deserialize_bincode::(None).into()), - input: Box::new(HydroNode::Source { - source: HydroSource::ExternalNetwork(), - location_kind: LocationId::ExternalProcess(self.id), + HydroNode::Persist { + inner: Box::new(HydroNode::Network { + from_location: LocationId::ExternalProcess(self.id), + from_key: Some(next_external_port_id), + to_location: to.id(), + to_key: None, + serialize_fn: None, + instantiate_fn: crate::ir::DebugInstantiate::Building(), + deserialize_fn: Some(crate::stream::deserialize_bincode::(None).into()), + input: Box::new(HydroNode::Source { + source: HydroSource::ExternalNetwork(), + location_kind: LocationId::ExternalProcess(self.id), + metadata: self.new_node_metadata::(), + }), + metadata: to.new_node_metadata::(), }), - })), + metadata: to.new_node_metadata::(), + }, ), ) } diff --git a/hydro_lang/src/location/mod.rs b/hydro_lang/src/location/mod.rs index 8b91d1d81e1..f8902476ed3 100644 --- a/hydro_lang/src/location/mod.rs +++ b/hydro_lang/src/location/mod.rs @@ -9,7 +9,7 @@ use stageleft::{q, QuotedWithContext}; use super::builder::FlowState; use crate::cycle::{CycleCollection, ForwardRef, ForwardRefMarker}; -use crate::ir::{HydroNode, HydroSource}; +use crate::ir::{DebugType, HydroNode, HydroNodeMetadata, HydroSource}; use crate::{Singleton, Stream, Unbounded}; pub mod external_process; @@ -27,7 +27,7 @@ pub use can_send::CanSend; pub mod tick; pub use tick::{NoTick, Tick, Timestamped}; -#[derive(PartialEq, Eq, Clone, Debug)] +#[derive(PartialEq, Eq, Clone, Debug, Hash)] pub enum LocationId { Process(usize), Cluster(usize), @@ -82,16 +82,33 @@ pub trait Location<'a>: Clone { } } + fn next_node_id(&self) -> usize { + let next_id = self.flow_state().borrow_mut().next_node_id; + self.flow_state().borrow_mut().next_node_id += 1; + next_id + } + + fn new_node_metadata(&self) -> HydroNodeMetadata { + HydroNodeMetadata { + location_kind: self.id(), + output_type: Some(DebugType(stageleft::quote_type::())), + } + } + fn spin(&self) -> Stream<(), Self, Unbounded> where Self: Sized + NoTick, { Stream::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Spin(), - location_kind: self.id(), - })), + HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Spin(), + location_kind: self.id(), + metadata: self.new_node_metadata::<()>(), + }), + metadata: self.new_node_metadata::<()>(), + }, ) } @@ -106,10 +123,14 @@ pub trait Location<'a>: Clone { Stream::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Stream(e.into()), - location_kind: self.id(), - })), + HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Stream(e.into()), + location_kind: self.id(), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }, ) } @@ -126,10 +147,14 @@ pub trait Location<'a>: Clone { Stream::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Iter(e.into()), - location_kind: self.id(), - })), + HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Iter(e.into()), + location_kind: self.id(), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }, ) } @@ -151,10 +176,17 @@ pub trait Location<'a>: Clone { // so that it grows every tick Singleton::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Iter(e.into()), - location_kind: self.id(), - })))), + HydroNode::Persist { + inner: Box::new(HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Iter(e.into()), + location_kind: self.id(), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }, ) } diff --git a/hydro_lang/src/location/tick.rs b/hydro_lang/src/location/tick.rs index 2841db80646..a5f6f7a90c7 100644 --- a/hydro_lang/src/location/tick.rs +++ b/hydro_lang/src/location/tick.rs @@ -10,7 +10,7 @@ use crate::cycle::{ CycleCollection, CycleCollectionWithInitial, DeferTick, ForwardRef, ForwardRefMarker, TickCycle, TickCycleMarker, }; -use crate::ir::{HydroNode, HydroSource}; +use crate::ir::{HydroNode, HydroNodeMetadata, HydroSource}; use crate::{Bounded, Optional, Singleton, Stream}; #[sealed] @@ -82,6 +82,14 @@ impl<'a, L: Location<'a>> Location<'a> for Tick { fn is_top_level() -> bool { false } + + fn next_node_id(&self) -> usize { + self.l.next_node_id() + } + + fn new_node_metadata(&self) -> HydroNodeMetadata { + self.l.new_node_metadata::() + } } impl<'a, L: Location<'a>> Tick { @@ -138,6 +146,7 @@ impl<'a, L: Location<'a>> Tick { HydroNode::Source { source: HydroSource::Iter(e.into()), location_kind: self.l.id(), + metadata: self.new_node_metadata::(), }, ) } diff --git a/hydro_lang/src/optional.rs b/hydro_lang/src/optional.rs index dbfa591e0b3..b75ae8843f5 100644 --- a/hydro_lang/src/optional.rs +++ b/hydro_lang/src/optional.rs @@ -54,10 +54,11 @@ impl<'a, T, L: Location<'a>> CycleCollection<'a, TickCycleMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Optional::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -92,10 +93,11 @@ impl<'a, T, L: Location<'a>> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Optional::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -130,11 +132,15 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: L) -> Self { let location_id = location.id(); Optional::new( - location, - HydroNode::Persist(Box::new(HydroNode::CycleSource { - ident, - location_kind: location_id, - })), + location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CycleSource { + ident, + location_kind: location_id, + metadata: location.new_node_metadata::(), + }), + metadata: location.new_node_metadata::(), + }, ) } } @@ -155,7 +161,10 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleComplete<'a, ForwardRefMarker> for .push(HydroLeaf::CycleSink { ident, location_kind: self.location_kind(), - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), }); } } @@ -178,14 +187,16 @@ impl<'a, T: Clone, L: Location<'a>, B> Clone for Optional { let orig_ir_node = self.ir_node.replace(HydroNode::Placeholder); *self.ir_node.borrow_mut() = HydroNode::Tee { inner: TeeNode(Rc::new(RefCell::new(orig_ir_node))), + metadata: self.location.new_node_metadata::(), }; } - if let HydroNode::Tee { inner } = self.ir_node.borrow().deref() { + if let HydroNode::Tee { inner, metadata } = self.ir_node.borrow().deref() { Optional { location: self.location.clone(), ir_node: HydroNode::Tee { inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), } .into(), _phantom: PhantomData, @@ -218,10 +229,11 @@ impl<'a, T, L: Location<'a>, B> Optional { pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F, L>) -> Optional { let f = f.splice_fn1_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::Map { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -232,10 +244,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -246,10 +259,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -274,10 +288,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Optional { let f = f.splice_fn1_borrow_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::Filter { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -288,10 +303,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Optional { let f = f.splice_fn1_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::FilterMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -301,19 +317,30 @@ impl<'a, T, L: Location<'a>, B> Optional { if L::is_top_level() { Optional::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Chain( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(other.ir_node.into_inner()))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Chain { + first: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + second: Box::new(HydroNode::Unpersist { + inner: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }, ) } else { Optional::new( - self.location, - HydroNode::Chain( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Chain { + first: Box::new(self.ir_node.into_inner()), + second: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -327,19 +354,30 @@ impl<'a, T, L: Location<'a>, B> Optional { if L::is_top_level() { Optional::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::CrossSingleton( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(other.ir_node.into_inner()))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CrossSingleton { + left: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + right: Box::new(HydroNode::Unpersist { + inner: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::<(T, O)>(), + }), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } else { Optional::new( - self.location, - HydroNode::CrossSingleton( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::CrossSingleton { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } } @@ -349,19 +387,30 @@ impl<'a, T, L: Location<'a>, B> Optional { if L::is_top_level() { Singleton::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Chain( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(other.ir_node.into_inner()))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Chain { + first: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + second: Box::new(HydroNode::Unpersist { + inner: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }, ) } else { Singleton::new( - self.location, - HydroNode::Chain( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Chain { + first: Box::new(self.ir_node.into_inner()), + second: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -371,13 +420,23 @@ impl<'a, T, L: Location<'a>, B> Optional { T: Clone, { let none: syn::Expr = parse_quote!([::std::option::Option::None]); - let core_ir = HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Iter(none.into()), - location_kind: self.location.id().root().clone(), - })); + let core_ir = HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Iter(none.into()), + location_kind: self.location.id().root().clone(), + metadata: self.location.new_node_metadata::>(), + }), + metadata: self.location.new_node_metadata::>(), + }; let none_singleton = if L::is_top_level() { - Singleton::new(self.location.clone(), HydroNode::Persist(Box::new(core_ir))) + Singleton::new( + self.location.clone(), + HydroNode::Persist { + inner: Box::new(core_ir), + metadata: self.location.new_node_metadata::>(), + }, + ) } else { Singleton::new(self.location.clone(), core_ir) }; @@ -427,8 +486,11 @@ impl<'a, T, L: Location<'a> + NoTick, B> Optional, B> { /// arbitrary point in time. pub unsafe fn latest_tick(self) -> Optional, Bounded> { Optional::new( - self.location.tick, - HydroNode::Unpersist(Box::new(self.ir_node.into_inner())), + self.location.clone().tick, + HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -501,39 +563,54 @@ impl<'a, T, L: Location<'a>> Optional, Bounded> { pub fn all_ticks(self) -> Stream, Unbounded> { Stream::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn latest(self) -> Optional, Unbounded> { Optional::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn defer_tick(self) -> Optional, Bounded> { Optional::new( - self.location, - HydroNode::DeferTick(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::DeferTick { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn persist(self) -> Stream, Bounded> { Stream::new( - self.location, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn delta(self) -> Optional, Bounded> { Optional::new( - self.location, - HydroNode::Delta(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Delta { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } diff --git a/hydro_lang/src/rewrites/persist_pullup.rs b/hydro_lang/src/rewrites/persist_pullup.rs index 49a0e9c4418..a4429d7a067 100644 --- a/hydro_lang/src/rewrites/persist_pullup.rs +++ b/hydro_lang/src/rewrites/persist_pullup.rs @@ -9,18 +9,23 @@ fn persist_pullup_node( ) { *node = match_box::match_box! { match std::mem::replace(node, HydroNode::Placeholder) { - HydroNode::Unpersist(mb!(* HydroNode::Persist(mb!(* behind_persist)))) => behind_persist, + HydroNode::Unpersist { inner: mb!(* HydroNode::Persist { inner: mb!(* behind_persist), .. }), .. } => behind_persist, - HydroNode::Delta(mb!(* HydroNode::Persist(mb!(* behind_persist)))) => behind_persist, + HydroNode::Delta { inner: mb!(* HydroNode::Persist { inner: mb!(* behind_persist), .. }), .. } => behind_persist, - HydroNode::Tee { inner } => { + // TODO: Figure out if persist needs to copy its metadata or can just use original metadata here. If it can just use original, figure out where that is + HydroNode::Tee { inner, metadata } => { if persist_pulled_tees.contains(&(inner.0.as_ref() as *const RefCell)) { - HydroNode::Persist(Box::new(HydroNode::Tee { - inner: TeeNode(inner.0.clone()), - })) - } else if matches!(*inner.0.borrow(), HydroNode::Persist(_)) { + HydroNode::Persist { + inner: Box::new(HydroNode::Tee { + inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + } + } else if matches!(*inner.0.borrow(), HydroNode::Persist { .. }) { persist_pulled_tees.insert(inner.0.as_ref() as *const RefCell); - if let HydroNode::Persist(behind_persist) = + if let HydroNode::Persist { inner: behind_persist, .. } = inner.0.replace(HydroNode::Placeholder) { *inner.0.borrow_mut() = *behind_persist; @@ -28,45 +33,69 @@ fn persist_pullup_node( unreachable!() } - HydroNode::Persist(Box::new(HydroNode::Tee { - inner: TeeNode(inner.0.clone()), - })) + HydroNode::Persist { + inner: Box::new(HydroNode::Tee { + inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + } } else { - HydroNode::Tee { inner } + HydroNode::Tee { inner, metadata } } } HydroNode::Map { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::Map { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::Map { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + }, HydroNode::FilterMap { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::FilterMap { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::FilterMap { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, HydroNode::FlatMap { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::FlatMap { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::FlatMap { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, HydroNode::Filter { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::Filter { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::Filter { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, HydroNode::Network { from_location, @@ -76,44 +105,73 @@ fn persist_pullup_node( serialize_fn, instantiate_fn, deserialize_fn, - input: mb!(* HydroNode::Persist(behind_persist)), - .. - } => HydroNode::Persist(Box::new(HydroNode::Network { - from_location, - from_key, - to_location, - to_key, - serialize_fn, - instantiate_fn, - deserialize_fn, - input: behind_persist, - })), - - HydroNode::Chain(mb!(* HydroNode::Persist(left)), mb!(* HydroNode::Persist(right))) => { - HydroNode::Persist(Box::new(HydroNode::Chain(left, right))) - } - - HydroNode::CrossProduct(mb!(* HydroNode::Persist(left)), mb!(* HydroNode::Persist(right))) => { - HydroNode::Persist(Box::new(HydroNode::Delta(Box::new( - HydroNode::CrossProduct( - Box::new(HydroNode::Persist(left)), - Box::new(HydroNode::Persist(right)), - ), - )))) - } - - HydroNode::Join(mb!(* HydroNode::Persist(left)), mb!(* HydroNode::Persist(right))) => { - HydroNode::Persist(Box::new(HydroNode::Delta(Box::new(HydroNode::Join( - Box::new(HydroNode::Persist(left)), - Box::new(HydroNode::Persist(right)), - ))))) - } - - HydroNode::Unique(mb!(* HydroNode::Persist(inner))) => { - HydroNode::Persist(Box::new(HydroNode::Delta(Box::new(HydroNode::Unique( - Box::new(HydroNode::Persist(inner)), - ))))) - } + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::Network { + from_location, + from_key, + to_location, + to_key, + serialize_fn, + instantiate_fn, + deserialize_fn, + input: behind_persist, + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }, + + HydroNode::Chain { + first: mb!(* HydroNode::Persist { inner: first, metadata: persist_metadata }), + second: mb!(* HydroNode::Persist { inner: second, .. }), + metadata + } => HydroNode::Persist { + inner: Box::new(HydroNode::Chain { first, second, metadata }), + metadata: persist_metadata + }, + + HydroNode::CrossProduct { + left: mb!(* HydroNode::Persist { inner: left, metadata: left_metadata }), + right: mb!(* HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata + } => HydroNode::Persist { + inner: Box::new(HydroNode::Delta { + inner: Box::new(HydroNode::CrossProduct { + left: Box::new(HydroNode::Persist { inner: left, metadata: left_metadata }), + right: Box::new(HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + }, + HydroNode::Join { + left: mb!(* HydroNode::Persist { inner: left, metadata: left_metadata }), + right: mb!(* HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata + } => HydroNode::Persist { + inner: Box::new(HydroNode::Delta { + inner: Box::new(HydroNode::Join { + left: Box::new(HydroNode::Persist { inner: left, metadata: left_metadata }), + right: Box::new(HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + }, + + HydroNode::Unique { input: mb!(* HydroNode::Persist {inner, metadata: persist_metadata } ), metadata } => HydroNode::Persist { + inner: Box::new(HydroNode::Delta { + inner: Box::new(HydroNode::Unique { + input: Box::new(HydroNode::Persist { inner, metadata: persist_metadata }), + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, node => node, } diff --git a/hydro_lang/src/rewrites/profiler.rs b/hydro_lang/src/rewrites/profiler.rs index 8e2b01a0839..197dcc73f32 100644 --- a/hydro_lang/src/rewrites/profiler.rs +++ b/hydro_lang/src/rewrites/profiler.rs @@ -30,6 +30,7 @@ fn add_profiling_node<'a>( seen_tees, ); let orig_node = std::mem::replace(node, HydroNode::Placeholder); + let new_metadata = orig_node.metadata().clone(); *node = HydroNode::Inspect { f: quoted_any_fn(q!({ // Put counters on queue @@ -45,6 +46,7 @@ fn add_profiling_node<'a>( .splice_untyped() .into(), input: Box::new(orig_node), + metadata: new_metadata, } } diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap index dcb22d2ce6c..0838490b137 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap @@ -5,8 +5,8 @@ expression: optimized.ir() [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Persist( - Map { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { inner: : Source { @@ -16,15 +16,47 @@ expression: optimized.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, }, ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Persist( - Map { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { inner: : Source { @@ -34,9 +66,41 @@ expression: optimized.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap index d960d065ddf..12b1fb81402 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap @@ -5,54 +5,194 @@ expression: built.ir() [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Persist( - Map { + input: Unpersist { + inner: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { - inner: : Persist( - Unpersist( - Persist( - Source { + inner: : Persist { + inner: Unpersist { + inner: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, }, ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Persist( - Map { + input: Unpersist { + inner: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { - inner: : Persist( - Unpersist( - Persist( - Source { + inner: : Persist { + inner: Unpersist { + inner: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap index 667076ada4c..88e70589f0e 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap @@ -14,6 +14,22 @@ expression: optimized.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, }, diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap index bd39e1a4076..47de226aada 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap @@ -5,20 +5,52 @@ expression: built.ir() [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Map { + input: Unpersist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), - input: Persist( - Source { + input: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, - ), + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap index 0416e6a9a06..2509e54f6b8 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap @@ -18,9 +18,41 @@ expression: "&pushed_down.ir()" location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap index 6e0a06f8ade..f5d79eeaa2d 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap @@ -5,20 +5,52 @@ expression: "&built.ir()" [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: profiler :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Map { + input: Unpersist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: profiler :: tests :: * ; | v | v + 1 }), - input: Persist( - Source { + input: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: profiler :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, - ), + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap index 4c74a724fe9..dfcc8af6253 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap @@ -17,7 +17,31 @@ expression: built.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + std :: string :: String, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), }, }, }, diff --git a/hydro_lang/src/singleton.rs b/hydro_lang/src/singleton.rs index 5994670ce95..c5e8d00dea7 100644 --- a/hydro_lang/src/singleton.rs +++ b/hydro_lang/src/singleton.rs @@ -56,14 +56,16 @@ impl<'a, T, L: Location<'a>> CycleCollectionWithInitial<'a, TickCycleMarker> fn create_source(ident: syn::Ident, initial: Self, location: Tick) -> Self { let location_id = location.id(); Singleton::new( - location, - HydroNode::Chain( - Box::new(HydroNode::CycleSource { + location.clone(), + HydroNode::Chain { + first: Box::new(HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }), - initial.ir_node.into_inner().into(), - ), + second: initial.ir_node.into_inner().into(), + metadata: location.new_node_metadata::(), + }, ) } } @@ -97,10 +99,11 @@ impl<'a, T, L: Location<'a>> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Singleton::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -137,11 +140,15 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: L) -> Self { let location_id = location.id(); Singleton::new( - location, - HydroNode::Persist(Box::new(HydroNode::CycleSource { - ident, - location_kind: location_id, - })), + location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CycleSource { + ident, + location_kind: location_id, + metadata: location.new_node_metadata::(), + }), + metadata: location.new_node_metadata::(), + }, ) } } @@ -164,7 +171,10 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleComplete<'a, ForwardRefMarker> .push(HydroLeaf::CycleSink { ident, location_kind: self.location_kind(), - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), }); } } @@ -175,14 +185,16 @@ impl<'a, T: Clone, L: Location<'a>, B> Clone for Singleton { let orig_ir_node = self.ir_node.replace(HydroNode::Placeholder); *self.ir_node.borrow_mut() = HydroNode::Tee { inner: TeeNode(Rc::new(RefCell::new(orig_ir_node))), + metadata: self.location.new_node_metadata::(), }; } - if let HydroNode::Tee { inner } = self.ir_node.borrow().deref() { + if let HydroNode::Tee { inner, metadata } = self.ir_node.borrow().deref() { Singleton { location: self.location.clone(), ir_node: HydroNode::Tee { inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), } .into(), _phantom: PhantomData, @@ -197,10 +209,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F, L>) -> Singleton { let f = f.splice_fn1_ctx(&self.location).into(); Singleton::new( - self.location, + self.location.clone(), HydroNode::Map { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -211,10 +224,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -225,10 +239,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -239,10 +254,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Optional { let f = f.splice_fn1_borrow_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::Filter { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -253,10 +269,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Optional { let f = f.splice_fn1_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::FilterMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -268,20 +285,42 @@ impl<'a, T, L: Location<'a>, B> Singleton { check_matching_location(&self.location, &Self::other_location(&other)); if L::is_top_level() { + let left_ir_node = self.ir_node.into_inner(); + let left_ir_node_metadata = left_ir_node.metadata().clone(); + let right_ir_node = Self::other_ir_node(other); + let right_ir_node_metadata = right_ir_node.metadata().clone(); + Self::make( - self.location, - HydroNode::Persist(Box::new(HydroNode::CrossSingleton( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(Self::other_ir_node(other)))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CrossSingleton { + left: Box::new(HydroNode::Unpersist { + inner: Box::new(left_ir_node), + metadata: left_ir_node_metadata, + }), + right: Box::new(HydroNode::Unpersist { + inner: Box::new(right_ir_node), + metadata: right_ir_node_metadata, + }), + metadata: self + .location + .new_node_metadata::<>::ElementType>(), + }), + metadata: self + .location + .new_node_metadata::<>::ElementType>(), + }, ) } else { Self::make( - self.location, - HydroNode::CrossSingleton( - Box::new(self.ir_node.into_inner()), - Box::new(Self::other_ir_node(other)), - ), + self.location.clone(), + HydroNode::CrossSingleton { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(Self::other_ir_node(other)), + metadata: self + .location + .new_node_metadata::<>::ElementType>(), + }, ) } } @@ -322,8 +361,11 @@ impl<'a, T, L: Location<'a> + NoTick, B> Singleton, B> { /// arbitrary point in time. pub unsafe fn latest_tick(self) -> Singleton, Bounded> { Singleton::new( - self.location.tick, - HydroNode::Unpersist(Box::new(self.ir_node.into_inner())), + self.location.clone().tick, + HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -396,39 +438,54 @@ impl<'a, T, L: Location<'a>> Singleton, Bounded> { pub fn all_ticks(self) -> Stream, Unbounded> { Stream::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn latest(self) -> Singleton, Unbounded> { Singleton::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn defer_tick(self) -> Singleton, Bounded> { Singleton::new( - self.location, - HydroNode::DeferTick(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::DeferTick { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn persist(self) -> Stream, Bounded> { Stream::new( - self.location, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn delta(self) -> Optional, Bounded> { Optional::new( - self.location, - HydroNode::Delta(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Delta { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -439,6 +496,7 @@ impl<'a, T, L: Location<'a>> Singleton, Bounded> { pub trait ZipResult<'a, Other> { type Out; + type ElementType; type Location; fn other_location(other: &Other) -> Self::Location; @@ -451,6 +509,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Singleton, B> { type Out = Singleton<(T, U), Timestamped, B>; + type ElementType = (T, U); type Location = Timestamped; fn other_location(other: &Singleton, B>) -> Timestamped { @@ -470,6 +529,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Optional, B> { type Out = Optional<(T, U), Timestamped, B>; + type ElementType = (T, U); type Location = Timestamped; fn other_location(other: &Optional, B>) -> Timestamped { @@ -489,6 +549,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Singleton, B> for Singleton, B> { type Out = Singleton<(T, U), Tick, B>; + type ElementType = (T, U); type Location = Tick; fn other_location(other: &Singleton, B>) -> Tick { @@ -508,6 +569,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Optional, B>> for Singleton, B> { type Out = Optional<(T, U), Tick, B>; + type ElementType = (T, U); type Location = Tick; fn other_location(other: &Optional, B>) -> Tick { diff --git a/hydro_lang/src/stream.rs b/hydro_lang/src/stream.rs index 6ea29059329..0d902f3cf66 100644 --- a/hydro_lang/src/stream.rs +++ b/hydro_lang/src/stream.rs @@ -113,10 +113,11 @@ impl<'a, T, L: Location<'a>, Order> CycleCollection<'a, TickCycleMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Stream::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -152,12 +153,17 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> CycleCollection<'a, ForwardRefMa fn create_source(ident: syn::Ident, location: L) -> Self { let location_id = location.id(); + Stream::new( - location, - HydroNode::Persist(Box::new(HydroNode::CycleSource { - ident, - location_kind: location_id, - })), + location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CycleSource { + ident, + location_kind: location_id, + metadata: location.new_node_metadata::(), + }), + metadata: location.new_node_metadata::(), + }, ) } } @@ -171,6 +177,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> CycleComplete<'a, ForwardRefMark expected_location, "locations do not match" ); + let metadata = self.location.new_node_metadata::(); self.location .flow_state() .borrow_mut() @@ -180,7 +187,10 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> CycleComplete<'a, ForwardRefMark .push(HydroLeaf::CycleSink { ident, location_kind: self.location_kind(), - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata, + }), }); } } @@ -201,14 +211,16 @@ impl<'a, T: Clone, L: Location<'a>, B, Order> Clone for Stream { let orig_ir_node = self.ir_node.replace(HydroNode::Placeholder); *self.ir_node.borrow_mut() = HydroNode::Tee { inner: TeeNode(Rc::new(RefCell::new(orig_ir_node))), + metadata: self.location.new_node_metadata::(), }; } - if let HydroNode::Tee { inner } = self.ir_node.borrow().deref() { + if let HydroNode::Tee { inner, metadata } = self.ir_node.borrow().deref() { Stream { location: self.location.clone(), ir_node: HydroNode::Tee { inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), } .into(), _phantom: PhantomData, @@ -244,10 +256,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::Map { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -280,10 +293,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -318,10 +332,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -411,10 +426,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_borrow_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::Filter { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -441,10 +457,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FilterMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -483,11 +500,12 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::CrossSingleton( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::CrossSingleton { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } @@ -512,11 +530,12 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::CrossProduct( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::CrossProduct { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } @@ -527,8 +546,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { T: Eq + Hash, { Stream::new( - self.location, - HydroNode::Unique(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Unique { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -543,11 +565,12 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::Difference( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Difference { + pos: Box::new(self.ir_node.into_inner()), + neg: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -562,18 +585,26 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { if L::is_top_level() { Stream::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Inspect { - f, - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - })), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Inspect { + f, + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }, ) } else { Stream::new( - self.location, + self.location.clone(), HydroNode::Inspect { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -655,13 +686,17 @@ where init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { // top-level (possibly unbounded) singletons are represented as // a stream which produces all values from all ticks every tick, // so Unpersist will always give the lastest aggregation - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Singleton::new(self.location, core) @@ -699,10 +734,14 @@ where let mut core = HydroNode::Reduce { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Optional::new(self.location, core) @@ -772,10 +811,14 @@ where let mut core = HydroNode::Reduce { f: wrapped.into(), input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Optional::new(self.location, core) @@ -834,18 +877,26 @@ impl<'a, T, L: Location<'a>, B> Stream { pub fn enumerate(self) -> Stream<(usize, T), L, B, TotalOrder> { if L::is_top_level() { Stream::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Enumerate { - is_static: true, - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - })), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Enumerate { + is_static: true, + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::<(usize, T)>(), + }), + metadata: self.location.new_node_metadata::<(usize, T)>(), + }, ) } else { Stream::new( - self.location, + self.location.clone(), HydroNode::Enumerate { is_static: false, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(usize, T)>(), }, ) } @@ -935,13 +986,17 @@ impl<'a, T, L: Location<'a>, B> Stream { init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { // top-level (possibly unbounded) singletons are represented as // a stream which produces all values from all ticks every tick, // so Unpersist will always give the lastest aggregation - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Singleton::new(self.location, core) @@ -980,10 +1035,14 @@ impl<'a, T, L: Location<'a>, B> Stream { let mut core = HydroNode::Reduce { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Optional::new(self.location, core) @@ -1061,8 +1120,11 @@ impl<'a, T, L: Location<'a>, Order> Stream { T: Ord, { Stream::new( - self.location, - HydroNode::Sort(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Sort { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -1103,11 +1165,12 @@ impl<'a, T, L: Location<'a>, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::Chain( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Chain { + first: Box::new(self.ir_node.into_inner()), + second: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -1122,11 +1185,12 @@ impl<'a, K, V1, L: Location<'a>, B, Order> Stream<(K, V1), L, B, Order> { check_matching_location(&self.location, &n.location); Stream::new( - self.location, - HydroNode::Join( - Box::new(self.ir_node.into_inner()), - Box::new(n.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Join { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(n.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, (V1, V2))>(), + }, ) } @@ -1141,11 +1205,12 @@ impl<'a, K, V1, L: Location<'a>, B, Order> Stream<(K, V1), L, B, Order> { check_matching_location(&self.location, &n.location); Stream::new( - self.location, - HydroNode::AntiJoin( - Box::new(self.ir_node.into_inner()), - Box::new(n.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::AntiJoin { + pos: Box::new(self.ir_node.into_inner()), + neg: Box::new(n.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, V1)>(), + }, ) } } @@ -1188,11 +1253,12 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>> Stream<(K, V), Tick, Bounded> { let comb = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FoldKeyed { init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, A)>(), }, ) } @@ -1231,10 +1297,11 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>> Stream<(K, V), Tick, Bounded> { let f = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::ReduceKeyed { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, V)>(), }, ) } @@ -1277,11 +1344,12 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>, Order> Stream<(K, V), Tick, Bounde let comb = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FoldKeyed { init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, A)>(), }, ) } @@ -1325,10 +1393,11 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>, Order> Stream<(K, V), Tick, Bounde let f = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::ReduceKeyed { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, V)>(), }, ) } @@ -1343,8 +1412,11 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream, B, Ord /// The batch boundaries are non-deterministic and may change across executions. pub unsafe fn tick_batch(self) -> Stream, Bounded, Order> { Stream::new( - self.location.tick, - HydroNode::Unpersist(Box::new(self.ir_node.into_inner())), + self.location.clone().tick, + HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -1442,6 +1514,7 @@ impl<'a, T, L: Location<'a> + NoTick + NoTimestamp, B, Order> Stream + NoTick, B, Order> Stream { pub fn for_each(self, f: impl IntoQuotedMut<'a, F, L>) { let f = f.splice_fn1_ctx(&self.location).into(); + let metadata = self.location.new_node_metadata::(); self.location .flow_state() .borrow_mut() @@ -1449,7 +1522,10 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { .as_mut() .expect(FLOW_USED_MESSAGE) .push(HydroLeaf::ForEach { - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata, + }), f, }); } @@ -1477,7 +1553,10 @@ impl<'a, T, L: Location<'a>, Order> Stream, Bounded, Order> { Timestamped { tick: self.location.clone(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -1486,22 +1565,31 @@ impl<'a, T, L: Location<'a>, Order> Stream, Bounded, Order> { T: Clone, { Stream::new( - self.location, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn defer_tick(self) -> Stream, Bounded, Order> { Stream::new( - self.location, - HydroNode::DeferTick(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::DeferTick { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn delta(self) -> Stream, Bounded, Order> { Stream::new( - self.location, - HydroNode::Delta(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Delta { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -1572,6 +1660,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { instantiate_fn: DebugInstantiate::Building(), deserialize_fn: deserialize_pipeline.map(|e| e.into()), input: Box::new(self.ir_node.into_inner()), + metadata: other.new_node_metadata::(), }, ) } @@ -1587,6 +1676,8 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { { let serialize_pipeline = Some(serialize_bincode::(L::is_demux())); + let metadata = other.new_node_metadata::(); + let mut flow_state_borrow = self.location.flow_state().borrow_mut(); let external_key = flow_state_borrow.next_external_out; @@ -1607,6 +1698,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { instantiate_fn: DebugInstantiate::Building(), deserialize_fn: None, input: Box::new(self.ir_node.into_inner()), + metadata, }), }); @@ -1643,6 +1735,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { Some(expr.into()) }, input: Box::new(self.ir_node.into_inner()), + metadata: other.new_node_metadata::(), }, ) } @@ -1651,6 +1744,8 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { where L::Root: CanSend<'a, ExternalProcess<'a, L2>, In = T, Out = Bytes>, { + let metadata = other.new_node_metadata::(); + let mut flow_state_borrow = self.location.flow_state().borrow_mut(); let external_key = flow_state_borrow.next_external_out; flow_state_borrow.next_external_out += 1; @@ -1670,6 +1765,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { instantiate_fn: DebugInstantiate::Building(), deserialize_fn: None, input: Box::new(self.ir_node.into_inner()), + metadata, }), }); diff --git a/hydro_std/Cargo.toml b/hydro_std/Cargo.toml index fdb4eaf80ff..8436273ea9c 100644 --- a/hydro_std/Cargo.toml +++ b/hydro_std/Cargo.toml @@ -15,14 +15,14 @@ workspace = true path = "src/lib.rs" [dependencies] -hydro_lang = { path = "../hydro_lang", version = "^0.11.0", default-features = false } +hydro_lang = { path = "../hydro_lang", version = "^0.11.0" } stageleft = { path = "../stageleft", version = "^0.6.0" } [build-dependencies] stageleft_tool = { path = "../stageleft_tool", version = "^0.5.0" } [dev-dependencies] -hydro_lang = { path = "../hydro_lang", version = "^0.11.0" } +hydro_lang = { path = "../hydro_lang", version = "^0.11.0", features = ["deploy"] } insta = "1.39" hydro_deploy = { path = "../hydro_deploy/core", version = "^0.11.0" } async-ssh2-lite = { version = "0.5.0", features = ["vendored-openssl"] } diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap index e284ea81961..9e805cb763b 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap @@ -7,11 +7,11 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (u64 , u64) , () > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | (inside , total) | { println ! ("pi: {} ({} trials)" , 4.0 * inside as f64 / total as f64 , total) ; } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((u64 , u64) , ()) , (u64 , u64) > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Reduce { + input: CrossSingleton { + left: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u64 , u64) , (u64 , u64) , () > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | (inside , total) , (inside_batch , total_batch) | { * inside += inside_batch ; * total += total_batch ; } }), - input: Persist( - Map { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: compute_pi :: Worker > , (u64 , u64)) , (u64 , u64) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -45,17 +45,97 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (f64 , f64), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + bool, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (u64 , u64), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Source { source: Stream( @@ -64,9 +144,41 @@ expression: built.ir() location_kind: Process( 1, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + ((u64 , u64) , ()), + ), }, - ), + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, }, }, ] diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap index 74c0e31cdf6..36d54d0931e 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap @@ -30,7 +30,31 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + i32, + ), }, }, }, diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap index 64648d904d4..22ed07d5321 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap @@ -7,8 +7,8 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: string :: String , i32) , () > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | (string , count) | println ! ("{}: {}" , string , count) }), input: ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | total , count | * total += count }), - input: Persist( - Map { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , (std :: string :: String , i32)) , (std :: string :: String , i32) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -62,17 +62,113 @@ expression: built.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + & str, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + std :: string :: String, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (usize , std :: string :: String), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , std :: string :: String), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + std :: string :: String, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (std :: string :: String , ()), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), }, - ), + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, }, }, ] diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap index eb4f676ebc9..c3b6cbc7bbf 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap @@ -12,6 +12,14 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + & str, + ), + }, }, }, ForEach { @@ -23,6 +31,14 @@ expression: built.ir() location_kind: Cluster( 1, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + & str, + ), + }, }, }, CycleSink { @@ -35,60 +51,148 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (__hydro_lang_cluster_self_id_0) ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , proposer_id : CLUSTER_SELF_ID__free , }) { received_max_ballot . num + 1 } else { ballot_num } } }), - input: CrossSingleton( - Tee { - inner: : Chain( - Reduce { + input: CrossSingleton { + left: Tee { + inner: : Chain { + first: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Chain( - Chain( - CycleSource { + input: Persist { + inner: Chain { + first: Chain { + first: CycleSource { ident: Ident { sym: cycle_1, }, location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - CycleSource { + second: CycleSource { ident: Ident { sym: cycle_0, }, location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, - ), - CycleSource { + }, + second: CycleSource { ident: Ident { sym: cycle_2, }, location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, ), - ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e__free] }, ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, ), - ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Tee { - inner: : Chain( - CycleSource { + right: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_4, }, @@ -98,22 +202,86 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e__free] }, ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , u32), ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, ), }, - ), + }, }, CycleSink { ident: Ident { @@ -145,19 +313,43 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >] > (__hydro_lang_cluster_ids_0) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (__hydro_lang_cluster_self_id_0) ; move | num | Ballot { num , proposer_id : CLUSTER_SELF_ID__free } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { inner: : CycleSource { @@ -170,12 +362,52 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Source { source: Stream( @@ -184,13 +416,77 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, }, CycleSink { @@ -203,17 +499,17 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Difference( - FilterMap { + input: DeferTick { + input: Difference { + pos: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)) , core :: option :: Option < hydro_test :: cluster :: paxos :: Ballot > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , usize) > ({ use hydro_std :: __staged :: quorum :: * ; move | | (0 , 0) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { - inner: : Chain( - CycleSource { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_5, }, @@ -223,8 +519,16 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + second: Tee { inner: : Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Map { @@ -247,9 +551,9 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((ballot , max_ballot) , log) | (ballot . proposer_id , (ballot , if ballot == max_ballot { Ok (log) } else { Err (max_ballot) })) }), - input: CrossSingleton( - CrossSingleton( - Tee { + input: CrossSingleton { + left: CrossSingleton { + left: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -274,31 +578,71 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | println ! ("Proposer leader expired, sending P1a") }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - FilterMap { + input: CrossSingleton { + left: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , core :: option :: Option < () > > ({ use hydro_lang :: __staged :: stream :: * ; let duration__free = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout__free = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout__free) } ; move | latest_received | { if let Some (latest_received) = latest_received { if Instant :: now () . duration_since (latest_received) > duration__free { Some (()) } else { None } } else { Some (()) } } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < tokio :: time :: Instant > > ({ use hydro_lang :: __staged :: stream :: * ; | | None }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | latest , _ | { * latest = Some (Instant :: now ()) ; } }), - input: Persist( - Tee { + input: Persist { + inner: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < tokio :: time :: Instant >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: optional :: * ; | c | * c == 0 }), @@ -307,13 +651,61 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Source { source: Stream( @@ -322,45 +714,219 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - Tee { - inner: : Chain( - Reduce { + right: Tee { + inner: : Chain { + first: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Inspect { + input: Persist { + inner: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e__free] }, ), location_kind: Cluster( 1, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), ), }, - ), - CycleSource { + }, + right: CycleSource { ident: Ident { sym: cycle_0, }, @@ -370,28 +936,167 @@ expression: built.ir() 1, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + ((hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - Tee { + neg: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)) , core :: option :: Option < hydro_test :: cluster :: paxos :: Ballot > > ({ use hydro_std :: __staged :: quorum :: * ; let max__free = 3usize ; move | (key , (success , error)) | if (success + error) >= max__free { Some (key) } else { None } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, }, CycleSink { ident: Ident { @@ -403,16 +1108,48 @@ expression: built.ir() 0, ), ), - input: DeferTick( - AntiJoin( - Tee { + input: DeferTick { + input: AntiJoin { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + neg: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -427,34 +1164,66 @@ expression: built.ir() input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), input: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | ((quorum_ballot , quorum_accepted) , my_ballot) | if quorum_ballot == my_ballot { Some (quorum_accepted) } else { None } }), - input: CrossSingleton( - Reduce { + input: CrossSingleton { + left: Reduce { f: { let key_fn = stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }) ; move | curr , new | { if key_fn (& new) > key_fn (& * curr) { * curr = new ; } } }, input: FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | | vec ! [] }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | logs , log | { logs . push (log) ; } }), - input: Persist( - FilterMap { + input: Persist { + inner: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (v) => Some ((key , v)) , Err (_) => None , } }), - input: AntiJoin( - AntiJoin( - Tee { + input: AntiJoin { + pos: AntiJoin { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - FilterMap { + neg: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)) , core :: option :: Option < hydro_test :: cluster :: paxos :: Ballot > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success < min__free { Some (key) } else { None } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - ), - CycleSource { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + neg: CycleSource { ident: Ident { sym: cycle_6, }, @@ -464,38 +1233,206 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >), ), }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (received_max_ballot , cur_ballot) | * received_max_ballot <= * cur_ballot }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, @@ -513,8 +1450,41 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, }, CycleSink { @@ -527,13 +1497,13 @@ expression: built.ir() 2, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: : Chain( - CycleSource { + input: CrossSingleton { + left: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_1, }, @@ -543,8 +1513,16 @@ expression: built.ir() 2, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; move | (key , value) | KvPayload { key , value : (CLUSTER_SELF_ID__free , value) } }), input: CycleSource { ident: Ident { @@ -553,11 +1531,43 @@ expression: built.ir() location_kind: Cluster( 2, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: stream :: * ; | c | * c == 0 }), @@ -569,8 +1579,8 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; | ballot : Ballot | ballot . proposer_id }), input: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Inspect { + input: Persist { + inner: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), @@ -594,54 +1604,270 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: bench_client :: Client >] > (__hydro_lang_cluster_ids_2) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: optional :: * ; | c | * c == 0 }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: DeferTick( - Tee { + input: DeferTick { + input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , hydro_test :: cluster :: paxos :: Ballot), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, - ), + }, }, CycleSink { ident: Ident { @@ -653,24 +1879,24 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , base_slot) | base_slot + num_payloads }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , usize) , (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((index , payload) , base_slot) | (base_slot + index , payload) }), - input: CrossSingleton( - Enumerate { + input: CrossSingleton { + left: Enumerate { is_static: false, input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -690,29 +1916,120 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; move | (payload , leader_id) | (leader_id , payload) }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 7, + Cluster( + 2, + ), + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), ), }, }, - Tee { - inner: : Chain( - Map { + right: Tee { + inner: : Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), input: Tee { inner: : Reduce { @@ -734,20 +2051,116 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > ({ use hydro_lang :: __staged :: optional :: * ; | v | v }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Chain( - CycleSource { + second: Chain { + first: CycleSource { ident: Ident { sym: cycle_7, }, @@ -757,31 +2170,151 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e__free] }, ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , usize), ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , usize), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, ), }, - ), + }, }, CycleSink { ident: Ident { @@ -793,9 +2326,9 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Difference( - Tee { + input: DeferTick { + input: Difference { + pos: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)) , core :: option :: Option < (usize , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { @@ -803,8 +2336,8 @@ expression: built.ir() init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , usize) > ({ use hydro_std :: __staged :: quorum :: * ; move | | (0 , 0) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { - inner: : Chain( - CycleSource { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_8, }, @@ -814,8 +2347,16 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + second: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -836,8 +2377,8 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , ((p2a . slot , p2a . ballot) , if p2a . ballot == max_ballot { Ok (()) } else { Err (max_ballot) })) }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -863,36 +2404,92 @@ expression: built.ir() input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , ()) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Chain( - Map { + input: CrossSingleton { + left: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , payload) , ballot) | ((slot , ballot) , Some (payload)) }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2a | ((p2a . slot , p2a . ballot) , p2a . value) }), - input: Chain( - FilterMap { + input: Chain { + first: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < usize >) , core :: option :: Option < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | (((slot , (count , entry)) , ballot) , checkpoint) | { if count > f__free { return None ; } else if let Some (checkpoint) = checkpoint { if slot <= checkpoint { return None ; } } Some (P2a { ballot , slot , value : entry . value , }) } }), - input: CrossSingleton( - CrossSingleton( - Tee { + input: CrossSingleton { + left: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), - Tee { - inner: : Chain( - Map { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + right: Tee { + inner: : Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < usize > > ({ use hydro_lang :: __staged :: optional :: * ; | v | Some (v) }), input: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), @@ -900,92 +2497,504 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (checkpoint , _log) | checkpoint }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( [:: std :: option :: Option :: None], ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < usize >), ), }, - ), + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , ballot) | P2a { ballot , slot , value : None } }), - input: CrossSingleton( - Difference( - FlatMap { + input: CrossSingleton { + left: Difference { + pos: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < usize >) , std :: ops :: Range < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (max_slot , checkpoint) | { if let Some (checkpoint) = checkpoint { (checkpoint + 1) .. max_slot } else { 0 .. max_slot } } }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , core :: option :: Option < usize >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Map { + neg: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, - ), - Tee { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), ), }, - ), - Map { + }, + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - Tee { + neg: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)) , core :: option :: Option < (usize , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; let max__free = 3usize ; move | (key , (success , error)) | if (success + error) >= max__free { Some (key) } else { None } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, }, CycleSink { ident: Ident { @@ -997,16 +3006,48 @@ expression: built.ir() 0, ), ), - input: DeferTick( - AntiJoin( - Tee { + input: DeferTick { + input: AntiJoin { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + neg: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1018,11 +3059,11 @@ expression: built.ir() 0, ), ), - input: DeferTick( - AntiJoin( - Tee { - inner: : Chain( - CycleSource { + input: DeferTick { + input: AntiJoin { + pos: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_10, }, @@ -1032,22 +3073,65 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - Tee { + second: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) , (usize , hydro_test :: cluster :: paxos :: Ballot) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , _) | key }), - input: Tee { - inner: : Map { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + neg: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) , (usize , hydro_test :: cluster :: paxos :: Ballot) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , _) | key }), + input: Tee { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) > ({ use crate :: __staged :: cluster :: paxos :: * ; | k | (k , ()) }), - input: Difference( - Tee { + input: Difference { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - CycleSource { + neg: CycleSource { ident: Ident { sym: cycle_9, }, @@ -1057,13 +3141,72 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1078,34 +3221,66 @@ expression: built.ir() input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (None , HashMap :: new ()) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { if prev_checkpoint . map (| prev | new_checkpoint > prev) . unwrap_or (true) { for slot in (prev_checkpoint . unwrap_or (0)) .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = Some (new_checkpoint) ; } } CheckpointOrP2a :: P2a (p2a) => { if prev_checkpoint . map (| prev | p2a . slot > prev) . unwrap_or (true) && log . get (& p2a . slot) . map (| prev_p2a : & LogValue < _ > | p2a . ballot > prev_p2a . ballot) . unwrap_or (true) { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } } }), - input: Persist( - Chain( - FilterMap { + input: Persist { + inner: Chain { + first: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), - input: Delta( - Reduce { + input: Delta { + inner: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , ()) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), - input: Persist( - Network { + input: Persist { + inner: Network { from_location: Cluster( 3, ), @@ -1130,13 +3305,61 @@ expression: built.ir() location_kind: Cluster( 3, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , usize), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | num_received | if num_received == f__free + 1 { Some (true) } else { None } }), @@ -1145,18 +3368,125 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + bool, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + ((hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , ()), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 2, + Cluster( + 1, + ), ), - ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, }, }, CycleSink { @@ -1172,8 +3502,41 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < ((usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, }, CycleSink { @@ -1186,16 +3549,16 @@ expression: built.ir() 3, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), - input: CrossSingleton( - Tee { - inner: : Sort( - Chain( - Map { + input: CrossSingleton { + left: Tee { + inner: : Sort { + input: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (slot , kv) | SequencedKv { seq : slot , kv } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), @@ -1221,21 +3584,96 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , _ballot) , (value , _)) | (slot , value) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , (meta , resp)) | (key , (meta , resp)) }), - input: Join( - Tee { + input: Join { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, }, - CycleSource { + second: CycleSource { ident: Ident { sym: cycle_1, }, @@ -1245,22 +3683,62 @@ expression: built.ir() 3, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, ), - ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Tee { + right: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < usize > , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | v | v }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | | None }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , () > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | filled_slot , (sorted_payload , highest_seq) | { let expected_next_slot = std :: cmp :: max (filled_slot . map (| v | v + 1) . unwrap_or (0) , highest_seq . map (| v | v + 1) . unwrap_or (0) ,) ; if sorted_payload . seq == expected_next_slot { * filled_slot = Some (sorted_payload . seq) ; } } }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Chain( - Map { + right: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < usize > > ({ use hydro_lang :: __staged :: optional :: * ; | v | Some (v) }), input: CycleSource { ident: Ident { @@ -1272,27 +3750,131 @@ expression: built.ir() 3, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( [:: std :: option :: Option :: None], ), location_kind: Cluster( 3, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, - ), - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, }, CycleSink { ident: Ident { @@ -1304,35 +3886,123 @@ expression: built.ir() 3, ), ), - input: DeferTick( - Tee { + input: DeferTick { + input: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | | (HashMap :: new () , None) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , () > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (kv_store , last_seq) , payload | { if let Some (kv) = payload . kv { kv_store . insert (kv . key , kv . value) ; } debug_assert ! (payload . seq == (last_seq . map (| s | s + 1) . unwrap_or (0)) , "Hole in log between seq {:?} and {}" , * last_seq , payload . seq) ; * last_seq = Some (payload . seq) ; } }), - input: Persist( - Tee { + input: Persist { + inner: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, }, CycleSink { ident: Ident { @@ -1344,18 +4014,18 @@ expression: built.ir() 3, ), ), - input: DeferTick( - Tee { + input: DeferTick { + input: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , usize) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; let checkpoint_frequency__free = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if max_checkpointed_seq . map (| m | new_highest_seq - m >= checkpoint_frequency__free) . unwrap_or (true) { Some (new_highest_seq) } else { None } }), - input: CrossSingleton( - Chain( - Map { + input: CrossSingleton { + left: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < usize > > ({ use hydro_lang :: __staged :: optional :: * ; | v | Some (v) }), input: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - CycleSource { + input: Persist { + inner: CycleSource { ident: Ident { sym: cycle_3, }, @@ -1365,28 +4035,124 @@ expression: built.ir() 3, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, ), }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( [:: std :: option :: Option :: None], ), location_kind: Cluster( 3, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, - ), - ), - Tee { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (core :: option :: Option < usize > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, }, CycleSink { ident: Ident { @@ -1397,6 +4163,14 @@ expression: built.ir() ), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, }, CycleSink { @@ -1409,11 +4183,11 @@ expression: built.ir() 2, ), ), - input: DeferTick( - AntiJoin( - Tee { - inner: : Chain( - CycleSource { + input: DeferTick { + input: AntiJoin { + pos: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_2, }, @@ -1423,8 +4197,16 @@ expression: built.ir() 2, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, }, - Tee { + second: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , ((u32 , u32) , core :: result :: Result < () , () >)) , ((u32 , u32) , core :: result :: Result < () , () >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -1449,15 +4231,82 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | payload | payload . kv }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , ((u32 , u32) , core :: result :: Result < () , () >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 9, + Cluster( + 2, + ), + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, }, - Tee { + neg: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , (usize , usize)) , core :: option :: Option < (u32 , u32) > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { @@ -1466,13 +4315,69 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < () , () > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , (usize , usize)), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1481,8 +4386,8 @@ expression: built.ir() location_kind: Cluster( 2, ), - input: Chain( - FlatMap { + input: Chain { + first: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: bench_client :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; let num_clients_per_node__free = 1usize ; move | _ | (0 .. num_clients_per_node__free) . map (move | i | ((CLUSTER_SELF_ID__free . raw_id * (num_clients_per_node__free as u32)) + i as u32 , 0)) }), input: Tee { inner: : Source { @@ -1492,18 +4397,74 @@ expression: built.ir() location_kind: Cluster( 2, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (u32 , u32) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | payload | (payload . 0 , payload . 1 + 1) }), input: Tee { inner: : Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1515,12 +4476,12 @@ expression: built.ir() 2, ), ), - input: DeferTick( - ReduceKeyed { + input: DeferTick { + input: ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < tokio :: time :: Instant , tokio :: time :: Instant , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; | curr_time , new_time | { if new_time > * curr_time { * curr_time = new_time ; } } }), - input: Chain( - Chain( - Tee { + input: Chain { + first: Chain { + first: Tee { inner: : CycleSource { ident: Ident { sym: cycle_3, @@ -1531,58 +4492,186 @@ expression: built.ir() 2, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), }, }, - FlatMap { + second: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: bench_client :: * ; let num_clients_per_node__free = 1usize ; move | now | (0 .. num_clients_per_node__free) . map (move | virtual_id | (virtual_id , now)) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , tokio :: time :: Instant > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | Instant :: now () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, }, - ), - Tee { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + second: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (usize , tokio :: time :: Instant) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (key , _prev_count) | (key as usize , Instant :: now ()) }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), ), }, - ), + }, }, ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; move | (latencies , throughput) | { let mut latencies_mut = latencies . borrow_mut () ; if latencies_mut . len () > 0 { let middle_idx = latencies_mut . len () / 2 ; let (_ , median , _) = latencies_mut . select_nth_unstable (middle_idx) ; println ! ("Median latency: {}ms" , median . as_micros () as f64 / 1000.0) ; } println ! ("Throughput: {} requests/s" , throughput) ; } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , ()) , (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - CrossSingleton( - Map { + input: CrossSingleton { + left: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (latencies , _) | latencies }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) > ({ use crate :: __staged :: cluster :: bench_client :: * ; let median_latency_window_size__free = 1usize ; move | | (Rc :: new (RefCell :: new (Vec :: < Duration > :: with_capacity (median_latency_window_size__free))) , 0usize ,) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , core :: time :: Duration , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; let median_latency_window_size__free = 1usize ; move | (latencies , write_index) , latency | { let mut latencies_mut = latencies . borrow_mut () ; if * write_index < latencies_mut . len () { latencies_mut [* write_index] = latency ; } else { latencies_mut . push (latency) ; } * write_index = (* write_index + 1) % median_latency_window_size__free ; } }), - input: Persist( - FlatMap { + input: Persist { + inner: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < core :: time :: Duration > , core :: option :: Option < core :: time :: Duration > > ({ use hydro_lang :: __staged :: stream :: * ; | d | d }), - input: Chain( - Map { + input: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (tokio :: time :: Instant , tokio :: time :: Instant)) , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (_virtual_id , (prev_time , curr_time)) | Some (curr_time . duration_since (prev_time)) }), - input: Join( - Tee { + input: Join { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , (tokio :: time :: Instant , tokio :: time :: Instant)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), + }, }, - DeferTick( - Map { + second: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | None }), input: Tee { inner: : Source { @@ -1592,33 +4681,133 @@ expression: built.ir() location_kind: Cluster( 2, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + core :: time :: Duration, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, ), ), + output_type: Some( + core :: time :: Duration, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > >, ), }, }, - Fold { + right: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: bench_client :: * ; | | 0 }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , bool) , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; | total , (batch_size , reset) | { if reset { * total = 0 ; } else { * total += batch_size ; } } }), - input: Persist( - Chain( - Map { + input: Persist { + inner: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , (usize , bool) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | batch_size | (batch_size , false) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Fold { + input: CrossSingleton { + left: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (u32 , u32) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: singleton :: * ; | c | * c == 0 }), @@ -1627,32 +4816,182 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), ), }, }, - DeferTick( - Map { + second: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , (usize , bool) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | (0 , true) }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, ), ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize), ), }, - ), - Map { + }, + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize), + ), + }, }, }, ] diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap index 7c48e8e03c4..c27da6e316d 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap @@ -41,10 +41,10 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32)) > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | (id , n) | (id , (id , n)) }), - input: Delta( - CrossProduct( - Persist( - Map { + input: Delta { + inner: CrossProduct { + left: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < & hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | & id | id }), input: Source { source: Iter( @@ -53,23 +53,111 @@ expression: built.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + & hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () >, + ), }, }, - ), - Persist( - Source { + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () >, + ), + }, + }, + right: Persist { + inner: Source { source: Iter( { use crate :: __staged :: cluster :: simple_cluster :: * ; 0 .. 5 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32)), ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), }, }, }, diff --git a/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap b/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap index 1bc38df8f75..0f408f4ec20 100644 --- a/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap +++ b/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap @@ -26,6 +26,22 @@ expression: built.ir() location_kind: ExternalProcess( 0, ), + metadata: HydroNodeMetadata { + location_kind: ExternalProcess( + 0, + ), + output_type: Some( + std :: string :: String, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + std :: string :: String, + ), }, }, }, @@ -56,8 +72,32 @@ expression: built.ir() location_kind: Process( 1, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + hydro_test :: distributed :: first_ten :: SendOverNetwork, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 2, + ), + output_type: Some( + hydro_test :: distributed :: first_ten :: SendOverNetwork, + ), + }, }, }, ] From dc51d94d390672bd687a7fdc9c826a778bdf0256 Mon Sep 17 00:00:00 2001 From: Mingwei Samuel Date: Wed, 29 Jan 2025 10:09:34 -0800 Subject: [PATCH 13/29] chore(dfir_rs)!: switch transducer -> process (fix #1572) (#1684) From #1635. fix #1572. BREAKING CHANGE: Renames `TransducerBuilderContext` to `ProcessBuilderContext` --------- Co-authored-by: Joe Hellerstein --- datastores/gossip_kv/kv/membership.rs | 4 +- datastores/gossip_kv/kv/server.rs | 2 +- .../gossip_kv/load_test_server/server.rs | 2 +- dfir_rs/examples/kvs_bench/server.rs | 16 ++--- dfir_rs/examples/shopping/README.md | 8 +-- dfir_rs/examples/shopping/driver.rs | 8 +-- .../shopping/flows/client_state_flow.rs | 6 +- dfir_rs/examples/shopping/flows/orig_flow.rs | 2 +- .../shopping/flows/server_state_flow.rs | 6 +- dfir_rs/src/scheduled/ticks.rs | 12 ++-- dfir_rs/src/util/simulation.rs | 68 +++++++++---------- docs/docs/deploy/your-first-deploy.md | 2 +- docs/docs/dfir/concepts/distributed_time.md | 2 +- docs/docs/dfir/concepts/index.md | 24 +++---- docs/docs/dfir/concepts/life_and_times.md | 18 ++--- docs/docs/dfir/concepts/stratification.md | 12 ++-- docs/docs/dfir/ecosystem.md | 6 +- docs/docs/dfir/faq.md | 10 +-- docs/docs/dfir/quickstart/setup.md | 6 +- docs/docs/hydro/dataflow-programming.mdx | 4 +- docs/docs/hydro/index.mdx | 2 +- .../hydro/live-collections/determinism.md | 8 +-- docs/docs/hydro/quickstart/clusters.mdx | 10 +-- docs/docs/hydro/quickstart/first-dataflow.mdx | 6 +- docs/docs/hydro/stageleft.mdx | 7 ++ docs/package-lock.json | 27 +++++--- 26 files changed, 146 insertions(+), 132 deletions(-) diff --git a/datastores/gossip_kv/kv/membership.rs b/datastores/gossip_kv/kv/membership.rs index 3657c87d3cd..1a06c50fc2e 100644 --- a/datastores/gossip_kv/kv/membership.rs +++ b/datastores/gossip_kv/kv/membership.rs @@ -7,8 +7,8 @@ pub type MemberId = String; /// Information about a member in the cluster. /// -/// A member is a transducer that is part of the cluster. Leaving or failing is a terminal -/// state for a member. When a transducer restarts and rejoins the cluster, it is considered a +/// A member is a process that is part of the cluster. Leaving or failing is a terminal +/// state for a member. When a process restarts and rejoins the cluster, it is considered a /// new member. /// /// # Generic Parameters diff --git a/datastores/gossip_kv/kv/server.rs b/datastores/gossip_kv/kv/server.rs index f3d2afc199e..d7727c463c5 100644 --- a/datastores/gossip_kv/kv/server.rs +++ b/datastores/gossip_kv/kv/server.rs @@ -105,7 +105,7 @@ where dfir_syntax! { on_start = initialize() -> tee(); - on_start -> for_each(|_| info!("{:?}: Transducer {} started.", context.current_tick(), member_id_6)); + on_start -> for_each(|_| info!("{:?}: Process {} started.", context.current_tick(), member_id_6)); seed_nodes = source_stream(seed_node_stream) -> fold::<'static>(|| Box::new(seed_nodes), |last_seed_nodes, new_seed_nodes: Vec>| { diff --git a/datastores/gossip_kv/load_test_server/server.rs b/datastores/gossip_kv/load_test_server/server.rs index df9df93699a..0b36491e898 100644 --- a/datastores/gossip_kv/load_test_server/server.rs +++ b/datastores/gossip_kv/load_test_server/server.rs @@ -27,7 +27,7 @@ const UNKNOWN_ADDRESS: LoadTestAddress = 9999999999; #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Parser)] struct Opts { - /// Number of threads to run. Each thread will run an instance of the gossip-kv server transducer. + /// Number of threads to run. Each thread will run an instance of the gossip-kv server process. #[clap(short, long, default_value = "5")] thread_count: usize, diff --git a/dfir_rs/examples/kvs_bench/server.rs b/dfir_rs/examples/kvs_bench/server.rs index 7a270f8bac3..b6fcfa02f6d 100644 --- a/dfir_rs/examples/kvs_bench/server.rs +++ b/dfir_rs/examples/kvs_bench/server.rs @@ -46,12 +46,12 @@ pub fn run_server( let buffer_pool = BufferPool::::create_buffer_pool(); - let (transducer_to_peers_tx, mut transducer_to_peers_rx) = + let (process_to_peers_tx, mut process_to_peers_rx) = dfir_rs::util::unsync_channel::<(Bytes, NodeId)>(None); - let (client_to_transducer_tx, client_to_transducer_rx) = + let (client_to_process_tx, client_to_process_rx) = dfir_rs::util::unsync_channel::<(KvsRequest, NodeId)>(None); - let (transducer_to_client_tx, mut _transducer_to_client_rx) = + let (process_to_client_tx, mut _process_to_client_rx) = dfir_rs::util::unsync_channel::<(KvsResponse, NodeId)>(None); let localset = task::LocalSet::new(); @@ -71,7 +71,7 @@ pub fn run_server( collector: Rc::clone(&buffer_pool), }.deserialize(&mut deserializer).unwrap(); - client_to_transducer_tx.try_send((req, node_id)).unwrap(); + client_to_process_tx.try_send((req, node_id)).unwrap(); } } }) @@ -86,7 +86,7 @@ pub fn run_server( // TODO: Eventually this would get moved into a dfir operator that would return a Bytes struct and be efficient and zero copy and etc. async move { loop { - while let Some((serialized_req, node_id)) = transducer_to_peers_rx.next().await { + while let Some((serialized_req, node_id)) = process_to_peers_rx.next().await { let index = lookup.binary_search(&node_id).unwrap(); topology.tx[index].send(serialized_req).unwrap(); } @@ -170,7 +170,7 @@ pub fn run_server( union_puts_and_gossip_requests = union(); simulated_put_requests -> union_puts_and_gossip_requests; - source_stream(client_to_transducer_rx) + source_stream(client_to_process_rx) // -> inspect(|x| println!("{server_id}:{:5}: from peers: {x:?}", context.current_tick())) -> union_puts_and_gossip_requests; @@ -244,7 +244,7 @@ pub fn run_server( peers // -> inspect(|x| println!("{server_id}:{:5}: sending to peers: {x:?}", context.current_tick())) - -> for_each(|(node_id, serialized_req)| transducer_to_peers_tx.try_send((serialized_req, node_id)).unwrap()); + -> for_each(|(node_id, serialized_req)| process_to_peers_tx.try_send((serialized_req, node_id)).unwrap()); // join for lookups lookup = _lattice_join_fused_join::<'static, 'tick, MyLastWriteWins, MySetUnion>(); @@ -275,7 +275,7 @@ pub fn run_server( }) -> flatten() // -> inspect(|x| println!("{gossip_addr}:{:5}: Response to client: {x:?}", context.current_tick())) - -> for_each(|x| transducer_to_client_tx.try_send(x).unwrap()); + -> for_each(|x| process_to_client_tx.try_send(x).unwrap()); }; diff --git a/dfir_rs/examples/shopping/README.md b/dfir_rs/examples/shopping/README.md index cc61b1d277e..48e95264f52 100644 --- a/dfir_rs/examples/shopping/README.md +++ b/dfir_rs/examples/shopping/README.md @@ -23,10 +23,10 @@ cargo run -p hydroflow --example shopping -- --opt 5 Adding the `--graph ` flag to the end of the command lines above will print out a node-and-edge diagram of the program. Supported values for `` include [mermaid](https://mermaid-js.github.io/) and [dot](https://graphviz.org/doc/info/lang.html). -For options 1-4, the driver runs a single Hydro transducer (thread) that handles client requests. +For options 1-4, the driver runs a single Hydro process (thread) that handles client requests. -For options 5-6, the driver runs two Hydro transducers, one for each side of the network communication. +For options 5-6, the driver runs two Hydro processes, one for each side of the network communication. -For option 7, the driver runs four Hydro transducers: one client proxy and 3 server replicas. +For option 7, the driver runs four Hydro processes: one client proxy and 3 server replicas. -Under all options, the driver runs an additional independent Hydro transducer (thread) to receive the output of the flow and print it to the console. The code for this transducer is in `flows/listener_flow.rs`. +Under all options, the driver runs an additional independent Hydro process (thread) to receive the output of the flow and print it to the console. The code for this process is in `flows/listener_flow.rs`. diff --git a/dfir_rs/examples/shopping/driver.rs b/dfir_rs/examples/shopping/driver.rs index 64a9dd53326..2f3f3033b30 100644 --- a/dfir_rs/examples/shopping/driver.rs +++ b/dfir_rs/examples/shopping/driver.rs @@ -83,8 +83,8 @@ pub(crate) async fn run_driver(opts: Opts) { let client_out_addr = ipv4_resolve("localhost:23460").unwrap(); let (client_out, _, _) = dfir_rs::util::bind_udp_bytes(client_out_addr).await; - // shopping input is handled by the client proxy transducer - // so the server transducer should get an empty iterator as its first argument + // shopping input is handled by the client proxy process + // so the server process should get an empty iterator as its first argument let empty_ssiv = std::iter::empty(); // Spawn server @@ -180,8 +180,8 @@ pub(crate) async fn run_driver(opts: Opts) { let (out, _, _) = dfir_rs::util::bind_udp_bytes(out_addr).await; let gossip_addrs = gossip_addrs.clone(); - // shopping input is handled by the client proxy transducer - // so the server transducers should get an empty iterator as first argument + // shopping input is handled by the client proxy process + // so the server processes should get an empty iterator as first argument let empty_ssiv = std::iter::empty(); // Spawn server diff --git a/dfir_rs/examples/shopping/flows/client_state_flow.rs b/dfir_rs/examples/shopping/flows/client_state_flow.rs index 7c8facd7659..0dd594728c1 100644 --- a/dfir_rs/examples/shopping/flows/client_state_flow.rs +++ b/dfir_rs/examples/shopping/flows/client_state_flow.rs @@ -29,9 +29,9 @@ pub(crate) async fn client_state_flow( // Set up the Udp socket for proxy-server communication let (carts_out, carts_in, _) = dfir_rs::util::bind_udp_bytes(local_addr).await; - // This is like server_state_flow, but we split it into two transducers at a different spot. - // Here, the first transducer takes in shopping_ssiv requests and runs a stateful fold_keyed, persisting all the shopping requests in ssiv's. - // The second transducer listens on reqs_in and runs the lookup join. + // This is like server_state_flow, but we split it into two processes at a different spot. + // Here, the first process takes in shopping_ssiv requests and runs a stateful fold_keyed, persisting all the shopping requests in ssiv's. + // The second process listens on reqs_in and runs the lookup join. dfir_syntax! { source_iter(shopping_ssiv) -> fold_keyed::<'static>(SSIV_BOT, ssiv_merge) diff --git a/dfir_rs/examples/shopping/flows/orig_flow.rs b/dfir_rs/examples/shopping/flows/orig_flow.rs index db0570dea9b..4f1fc343d93 100644 --- a/dfir_rs/examples/shopping/flows/orig_flow.rs +++ b/dfir_rs/examples/shopping/flows/orig_flow.rs @@ -17,7 +17,7 @@ pub(crate) async fn orig_flow( ) -> Dfir<'static> { let client_class = client_class_iter(); - // This is the straightforward single-transducer sequential case. + // This is the straightforward single-process sequential case. // For each Request in "shopping" we look up its "client_class" (basic or prime) // via a join operator, then we group by (client, class), and for each such pair // we grow a separate vector of ClLineItems. No seal is needed in the sequential case. diff --git a/dfir_rs/examples/shopping/flows/server_state_flow.rs b/dfir_rs/examples/shopping/flows/server_state_flow.rs index 0f69fe84c4d..5941c45ed3a 100644 --- a/dfir_rs/examples/shopping/flows/server_state_flow.rs +++ b/dfir_rs/examples/shopping/flows/server_state_flow.rs @@ -29,9 +29,9 @@ pub(crate) async fn server_state_flow( // Set up the Udp socket for proxy-server communication let (reqs_out, reqs_in, _) = dfir_rs::util::bind_udp_bytes(local_addr).await; - // This is like push_group_flow, but we split it into two transducers that communicate via reqs_out and reqs_in. - // The first transducer takes in shopping_ssiv requests, and forwards them via reqs_out to the second transducer. - // The second transducer listens on reqs_in and runs the stateful logic of fold_keyed and join. + // This is like push_group_flow, but we split it into two processes that communicate via reqs_out and reqs_in. + // The first process takes in shopping_ssiv requests, and forwards them via reqs_out to the second process. + // The second process listens on reqs_in and runs the stateful logic of fold_keyed and join. dfir_syntax! { // Networked: Server-Side State source_iter(shopping_ssiv) diff --git a/dfir_rs/src/scheduled/ticks.rs b/dfir_rs/src/scheduled/ticks.rs index 205f1db40bf..9229f60d2cb 100644 --- a/dfir_rs/src/scheduled/ticks.rs +++ b/dfir_rs/src/scheduled/ticks.rs @@ -1,16 +1,16 @@ //! This module contains types to work with ticks. //! -//! Each iteration of a Hydroflow transducer loop is called a tick. Associated with the transducer -//! is a clock value, which tells you how many ticks were executed by this transducer prior to the -//! current tick. Each transducer produces totally ordered, sequentially increasing clock values, -//! which you can think of as the "local logical time" at the transducer. +//! Each iteration of a Hydroflow process loop is called a tick. Associated with the process +//! is a clock value, which tells you how many ticks were executed by this process prior to the +//! current tick. Each process produces totally ordered, sequentially increasing clock values, +//! which you can think of as the "local logical time" at the process. use std::fmt::{Display, Formatter}; use std::ops::{Add, AddAssign, Neg, Sub, SubAssign}; use serde::{Deserialize, Serialize}; -/// A point in time during execution on transducer. +/// A point in time during execution on process. /// /// `TickInstant` instances can be subtracted to calculate the `TickDuration` between them. /// @@ -66,7 +66,7 @@ pub struct TickDuration { impl TickInstant { /// Create a new TickInstant /// - /// The specified parameter indicates the number of ticks that have elapsed on the transducer, + /// The specified parameter indicates the number of ticks that have elapsed on the process, /// prior to this one. pub fn new(ticks: u64) -> Self { TickInstant(ticks) diff --git a/dfir_rs/src/util/simulation.rs b/dfir_rs/src/util/simulation.rs index fa173ad0f02..c71c11cda2b 100644 --- a/dfir_rs/src/util/simulation.rs +++ b/dfir_rs/src/util/simulation.rs @@ -1,11 +1,11 @@ //! # Hydroflow Deterministic Simulation Testing Framework //! //! This module provides a deterministic simulation testing framework for testing Hydroflow -//! transducers. +//! processes. //! -//! It can be used to test complex interactions between multiple Hydroflow transducers in a +//! It can be used to test complex interactions between multiple Hydroflow processes in a //! deterministic manner by running them in a single-threaded environment. The framework also -//! provides a "virtual network" implementation that allows production transducers to exchange +//! provides a "virtual network" implementation that allows production processes to exchange //! messages within the simulation. More importantly, the network is fully under control of the //! unit test and the test can introduce faults such as message delays, message drops and //! network partitions. @@ -13,7 +13,7 @@ //! ## Overview //! //! Conceptually, the simulation contains a "Fleet", which is a collection of "Hosts". These -//! aren't real hosts, but rather a collection of individual Hydroflow transducers (one per host) +//! aren't real hosts, but rather a collection of individual Hydroflow processes (one per host) //! that can communicate with each other over a virtual network. Every host has a "hostname" //! which uniquely identifies it within the fleet. //! @@ -47,7 +47,7 @@ //! ## Network Processing //! //! ### Outboxes & Inboxes -//! When a transducer wishes to send a message to another transducer, it sends the message to an +//! When a process wishes to send a message to another process, it sends the message to an //! "outbox" on its host. The unit test invokes the simulation's network message processing logic //! at some desired cadence to pick up all messages from all outboxes and deliver them to the //! corresponding inboxes on the destination hosts. The network message processing logic is the @@ -55,13 +55,13 @@ //! //! ### Interface Names //! Every inbox and outbox is associated with an "interface name". This is a string that uniquely -//! identifies the interface on the host. When a transducer sends a message, it specifies the +//! identifies the interface on the host. When a process sends a message, it specifies the //! destination hostname and the interface name on that host to which the message should be //! delivered. //! //! ## Progress of Time in the Simulation //! The single-threaded unit test can drive time forward on every host by invoking the `run_tick` -//! method on the host. This ultimately runs a single tick on the transducer. The unit test is +//! method on the host. This ultimately runs a single tick on the process. The unit test is //! also responsible for invoking the network message processing at the time of its choosing and //! can interleave the progress of time on various hosts and network processing as it sees fit. //! @@ -85,7 +85,7 @@ use crate::scheduled::graph::Dfir; use crate::util::{collect_ready_async, unbounded_channel}; /// A hostname is a unique identifier for a host in the simulation. It is used to address messages -/// to a specific host (and thus a specific Hydroflow transducer). +/// to a specific host (and thus a specific Hydroflow process). pub type Hostname = String; /// An interface name is a unique identifier for an inbox or an outbox on host. @@ -127,45 +127,45 @@ impl MessageSender for UnboundedSender<(T, Address)> { /// A message with an delivery address. pub type MessageWithAddress = (Box, Address); -/// An inbox is used by a host to receive messages for the transducer. +/// An inbox is used by a host to receive messages for the process. pub struct Inbox { sender: Box, } -/// Transducers can send messages to other transducers by putting those messages in an outbox +/// Processes can send messages to other processes by putting those messages in an outbox /// on their host. pub struct Outbox { receiver: Pin>>, } -/// A host is a single Hydroflow transducer running in the simulation. It has a unique hostname +/// A host is a single Hydroflow process running in the simulation. It has a unique hostname /// and can communicate with other hosts over the virtual network. It has a collection of inboxes /// and outboxes. pub struct Host { name: Hostname, - transducer: Dfir<'static>, + process: Dfir<'static>, inputs: HashMap, output: HashMap, } impl Host { - /// Run a single tick on the host's transducer. Returns true if any work was done by the - /// transducer. This effectively "advances" time on the transducer. + /// Run a single tick on the host's process. Returns true if any work was done by the + /// process. This effectively "advances" time on the process. pub fn run_tick(&mut self) -> bool { - self.transducer.run_tick() + self.process.run_tick() } } /// A builder for constructing a host in the simulation. pub struct HostBuilder { name: Hostname, - transducer: Option>, + process: Option>, inboxes: HashMap, outboxes: HashMap, } /// Used in conjunction with the `HostBuilder` to construct a host in the simulation. -pub struct TransducerBuilderContext<'context> { +pub struct ProcessBuilderContext<'context> { inboxes: &'context mut HashMap, outboxes: &'context mut HashMap, } @@ -177,9 +177,9 @@ fn sink_from_fn(mut f: impl FnMut(T)) -> impl Sink { }) } -impl TransducerBuilderContext<'_> { +impl ProcessBuilderContext<'_> { /// Create a new inbox on the host with the given interface name. Returns a stream that can - /// be read by the transducer using the source_stream dfir operator. + /// be read by the process using the source_stream dfir operator. pub fn new_inbox( &mut self, interface: InterfaceName, @@ -195,7 +195,7 @@ impl TransducerBuilderContext<'_> { } /// Creates a new outbox on the host with the given interface name. Returns a sink that can - /// be written to by the transducer using the dest_sink dfir operator. + /// be written to by the process using the dest_sink dfir operator. pub fn new_outbox( &mut self, interface: InterfaceName, @@ -220,35 +220,35 @@ impl HostBuilder { pub fn new(name: Hostname) -> Self { HostBuilder { name, - transducer: None, + process: None, inboxes: Default::default(), outboxes: Default::default(), } } - /// Supplies the (mandatory) transducer that runs on this host. - pub fn with_transducer(mut self, builder: F) -> Self + /// Supplies the (mandatory) process that runs on this host. + pub fn with_process(mut self, builder: F) -> Self where - F: FnOnce(&mut TransducerBuilderContext) -> Dfir<'static>, + F: FnOnce(&mut ProcessBuilderContext) -> Dfir<'static>, { - let mut context = TransducerBuilderContext { + let mut context = ProcessBuilderContext { inboxes: &mut self.inboxes, outboxes: &mut self.outboxes, }; - let transducer = builder(&mut context); - self.transducer = Some(transducer); + let process = builder(&mut context); + self.process = Some(process); self } /// Builds the host with the supplied configuration. pub fn build(self) -> Host { - if self.transducer.is_none() { - panic!("Transducer is required to build a host"); + if self.process.is_none() { + panic!("Process is required to build a host"); } Host { name: self.name, - transducer: self.transducer.unwrap(), + process: self.process.unwrap(), inputs: self.inboxes, output: self.outboxes, } @@ -269,13 +269,13 @@ impl Fleet { } } - /// Adds a new host to the fleet with the given name and transducer. - pub fn add_host(&mut self, name: String, transducer_builder: F) -> &Host + /// Adds a new host to the fleet with the given name and process. + pub fn add_host(&mut self, name: String, process_builder: F) -> &Host where - F: FnOnce(&mut TransducerBuilderContext) -> Dfir<'static>, + F: FnOnce(&mut ProcessBuilderContext) -> Dfir<'static>, { let host = HostBuilder::new(name.clone()) - .with_transducer(transducer_builder) + .with_process(process_builder) .build(); assert!( self.hosts.insert(host.name.clone(), host).is_none(), diff --git a/docs/docs/deploy/your-first-deploy.md b/docs/docs/deploy/your-first-deploy.md index 86f10de7808..3216ffc891d 100644 --- a/docs/docs/deploy/your-first-deploy.md +++ b/docs/docs/deploy/your-first-deploy.md @@ -5,7 +5,7 @@ sidebar_position: 3 # Your First Deploy Now that we have Hydro Deploy installed, let's deploy our first app. We'll start with a simple app that echoes packets. -First, we need to write the DFIR application, which will intergrate with Hydro Deploy to initialize connections to other services. We can create a project using the Cargo template: +First, we need to write the DFIR application, which will integrate with Hydro Deploy to initialize connections to other services. We can create a project using the Cargo template: ```bash # if you don't already have cargo-generate installed diff --git a/docs/docs/dfir/concepts/distributed_time.md b/docs/docs/dfir/concepts/distributed_time.md index e7d10efacfd..bd6420137ec 100644 --- a/docs/docs/dfir/concepts/distributed_time.md +++ b/docs/docs/dfir/concepts/distributed_time.md @@ -19,7 +19,7 @@ The way that Lamport clocks jump ahead provides a desirable property: the timest Lamport timestamps track not only the order of events on a single node, they also ensure that the timestamps on events reflect distributed ordering. Suppose that node `source` wants to send a message to node `dest`, and node source has current clock value *T_source*. The events that precede that message on node `source` have smaller timestamps. In addition, consider an event at node `dest` that follows the receipt of that message. That event must have a timestamp greater than *T_source* by Lamport's advancing rule above. Hence all the events on node `source` that preceded the sending of the message have lower timestamps than the events on node `dest` following the receipt of the message. This is Lamport's distributed "happens-before" relation, and the Lamport clock capture that relation. ## DFIR Time -As a built-in primitive, DFIR defines time only for a single transducer, as a sequence of consecutive ticks without any gaps. +As a built-in primitive, DFIR defines time only for a single process, as a sequence of consecutive ticks without any gaps. Thus the main difference between DFIR events and Lamport events are: diff --git a/docs/docs/dfir/concepts/index.md b/docs/docs/dfir/concepts/index.md index 0baa66c9aaa..2bbc4c7d179 100644 --- a/docs/docs/dfir/concepts/index.md +++ b/docs/docs/dfir/concepts/index.md @@ -25,31 +25,31 @@ DFIR is a library that can be used in any Rust program. It includes two main com DFIR's surface syntax must be embedded in a Rust program; the Rust compiler takes that DFIR syntax and compiles it into an efficient binary executable. -> We call a running DFIR binary a *transducer*. +> We call a running DFIR binary a *process*. -In typical usage, a developer writes a transducer as a single-threaded Rust program that is mostly composed of -DFIR surface syntax. Each transducer is typically responsible for a single -"node" (a machine, or a core) in a distributed system composed of many such transducers, +In typical usage, a developer writes a process as a single-threaded Rust program that is mostly composed of +DFIR surface syntax. Each process is typically responsible for a single +"node" (a machine, or a core) in a distributed system composed of many such processes, which send and receive flows of data to each other. -> DFIR itself does not generate distributed code. It is a library for specifying the transducers (individual nodes) that +> DFIR itself does not generate distributed code. It is a library for specifying the processes (individual nodes) that > participate in a distributed system. > > In the [Hydro Project](https://hydro.run), higher-level languages are being built on top of DFIR to generate -> distributed code in the form of multiple transducers. -> Meanwhile, you can use DFIR to write your own distributed code, by writing individual transducers that work together, +> distributed code in the form of multiple processes. +> Meanwhile, you can use DFIR to write your own distributed code, by writing individual processes that work together, > and deploying them manually or with a tool like [Hydroplane](https://github.com/hydro-project/hydroplane). See the [Hydro Ecosystem](../ecosystem) for more on this. ### So how might a human write distributed systems with DFIR? As an illustration of how you can work at the DFIR layer, consider the [Chat Server example](https://github.com/hydro-project/hydro/tree/main/dfir_rs/examples/chat). If you run that binary -with the command-line argument `--role server` it will start a single transducer that is responsible for a chat server: receiving +with the command-line argument `--role server` it will start a single process that is responsible for a chat server: receiving membership requests and messages from clients, and forwarding messages from individual clients to all other clients. -If you run that binary with the argument `--role client` it will start a transducer that is responsible for a chat client, which +If you run that binary with the argument `--role client` it will start a process that is responsible for a chat client, which forwards chat messages from stdin to the server, and prints out messages sent by the server. As a distributed system, the chat -service would typically consist of many client transducers and a single server transducer. +service would typically consist of many client processes and a single server process. -Note that this is an example of an extremely simple distributed system in a "star" or "hub-and spokes" topology: the multiple client transducers are completely independent of each other, and each talks only with the central server transducer. +Note that this is an example of an extremely simple distributed system in a "star" or "hub-and spokes" topology: the multiple client processes are completely independent of each other, and each talks only with the central server process.

- If we wanted something more interesting, we could consider deploying a cluster of multiple server transducers, say for fault tolerance or geo-distribution, perhaps like this: + If we wanted something more interesting, we could consider deploying a cluster of multiple server processes, say for fault tolerance or geo-distribution, perhaps like this:
diff --git a/docs/docs/dfir/concepts/life_and_times.md b/docs/docs/dfir/concepts/life_and_times.md index 41adf78b2a4..d35ba6b7fba 100644 --- a/docs/docs/dfir/concepts/life_and_times.md +++ b/docs/docs/dfir/concepts/life_and_times.md @@ -2,22 +2,22 @@ sidebar_position: 1 --- -# The Life and Times of a DFIR Transducer +# The Life and Times of a DFIR Process Time is a fundamental concept in many distributed systems. DFIR's model of time is very simple, but also powerful enough to serve as a building block for more complex models of time. -Like most reactive services, we can envision a DFIR transducer running as an unbounded loop that is managed -by the runtime library. Each iteration of the transducer's loop is called a *tick*. Associated with the transducer is +Like most reactive services, we can envision a DFIR process running as an unbounded loop that is managed +by the runtime library. Each iteration of the process's loop is called a *tick*. Associated with the process is a *clock* value (accessible via the `.current_tick()` method), which tells you how many ticks were executed -by this transducer prior to the current tick. Each transducer produces totally ordered, sequentially increasing clock values, -which you can think of as the "local logical time" at the transducer. +by this process prior to the current tick. Each process produces totally ordered, sequentially increasing clock values, +which you can think of as the "local logical time" at the process. -The transducer's main loop works as follows: +The process's main loop works as follows: 1. Given events and messages buffered from the operating system, ingest a batch of data items and deliver them to the appropriate `source_xxx` operators in the DFIR spec. 2. Run the DFIR spec. If the spec has cycles, continue executing it until it reaches a "fixpoint" on the current batch; i.e. it no longer produces any new data anywhere in the flow. Along the way, any data that appears in an outbound channel is streamed to the appropriate destination. 3. Once the spec reaches fixpoint and messages have all been sent, advance the local clock and then start the next tick. -The transducer's main loop is shown in the following diagram: +The process's main loop is shown in the following diagram: ```mermaid %%{init: {'theme':'neutral'}}%% @@ -25,7 +25,7 @@ flowchart LR subgraph external[External] network>"messages\n& events"\nfa:fa-telegram]-->buffer[[buffer]] end - subgraph transducer[Transducer Loop] + subgraph process[Process Loop] buffer-->ingest>ingest a batch\nof data]-->loop(((run DFIR\nspec to fixpoint\nfa:fa-cog)))-->stream[stream out\nmsgs & events\nfa:fa-telegram]-->clock((advance\nclock\nfa:fa-clock-o))-- new tick! -->ingest end style stream fill:#0fa,stroke:#aaa,stroke-width:2px,stroke-dasharray: 5 5 @@ -35,4 +35,4 @@ flowchart LR linkStyle 5 stroke:red,stroke-width:4px,color:red; ``` -In sum, an individual transducer advances sequentially through logical time; in each tick of its clock it ingests a batch of data from its inbound channels, executes the DFIR spec, and sends any outbound data to its outbound channels. \ No newline at end of file +In sum, an individual process advances sequentially through logical time; in each tick of its clock it ingests a batch of data from its inbound channels, executes the DFIR spec, and sends any outbound data to its outbound channels. \ No newline at end of file diff --git a/docs/docs/dfir/concepts/stratification.md b/docs/docs/dfir/concepts/stratification.md index 50900d709cc..063d766f67b 100644 --- a/docs/docs/dfir/concepts/stratification.md +++ b/docs/docs/dfir/concepts/stratification.md @@ -3,12 +3,12 @@ sidebar_position: 3 --- # Streaming, Blocking and Stratification -Many DFIR operators (e.g. `map`, `filter` and `join`) work in a streaming fashion. Streaming operators process data as it arrives, generating outputs in the midst of processing inputs. If you restrict yourself to operators that work in this streaming fashion, then your transducer may start sending data across the network mid-tick, even while it is still consuming the data in the input batch. +Many DFIR operators (e.g. `map`, `filter` and `join`) work in a streaming fashion. Streaming operators process data as it arrives, generating outputs in the midst of processing inputs. If you restrict yourself to operators that work in this streaming fashion, then your process may start sending data across the network mid-tick, even while it is still consuming the data in the input batch. But some operators are blocking, and must wait for all their input data to arrive before they can produce any output data. For example, a `sort` operator must wait for all its input data to arrive before it can produce a single output value. After all, the lowest value may be the last to arrive! ## Examples of Blocking Operators -The discussion above should raise questions in your mind. What do we mean by "all the input data" in a long-running service? We don't want to wait until the end of time—this is one reason we break time up into discrete "ticks" at each transducer. So when we say that a blocking operator waits for "all the input data", we mean "all the input data in the current tick". +The discussion above should raise questions in your mind. What do we mean by "all the input data" in a long-running service? We don't want to wait until the end of time—this is one reason we break time up into discrete "ticks" at each process. So when we say that a blocking operator waits for "all the input data", we mean "all the input data in the current tick". Consider the simple statement below, which receives data from a network source each tick, sorts that tick's worth of data, and prints it to stdout: ```rust,ignore @@ -59,14 +59,14 @@ end The `difference` operators is one with inputs of two different types. It is supposed to output all the items from its `pos` input that do not appear in its `neg` input. To achieve that, the `neg` input must be blocking, but the `pos` input can stream. Blocking on the `neg` input ensures that if the operator streams an output from the `pos` input, it will never need to retract that output. -Given these examples, we can refine our diagram of the DFIR transducer loop to account for stratified execution within each tick: +Given these examples, we can refine our diagram of the DFIR process loop to account for stratified execution within each tick: ```mermaid %%{init: {'theme':'neutral'}}%% flowchart LR subgraph external[External] network>"messages\n& events"\nfa:fa-telegram]-->buffer[[buffer]] end - subgraph transducer[Transducer Loop] + subgraph process[Process Loop] buffer-->ingest>"ingest a batch\nof data"]-->loop((("for each stratum in\n[0..max(strata)]\nrun stratum to fixpoint\nfa:fa-cog")))-->stream[stream out\nmsgs & events\nfa:fa-telegram]-->clock((advance\nclock\nfa:fa-clock-o))-- new tick! -->ingest end style stream fill:#0fa,stroke:#aaa,stroke-width:2px,stroke-dasharray: 5 5 @@ -88,7 +88,7 @@ The DFIR compiler performs stratification via static analysis of the DFIR spec. Given the acyclicity test, any legal DFIR program consists of a directed acyclic graph (DAG) of strata and handoffs. The strata are numbered in ascending order by assigning stratum number 0 to the "leaves" of the DAG (strata with no upstream operators), and then ensuring that each stratum is assigned a number that is one larger than any of its upstream strata. -As a DFIR operator executes, it is running on a particular transducer, in a particular tick, in a particular stratum. +As a DFIR operator executes, it is running on a particular process, in a particular tick, in a particular stratum. ### Determining whether an operator should block: Monotonicity @@ -102,6 +102,6 @@ By contrast, consider the output of a blocking operator like `difference`. The o DFIR is designed to use the monotonicity property to determine whether an operator should block. If an operator is monotone with respect to an input, that input is streaming. If an operator is non-monotone, it is blocking. -Monotonicity turns out to be particularly important for distributed systems. In particular, if all your transducers are fully monotone across ticks, then they can run in parallel without any coordination—they will always stream correct prefixes of the final outputs, and eventually will deliver the complete output. This is the positive direction of the [CALM Theorem](https://cacm.acm.org/magazines/2020/9/246941-keeping-calm/fulltext). +Monotonicity turns out to be particularly important for distributed systems. In particular, if all your processes are fully monotone across ticks, then they can run in parallel without any coordination—they will always stream correct prefixes of the final outputs, and eventually will deliver the complete output. This is the positive direction of the [CALM Theorem](https://cacm.acm.org/magazines/2020/9/246941-keeping-calm/fulltext). > In future versions of DFIR, the type system will represent monotonicity explicitly and reason about it automatically. diff --git a/docs/docs/dfir/ecosystem.md b/docs/docs/dfir/ecosystem.md index e5b45fd115d..0f3a4ab8d2e 100644 --- a/docs/docs/dfir/ecosystem.md +++ b/docs/docs/dfir/ecosystem.md @@ -10,15 +10,15 @@ A rough picture of the Hydro stack is below: Working down from the top: -- [*Hydro*](../hydro) is an end-user-facing high-level [choreographic](https://en.wikipedia.org/wiki/Choreographic_programming) [dataflow](https://en.wikipedia.org/wiki/Dataflow_programming) language. Hydro is a *global* language for programming a fleet of transducers. Programmers author dataflow pipelines that start with streams of events and data, and span boundaries across multiple `process` and (scalable) `cluster` specifications. +- [*Hydro*](../hydro) is an end-user-facing high-level [choreographic](https://en.wikipedia.org/wiki/Choreographic_programming) [dataflow](https://en.wikipedia.org/wiki/Dataflow_programming) language. Hydro is a *global* language for programming a fleet of processes. Programmers author dataflow pipelines that start with streams of events and data, and span boundaries across multiple `process` and (scalable) `cluster` specifications. - *Hydrolysis* is a compiler that translates a global Hydro spec to multiple single-threaded DFIR programs, which collectively implement the global spec. This compilation phase is currently a part of the Hydro codebase, but will evolve into a standalone optimizing compiler inspired by database query optimizers and [e-graphs](https://en.wikipedia.org/wiki/E-graph). - [DFIR and its compiler/runtime](https://github.com/hydro-project/hydro/tree/main/dfir_rs) are the subject of this book. -Where Hydro is a *global* language for programming a fleet of processes, DFIR is a *local* language for programming a single process that participates in a distributed system. More specifically, DFIR is an internal representation (IR) language and runtime library that generates the low-level Rust code for an individual transducer. As a low-level IR, DFIR is not intended for the general-purpose programmer. For most users it is intended as a readable compiler target from Hydro; advanced developers can also use it to manually program individual transducers. +Where Hydro is a *global* language for programming a fleet of processes, DFIR is a *local* language for programming a single process that participates in a distributed system. More specifically, DFIR is an internal representation (IR) language and runtime library that generates the low-level Rust code for an individual process. As a low-level IR, DFIR is not intended for the general-purpose programmer. For most users it is intended as a readable compiler target from Hydro; advanced developers can also use it to manually program individual processes. -- [HydroDeploy](../deploy) is a service for launching DFIR transducers on a variety of platforms. +- [HydroDeploy](../deploy) is a service for launching DFIR processes on a variety of platforms. - Hydro also supports *Deterministic Simulation Testing* to aid in debugging distributed programs. Documentation on this feature is forthcoming. diff --git a/docs/docs/dfir/faq.md b/docs/docs/dfir/faq.md index 99cd641e6bb..365dab22047 100644 --- a/docs/docs/dfir/faq.md +++ b/docs/docs/dfir/faq.md @@ -7,8 +7,8 @@ sidebar_position: 9 **A:** DFIR is designed as a substrate for building a wide range of distributed systems; traditional software dataflow systems like the ones mentioned are targeted more narrowly at large-scale data processing. As such, DFIR differs from these systems in several ways: -First, DFIR is a lower-level abstraction than the systems mentioned. DFIR adopts a dataflow abstraction for specifying a transducer -running on a *single core*; one implementes a distributed system out of multiple DFIR transducers. By contrast, Spark, Flink and MapReduce are +First, DFIR is a lower-level abstraction than the systems mentioned. DFIR adopts a dataflow abstraction for specifying a process +running on a *single core*; one implementes a distributed system out of multiple DFIR processes. By contrast, Spark, Flink and MapReduce are distributed systems, which make specific choices for implementing issues like scheduling, fault tolerance, and so on. Second, the systems mentioned above were designed specifically for distributed data processing tasks. @@ -24,13 +24,13 @@ like Paxos or Two-Phase Commit with low latency. #### Q: What model of parallelism does DFIR use? SPMD? MPMD? Actors? MPI? **A:** As a substrate for building individual nodes in a distributed systems, DFIR does not make any assumptions about the model of parallelism used. One can construct a distributed system out of DFIR -transducers that use any model of parallelism, including +processes that use any model of parallelism, including [SPMD](https://en.wikipedia.org/wiki/Flynn%27s_taxonomy#Single_program,_multiple_data_streams_(SPMD)), [MPMD](https://en.wikipedia.org/wiki/Flynn%27s_taxonomy#Multiple_programs,_multiple_data_streams_(MPMD)), [Actors](https://en.wikipedia.org/wiki/Actor_model) or [MPI Collective Operations](https://en.wikipedia.org/wiki/Collective_operation). -DFIR provides abstractions for implementing individual transducers to handle inputs, computing, and emitting outputs in +DFIR provides abstractions for implementing individual processes to handle inputs, computing, and emitting outputs in a general-purpose manner. -That said, it is common practice to deploy many instance of the same DFIR transducer; most distributed systems built in DFIR +That said, it is common practice to deploy many instance of the same DFIR process; most distributed systems built in DFIR therefore have some form of SPMD parallelism. (This is true of most distributed systems in general.) \ No newline at end of file diff --git a/docs/docs/dfir/quickstart/setup.md b/docs/docs/dfir/quickstart/setup.md index 29eac7a4cbc..a5150d8821d 100644 --- a/docs/docs/dfir/quickstart/setup.md +++ b/docs/docs/dfir/quickstart/setup.md @@ -83,10 +83,10 @@ simply echoes back the messages you sent it; it also implements a client to test We will replace the code in that example with our own, but it's a good idea to run it first to make sure everything is working. :::note -We call a running DFIR binary a *transducer*. +We call a running DFIR binary a *process*. ::: -Start by running a transducer for the server: +Start by running a process for the server: ```console #shell-command-next-line cargo run -- --role server @@ -94,7 +94,7 @@ Listening on 127.0.0.1: Server live! ``` -Take note of the server's port number, and in a separate terminal, start a client transducer: +Take note of the server's port number, and in a separate terminal, start a client process: ```console #shell-command-next-line cd diff --git a/docs/docs/hydro/dataflow-programming.mdx b/docs/docs/hydro/dataflow-programming.mdx index cbb3c69f9db..4fd51a2665a 100644 --- a/docs/docs/hydro/dataflow-programming.mdx +++ b/docs/docs/hydro/dataflow-programming.mdx @@ -5,9 +5,9 @@ sidebar_position: 1 # Dataflow Programming Hydro uses a dataflow programming model, which will be familiar if you have used APIs like Rust iterators. Instead of using RPCs or async/await to describe distributed computation, Hydro instead uses **asynchronous streams**, which represent data arriving over time. Streams can represent a series of asynchronous events (e.g. inbound network requests) or a sequence of data items. -Programs in Hydro describe how to **transform** entire collections of data using operators such as `map` (transforming elements one by one), `fold` (aggregating elements into a single value), or `join` (combining elements from multiple streams on matching keys). +Programs in Hydro describe how to **transform** streams and other collections of data using operators such as `map` (transforming elements one by one), `fold` (aggregating elements into a single value), or `join` (combining elements from multiple streams on matching keys). -If you are familiar with Spark, Flink or Pandas, you will find Hydro syntax familiar. However, note well that the semantics for asynchronous streams in Hydro differ significantly from bulk analytics systems like those above. In particular, Hydro uses the type system to distinguish between bounded streams (originating from finite data) and unbounded streams (originated from asynchronous input). Moreover, Hydro is designed to handle asynchronous streams of small, independent events very efficiently. +If you are familiar with Spark, Flink or Pandas, you will find Hydro syntax familiar. However, note well that the semantics for asynchronous streams in Hydro differ significantly from bulk analytics systems like those above. In particular, Hydro uses the type system to distinguish between bounded streams (originating from finite data collections) and unbounded streams (originated from asynchronous input). Moreover, Hydro is designed to handle asynchronous streams of small, independent events very efficiently. The core of the Hydro API are the following types and their methods: - [Stream](https://hydro.run/rustdoc/hydro_lang/stream/struct.Stream) diff --git a/docs/docs/hydro/index.mdx b/docs/docs/hydro/index.mdx index 777e98fe233..5cb770efceb 100644 --- a/docs/docs/hydro/index.mdx +++ b/docs/docs/hydro/index.mdx @@ -5,7 +5,7 @@ sidebar_position: 0 # Introduction Hydro is a high-level distributed programming framework for Rust powered by the [DFIR runtime](../dfir/index.mdx). Unlike traditional architectures such as actors or RPCs, Hydro offers _choreographic_ APIs, where expressions and functions can describe computation that takes place across many locations. It also integrates with [Hydro Deploy](../deploy/index.md) to make it easy to deploy and run Hydro programs to the cloud. -Hydro uses a two-stage compilation approach. Hydro programs are standard Rust programs, which first run on the developer's laptop to generate a _deployment plan_. This plan is then compiled to individual binaries for each machine in the distributed system (enabling zero-overhead abstractions), and are then deployed to the cloud using the generated plan along with specifications of cloud resources. +Hydro uses a two-stage compilation approach. Hydro programs are standard Rust programs, which first run on the developer's laptop to generate a _deployment plan_. This plan is then compiled to individual binaries for each machine in the distributed system (enabling zero-overhead abstractions), which are then deployed to the cloud using the generated plan along with specifications of cloud resources. Hydro has been used to write a variety of high-performance distributed systems, including implementations of classic distributed protocols such as two-phase commit and Paxos. Work is ongoing to develop a distributed systems standard library that will offer these protocols and more as reusable components. diff --git a/docs/docs/hydro/live-collections/determinism.md b/docs/docs/hydro/live-collections/determinism.md index f632f777214..32fdee95d58 100644 --- a/docs/docs/hydro/live-collections/determinism.md +++ b/docs/docs/hydro/live-collections/determinism.md @@ -20,7 +20,7 @@ Much existing literature in distributed systems focuses on consistency levels su ::: ## Unsafe Operations in Hydro -All **safe** APIs in Hydro (the ones you can call regularly in Rust), guarantee determinism. But oftentimes it is necessary to do something non-deterministic, like generate events at a fixed time interval or split an input into arbitrarily sized batches. +All **safe** APIs in Hydro (the ones you can call regularly in Rust) guarantee determinism. But often it is necessary to do something non-deterministic, like generate events at a fixed wall-clock-time interval, or split an input into arbitrarily sized batches. Hydro offers APIs for such concepts behind an **`unsafe`** guard. This keyword is typically used to mark Rust functions that may not be memory-safe, but we reuse this in Hydro to mark non-deterministic APIs. @@ -41,7 +41,7 @@ unsafe { When writing a function with Hydro that involves `unsafe` code, it is important to be extra careful about whether the non-determinism is exposed externally. In some applications, a utility function may involve local non-determinism (such as sending retries), but not expose it outside the function (via deduplication). -But other utilities may expose the non-determinism, in which case they should be marked `unsafe` as well. If the function is public, Rust will require you to put a `# Safety` section in its documentation explain the non-determinism. +But other utilities may expose the non-determinism, in which case they should be marked `unsafe` as well. If the function is public, Rust will require you to put a `# Safety` section in its documentation to explain the non-determinism. ```rust # use hydro_lang::*; @@ -65,12 +65,12 @@ unsafe fn print_samples( ``` ## User-Defined Functions -Another source of potential non-determinism is user-defined functions, such as those provided to `map` or `filter`. Hydro allows for arbitrary Rust functions to be called inside these closures, so it is possible to introduce non-determinism which will not be checked by the compiler. +Another source of potential non-determinism is user-defined functions, such as those provided to `map` or `filter`. Hydro allows for arbitrary Rust functions to be called inside these closures, so it is possible to introduce non-determinism that will not be checked by the compiler. In general, avoid using APIs like random number generators inside transformation functions unless that non-determinism is explicitly documented somewhere. :::info -To help avoid such bugs, we are working on ways to use formal verification tools (such as [Kani](https://model-checking.github.io/kani/)) to check arbitrary Rust code for properties such as determinism and more. But this remains active research for now and is not yet available. +To help avoid such bugs, we are working on ways to use formal verification tools (such as [Kani](https://model-checking.github.io/kani/)) to check arbitrary Rust code for properties such as determinism and more. This remains active research for now and is not yet available. ::: diff --git a/docs/docs/hydro/quickstart/clusters.mdx b/docs/docs/hydro/quickstart/clusters.mdx index 52869db3332..17772809b74 100644 --- a/docs/docs/hydro/quickstart/clusters.mdx +++ b/docs/docs/hydro/quickstart/clusters.mdx @@ -12,13 +12,13 @@ So far, we have looked at distributed systems where each process is running a di ## Dataflow with Clusters Just like we use the `Process` type to represent a virtual handle to a single node, we can use the **`Cluster`** type to represent a handle to a **set of nodes** (with size unknown at compile-time). -A `Stream` located on a `Cluster` can be thought of as SIMD-style programming, where each cluster member executes the same operators but on different pieces of data. +A `Stream` located on a `Cluster` can be thought of as [SPMD](https://en.wikipedia.org/wiki/Single_program,_multiple_data)-style programming, where each cluster member executes the same operators but on different streams of data. To start, we set up a new module in `src/first_ten_cluster.rs` with a dataflow program that takes in a `Process` for a leader and `Cluster` for a set of workers. {getLines(firstTenClusterSrc, 1, 6)} -We start by materializing a stream of numbers on the `leader`, as before. But rather than sending the stream to a single process, we will instead _distribute_ the data to each member of the cluster using `round_robin_bincode`. This API places data on a `cluster` in a round-robin fashion by using the order of elements to determine which cluster member each element is sent to. +We start by materializing a stream of numbers on the `leader`, as before. But rather than sending the stream to a single process, we will instead _distribute_ the data to each member of the cluster using `round_robin_bincode`. This API partitions (shards) data across a `cluster` in a round-robin fashion by using the order of elements to determine which cluster member each element is sent to. :::info @@ -32,12 +32,12 @@ On each cluster member, we will then do some work to transform the data (using ` {getLines(firstTenClusterSrc, 10, 11)} -Finally, we will send the data back to the leader. We achieve this using a variant of the APIs from before: `send_bincode_interleaved`. If we used `send_bincode`, we would get a stream of (cluster ID, data) tuples. Since it is a common pattern to ignore the IDs, `send_bincode_interleaved` is available as a helper. +Finally, we will send the data back to the leader. We achieve this using a variant of the APIs from before: `send_bincode_interleaved`. If we used `send_bincode`, we would get a stream of `(cluster ID, data)` tuples. Since it is a common pattern to ignore the IDs, `send_bincode_interleaved` is available as a helper. {getLines(firstTenClusterSrc, 12, 14)} ## Deploying Clusters -Deployment scripts are similar to before, except that when provisioning a cluster we provide a list of deployment hosts rather than a single one. In our example, we'll launch 4 nodes for the cluster by creating a `Vec` of 4 localhost instances. +Deployment scripts are similar to before, except that when provisioning a cluster we provide a *list* of deployment hosts rather than a single one. In our example, we'll launch 4 nodes for the cluster by creating a `Vec` of 4 localhost instances. {highlightLines(firstTenClusterExample, [14])} @@ -70,4 +70,4 @@ cargo run --example first_ten_cluster [hydro_template::first_ten_cluster::Leader (process 0)] 14 ``` -You'll notice the round-robin distribution in action here, as each cluster log is tagged with the ID of the member (e.g. `/ 0`). In our deployment, we are sending data round-robin across 4 members of the cluster, numbered `0` through `3`. Hence cluster member `0` receives values `0`, `4`, `8` (corresponding to the highlighted lines), member `1` receives values `1`, `5`, `9`, and so on. +You'll notice the round-robin distribution in action here, as each cluster log is tagged with the ID of the member (e.g. `/ 0`). In our deployment, we are sending the even numbers round-robin across 4 members of the cluster, numbered `0` through `3`. Hence cluster member `0` receives values `0`, `8`, `16` (corresponding to the highlighted lines), member `1` receives values `2`, `10`, `18`, and so on. diff --git a/docs/docs/hydro/quickstart/first-dataflow.mdx b/docs/docs/hydro/quickstart/first-dataflow.mdx index 286c4879d9f..7cda095dd41 100644 --- a/docs/docs/hydro/quickstart/first-dataflow.mdx +++ b/docs/docs/hydro/quickstart/first-dataflow.mdx @@ -24,7 +24,7 @@ cargo generate gh:hydro-project/hydro-template ::: ## Writing a Dataflow -In Hydro, streams are attached to a **location**, which is either a virtual handle to a **single machine** (the **`Process`** type) or **set of machines** (the **`Cluster`** type). A single piece of Hydro code can describe a distributed program that runs across multiple processes and clusters, each with their own local state and data. +In Hydro, streams are attached to a **location**, which is a virtual handle to either a **single machine** (the **`Process`** type) or **set of machines** (the **`Cluster`** type). A single piece of Hydro code can describe a distributed program that runs across multiple processes and clusters, each with their own local state and data. We'll write our first dataflow in `src/first_ten.rs`. This program will run on a single machine, so we take a single `&Process` parameter. We can materialize a stream on this machine using `process.source_iter` (which emits values from a static in-memory collection), and then print out the values using `for_each`. @@ -46,13 +46,13 @@ To run a Hydro program, we need to write some deployment configuration in `examp :::tip -When using Hydro, we will *always* place our deployment scripts in the `examples` directory. This is required because deployment is done via [Hydro Deploy](../../deploy/index.md) which is a _dev dependency_---i.e. not part of the dependencies used for generating binaries (but available to programs in the `examples` directory). +When using Hydro, we will *always* place our deployment scripts in the `examples` directory. This is required because deployment is done via [Hydro Deploy](../../deploy/index.md) which is a _dev dependency_---i.e. not part of the dependencies used for generating binaries, but available to programs in the `examples` directory. ::: {firstTenExampleSrc} -First, we initialize a new [Hydro Deploy](../../deploy/index.md) deployment with `Deployment::new()`. Then, we create a `FlowBuilder` which will store the entire dataflow program and manage its compilation. +First, we initialize a new [Hydro Deploy](../../deploy/index.md) deployment with `Deployment::new()`. Then, we create a `FlowBuilder` that will store the entire dataflow program and manage its compilation. To create a `Process`, we call `flow.process()`. After the dataflow has been created (by invoking the `hydro_template::first_ten::first_ten` function we created earlier), we must map each instantiated `Process` to a deployment target using `flow.with_process` (in this case we deploy to localhost). diff --git a/docs/docs/hydro/stageleft.mdx b/docs/docs/hydro/stageleft.mdx index a4d50c394b2..1027df8f2c1 100644 --- a/docs/docs/hydro/stageleft.mdx +++ b/docs/docs/hydro/stageleft.mdx @@ -7,4 +7,11 @@ import StageleftDocs from '../../../stageleft/README.md' Under the hood, Hydro uses a library called Stageleft to power the `q!` macro and code generation logic. The following docs, from the Stageleft README, outline the core architecture of Stageleft. +:::note + +Hydro programmers need not understand how to use Stageleft; this information is provided here for readers +interested in how its `q!` macro works. + +::: + diff --git a/docs/package-lock.json b/docs/package-lock.json index 046e857f96d..3901bb33654 100644 --- a/docs/package-lock.json +++ b/docs/package-lock.json @@ -7262,9 +7262,10 @@ } }, "node_modules/express": { - "version": "4.21.1", - "resolved": "https://registry.npmjs.org/express/-/express-4.21.1.tgz", - "integrity": "sha512-YSFlK1Ee0/GC8QaO91tHcDxJiE/X4FbpAyQWkxAvG6AXCuR65YzK8ua6D9hvi/TzUfZMpc+BwuM1IPw8fmQBiQ==", + "version": "4.21.2", + "resolved": "https://registry.npmjs.org/express/-/express-4.21.2.tgz", + "integrity": "sha512-28HqgMZAmih1Czt9ny7qr6ek2qddF4FclbMzwhCREB6OFfH+rXAnuNCwo1/wFvrtbgsQDb4kSbX9de9lFbrXnA==", + "license": "MIT", "dependencies": { "accepts": "~1.3.8", "array-flatten": "1.1.1", @@ -7285,7 +7286,7 @@ "methods": "~1.1.2", "on-finished": "2.4.1", "parseurl": "~1.3.3", - "path-to-regexp": "0.1.10", + "path-to-regexp": "0.1.12", "proxy-addr": "~2.0.7", "qs": "6.13.0", "range-parser": "~1.2.1", @@ -7300,6 +7301,10 @@ }, "engines": { "node": ">= 0.10.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/express" } }, "node_modules/express/node_modules/content-disposition": { @@ -7327,9 +7332,10 @@ "integrity": "sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A==" }, "node_modules/express/node_modules/path-to-regexp": { - "version": "0.1.10", - "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-0.1.10.tgz", - "integrity": "sha512-7lf7qcQidTku0Gu3YDPc8DJ1q7OOucfa/BSsIwjuh56VU7katFvuM8hULfkwB3Fns/rsVF7PwPKVw1sl5KQS9w==" + "version": "0.1.12", + "resolved": "https://registry.npmjs.org/path-to-regexp/-/path-to-regexp-0.1.12.tgz", + "integrity": "sha512-RA1GjUVMnvYFxuqovrEqZoxxW5NUZqbwKtYz/Tt7nXerk0LbLblQmrsgdeOxV5SFHf0UDggjS/bSeOZwt1pmEQ==", + "license": "MIT" }, "node_modules/express/node_modules/range-parser": { "version": "1.2.1", @@ -11987,15 +11993,16 @@ } }, "node_modules/nanoid": { - "version": "3.3.7", - "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.7.tgz", - "integrity": "sha512-eSRppjcPIatRIMC1U6UngP8XFcz8MQWGQdt1MTBQ7NaAmvXDfvNxbvWV3x2y6CdEUciCSsDHDQZbhYaB8QEo2g==", + "version": "3.3.8", + "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.8.tgz", + "integrity": "sha512-WNLf5Sd8oZxOm+TzppcYk8gVOgP+l58xNy58D0nbUnOxOWRWvlcCV4kUF7ltmI6PsrLl/BgKEyS4mqsGChFN0w==", "funding": [ { "type": "github", "url": "https://github.com/sponsors/ai" } ], + "license": "MIT", "bin": { "nanoid": "bin/nanoid.cjs" }, From f8313b018f6a1101935e4c06abbe5af3aafb400c Mon Sep 17 00:00:00 2001 From: Mingwei Samuel Date: Wed, 29 Jan 2025 11:22:04 -0800 Subject: [PATCH 14/29] docs: fix broken links, fix #1613 (#1686) --- dfir_datalog/src/lib.rs | 2 +- dfir_lang/src/graph/ops/lattice_bimorphism.rs | 2 +- dfir_lang/src/graph/ops/reduce_keyed.rs | 2 +- dfir_macro/build.rs | 1 + dfir_rs/src/util/monotonic.rs | 4 ++-- docs/docs/dfir/architecture/index.mdx | 2 +- 6 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dfir_datalog/src/lib.rs b/dfir_datalog/src/lib.rs index 70af732d2b8..8b70153ff11 100644 --- a/dfir_datalog/src/lib.rs +++ b/dfir_datalog/src/lib.rs @@ -7,7 +7,7 @@ use quote::{quote, ToTokens}; /// /// This uses a variant of Datalog that is similar to [Dedalus](https://www2.eecs.berkeley.edu/Pubs/TechRpts/2009/EECS-2009-173.pdf). /// -/// For examples, see [the datalog tests in the Hydroflow repo](https://github.com/hydro-project/hydro/blob/main/hydroflow/tests/datalog_frontend.rs). +/// For examples, see [the datalog tests in the Hydroflow repo](https://github.com/hydro-project/hydro/blob/main/dfir_rs/tests/datalog_frontend.rs). // TODO(mingwei): rustdoc examples inline. #[proc_macro] pub fn datalog(item: proc_macro::TokenStream) -> proc_macro::TokenStream { diff --git a/dfir_lang/src/graph/ops/lattice_bimorphism.rs b/dfir_lang/src/graph/ops/lattice_bimorphism.rs index 5fa9c7334b0..bdcc1017dc2 100644 --- a/dfir_lang/src/graph/ops/lattice_bimorphism.rs +++ b/dfir_lang/src/graph/ops/lattice_bimorphism.rs @@ -6,7 +6,7 @@ use super::{ RANGE_0, RANGE_1, }; -/// An operator representing a [lattice bimorphism](https://hydro.run/docs/hydroflow/lattices_crate/lattice_math#lattice-bimorphism). +/// An operator representing a [lattice bimorphism](https://hydro.run/docs/dfir/lattices_crate/lattice_math#lattice-bimorphism). /// /// > 2 input streams, of type `LhsItem` and `RhsItem`. /// diff --git a/dfir_lang/src/graph/ops/reduce_keyed.rs b/dfir_lang/src/graph/ops/reduce_keyed.rs index 93dba0c2fa7..7ed9430bad4 100644 --- a/dfir_lang/src/graph/ops/reduce_keyed.rs +++ b/dfir_lang/src/graph/ops/reduce_keyed.rs @@ -10,7 +10,7 @@ use crate::diagnostic::{Diagnostic, Level}; /// > The output will have one tuple for each distinct `K`, with an accumulated (reduced) value of /// > type `V`. /// -/// If you need the accumulated value to have a different type than the input, use [`fold_keyed`](#keyed_fold). +/// If you need the accumulated value to have a different type than the input, use [`fold_keyed`](#fold_keyed). /// /// > Arguments: one Rust closures. The closure takes two arguments: an `&mut` 'accumulator', and /// > an element. Accumulator should be updated based on the element. diff --git a/dfir_macro/build.rs b/dfir_macro/build.rs index f710e2947de..461bdcb1543 100644 --- a/dfir_macro/build.rs +++ b/dfir_macro/build.rs @@ -226,6 +226,7 @@ fn main() { const PREFIX: &str = "\ --- sidebar_position: 4 +custom_edit_url: https://github.com/hydro-project/hydro/tree/main/dfir_lang/src/graph/ops --- # DFIR Operators diff --git a/dfir_rs/src/util/monotonic.rs b/dfir_rs/src/util/monotonic.rs index e27135bba4f..1f9e1374d4c 100644 --- a/dfir_rs/src/util/monotonic.rs +++ b/dfir_rs/src/util/monotonic.rs @@ -1,9 +1,9 @@ -/// A wrapper christening a closure as a [monotonic function](https://hydro.run/docs/hydroflow/lattices_crate/lattice_math#the-calm-theorem-and-monotonicity) +/// A wrapper christening a closure as a [monotonic function](https://hydro.run/docs/dfir/lattices_crate/lattice_math#the-calm-theorem-and-monotonicity) #[repr(transparent)] #[derive(Clone, Copy)] pub struct MonotonicFn(pub F); -/// A wrapper christening a closure as a [lattice morphism](https://hydro.run/docs/hydroflow/lattices_crate/lattice_math#lattice-morphism) +/// A wrapper christening a closure as a [lattice morphism](https://hydro.run/docs/dfir/lattices_crate/lattice_math#lattice-morphism) #[repr(transparent)] #[derive(Clone, Copy)] pub struct Morphism(pub F); diff --git a/docs/docs/dfir/architecture/index.mdx b/docs/docs/dfir/architecture/index.mdx index 7ab3b337599..68b97951043 100644 --- a/docs/docs/dfir/architecture/index.mdx +++ b/docs/docs/dfir/architecture/index.mdx @@ -14,7 +14,7 @@ here is the graph from the -The [DFIR Architecture Design Doc](https://hydro-project.github.io/hydroflow/design_docs/2021-10_architecture_design_doc.html) +The [DFIR Architecture Design Doc](https://hydro-project.github.io/hydro/design_docs/2021-10_architecture_design_doc.html) contains a more detailed explanation of this section. Note that some aspects of the design doc are not implemented (e.g. early yielding) or may become out of date as time passes. From be50e86bd70b0b451bc906de7363a9a783aeaf25 Mon Sep 17 00:00:00 2001 From: Mingwei Samuel Date: Wed, 29 Jan 2025 11:23:04 -0800 Subject: [PATCH 15/29] chore(dfir_rs): remove unused cross_join, fix #1616 (#1688) --- dfir_rs/src/compiled/pull/cross_join.rs | 128 ------------------------ dfir_rs/src/compiled/pull/mod.rs | 3 - 2 files changed, 131 deletions(-) delete mode 100644 dfir_rs/src/compiled/pull/cross_join.rs diff --git a/dfir_rs/src/compiled/pull/cross_join.rs b/dfir_rs/src/compiled/pull/cross_join.rs deleted file mode 100644 index 6b9629fa65f..00000000000 --- a/dfir_rs/src/compiled/pull/cross_join.rs +++ /dev/null @@ -1,128 +0,0 @@ -use std::ops::Range; - -pub struct CrossJoinState { - ltab: Vec, - rtab: Vec, - draw_from_left: bool, - opposite_ix: Range, -} - -impl Default for CrossJoinState { - fn default() -> Self { - Self { - ltab: Vec::new(), - rtab: Vec::new(), - draw_from_left: true, - opposite_ix: 0..0, - } - } -} - -pub struct CrossJoin<'a, I1, V1, I2, V2> -where - V1: Eq + Clone, - V2: Eq + Clone, - I1: Iterator, - I2: Iterator, -{ - lhs: I1, - rhs: I2, - state: &'a mut CrossJoinState, -} - -impl Iterator for CrossJoin<'_, I1, V1, I2, V2> -where - V1: Eq + Clone, - V2: Eq + Clone, - I1: Iterator, - I2: Iterator, -{ - type Item = (V1, V2); - - fn next(&mut self) -> Option { - loop { - // see if there's a match from the opposite's iterator - if let Some(i) = self.state.opposite_ix.next() { - if self.state.draw_from_left { - let l = self.state.ltab.last().unwrap().clone(); - let r = self.state.rtab.get(i).unwrap().clone(); - return Some((l, r)); - } else { - let l = self.state.ltab.get(i).unwrap().clone(); - let r = self.state.rtab.last().unwrap().clone(); - return Some((l, r)); - } - } - // else fetch a new tuple, alternating the sides we fetch from, - // so we draw from each input at the same rate. - let mut found_new = false; - for _i in ["opposite", "same"] { - // toggle sides - self.state.draw_from_left = !self.state.draw_from_left; - - // try to fetch from the specified side - #[expect(clippy::collapsible_else_if, reason = "code symmetry")] - if self.state.draw_from_left { - if let Some(l) = self.lhs.next() { - self.state.draw_from_left = true; - self.state.ltab.push(l); - self.state.opposite_ix = 0..self.state.rtab.len(); - found_new = true; - break; - } - } else { - if let Some(r) = self.rhs.next() { - self.state.draw_from_left = false; - self.state.rtab.push(r); - self.state.opposite_ix = 0..self.state.ltab.len(); - found_new = true; - break; - } - } - } - if !found_new { - return None; - } - } - } -} -impl<'a, I1, V1, I2, V2> CrossJoin<'a, I1, V1, I2, V2> -where - V1: Eq + Clone, - V2: Eq + Clone, - I1: Iterator, - I2: Iterator, -{ - pub fn new(lhs: I1, rhs: I2, state: &'a mut CrossJoinState) -> Self { - Self { lhs, rhs, state } - } -} - -#[cfg(test)] -mod tests { - use super::{CrossJoin, CrossJoinState}; - - #[test] - fn cross_join() { - let lhs = (0..3).map(|x| (format!("left {}", x))); - let rhs = (10..13).map(|x| (format!("right {}", x))); - - let mut state = CrossJoinState::default(); - let join = CrossJoin::new(lhs, rhs, &mut state); - - assert_eq!( - join.collect::>(), - vec![ - ("left 0".into(), "right 10".into()), - ("left 0".into(), "right 11".into()), - ("left 1".into(), "right 10".into()), - ("left 1".into(), "right 11".into()), - ("left 0".into(), "right 12".into()), - ("left 1".into(), "right 12".into()), - ("left 2".into(), "right 10".into()), - ("left 2".into(), "right 11".into()), - ("left 2".into(), "right 12".into()) - ] - ); - } -} diff --git a/dfir_rs/src/compiled/pull/mod.rs b/dfir_rs/src/compiled/pull/mod.rs index 2872fb8ab0c..3f6e6e5ec0a 100644 --- a/dfir_rs/src/compiled/pull/mod.rs +++ b/dfir_rs/src/compiled/pull/mod.rs @@ -1,9 +1,6 @@ //! Pull-based operator helpers, i.e. [`Iterator`] helpers. #![allow(missing_docs, reason = "// TODO(mingwei)")] -mod cross_join; -pub use cross_join::*; - mod symmetric_hash_join; pub use symmetric_hash_join::*; From 19784f5bef45a823549bb9084d0f51a2b7ce0981 Mon Sep 17 00:00:00 2001 From: Mingwei Samuel Date: Wed, 29 Jan 2025 11:23:34 -0800 Subject: [PATCH 16/29] docs: fix extraneous `\<` escaping introduced in #1558, fix #1614 (#1690) Previous code also inserted `\<` into code blocks. This fixes the original issue of unescaped `<`s by ensuring all op docs have them in `code blocks`, removes the escaping. --- dfir_lang/src/graph/ops/join.rs | 2 +- dfir_lang/src/graph/ops/join_fused.rs | 2 +- dfir_lang/src/graph/ops/join_multiset.rs | 2 +- dfir_macro/build.rs | 16 ++++------------ 4 files changed, 7 insertions(+), 15 deletions(-) diff --git a/dfir_lang/src/graph/ops/join.rs b/dfir_lang/src/graph/ops/join.rs index 9b80a6631cf..4a08b9ee201 100644 --- a/dfir_lang/src/graph/ops/join.rs +++ b/dfir_lang/src/graph/ops/join.rs @@ -7,7 +7,7 @@ use super::{ }; use crate::diagnostic::{Diagnostic, Level}; -/// > 2 input streams of type <(K, V1)> and <(K, V2)>, 1 output stream of type <(K, (V1, V2))> +/// > 2 input streams of type `<(K, V1)>` and `<(K, V2)>`, 1 output stream of type `<(K, (V1, V2))>` /// /// Forms the equijoin of the tuples in the input streams by their first (key) attribute. Note that the result nests the 2nd input field (values) into a tuple in the 2nd output field. /// diff --git a/dfir_lang/src/graph/ops/join_fused.rs b/dfir_lang/src/graph/ops/join_fused.rs index 519f8a83371..688125840e0 100644 --- a/dfir_lang/src/graph/ops/join_fused.rs +++ b/dfir_lang/src/graph/ops/join_fused.rs @@ -9,7 +9,7 @@ use super::{ }; use crate::diagnostic::{Diagnostic, Level}; -/// > 2 input streams of type <(K, V1)> and <(K, V2)>, 1 output stream of type <(K, (V1, V2))> +/// > 2 input streams of type `<(K, V1)>` and `<(K, V2)>`, 1 output stream of type `<(K, (V1, V2))>` /// /// `join_fused` takes two arguments, they are the configuration options for the left hand side and right hand side inputs respectively. /// There are three available configuration options, they are `Reduce`: if the input type is the same as the accumulator type, diff --git a/dfir_lang/src/graph/ops/join_multiset.rs b/dfir_lang/src/graph/ops/join_multiset.rs index 961cf60b9c0..6089d50b7a2 100644 --- a/dfir_lang/src/graph/ops/join_multiset.rs +++ b/dfir_lang/src/graph/ops/join_multiset.rs @@ -5,7 +5,7 @@ use super::{ WriteContextArgs, RANGE_0, RANGE_1, }; -/// > 2 input streams of type <(K, V1)> and <(K, V2)>, 1 output stream of type <(K, (V1, V2))> +/// > 2 input streams of type `<(K, V1)>` and `<(K, V2)>`, 1 output stream of type `<(K, (V1, V2))>` /// /// This operator is equivalent to `join` except that the LHS and RHS are collected into multisets rather than sets before joining. /// diff --git a/dfir_macro/build.rs b/dfir_macro/build.rs index 461bdcb1543..51cd066d1ef 100644 --- a/dfir_macro/build.rs +++ b/dfir_macro/build.rs @@ -1,9 +1,9 @@ //! Build script to generate operator book docs. use std::env::VarError; -use std::fmt::Write as _FmtWrite; +use std::fmt::Write as _; use std::fs::File; -use std::io::{BufWriter, Read, Result, Write}; +use std::io::{BufWriter, Result, Write}; use std::path::{Path, PathBuf}; use dfir_lang::graph::ops::{PortListSpec, OPERATORS}; @@ -33,16 +33,8 @@ fn write_operator_docgen(op_name: &str, write: &mut impl Write) -> Result<()> { "../docs/docgen", &*format!("{}.md", op_name), ]); - let mut read_string = String::new(); - File::open(doctest_path)?.read_to_string(&mut read_string)?; - write!( - write, - "{}", - read_string - .split(" + :::caution It is possible to create **different** locations that still have the same type, for example: diff --git a/docs/docs/hydro/ticks-atomicity/index.md b/docs/docs/hydro/ticks-atomicity/index.md index 8f0ac9a4bfe..1abbeac3f8d 100644 --- a/docs/docs/hydro/ticks-atomicity/index.md +++ b/docs/docs/hydro/ticks-atomicity/index.md @@ -1,12 +1,14 @@ # Ticks and Atomicity By default, all live collections in Hydro are transformed **asynchronously**, which means that there may be arbitrary delays between when a live collection is updated and when downstream transformations see the updates. This is because Hydro is designed to work in a distributed setting where messages may be delayed. But for some programs, it is necessary to define local iterative loops where transformations are applied atomically; this is achieved with **ticks**. -## Loops +## Ticks In some programs, you may want to process batches or snapshots of a live collection in an iterative manner. For example, in a map-reduce program, it may be helpful to compute aggregations on small local batches of data before sending those intermediate results to a reducer. -To create such iterative loops, Hydro provides the concept of **ticks**. A **tick** captures the body of an infinite loop running locally to the machine (importantly, this means that ticks define a **logical time** which is not comparable across machines). Ticks are non-deterministically generated, so batching data into ticks is an **unsafe** operation that requires special attention. +To create and track such iterative loops, Hydro provides the concept of **ticks**. A **tick** captures the execution of the body of an infinite loop running locally to the machine (importantly, this means that ticks define a [**logical time**](https://en.wikipedia.org/wiki/Logical_clock) which is not comparable across machines). Ticks are non-deterministically generated, so batching data into ticks is an **unsafe** operation that requires special attention. ## Atomicity -In other programs, it is necessary to define an atomic section where a set of transformations are guaranteed to be executed **all at once**. For example, in a transaction processing program, it is important that the transaction is applied **before** an acknowledgment is sent to the client. +In some cases it is necessary to define an atomic section where a set of transformations are guaranteed to be **executed sequentially without interrupts**. For example, in a transaction processing program, it is important that the transaction is applied **before** an acknowledgment is sent to the client. -In Hydro, this can be achieved by placing the transaction and acknowledgment in the same atomic **tick**. Hydro guarantees that all the outputs of a tick will be computed before any are released. Importantly, atomic ticks cannot span several locations, since that would require a locking mechanism that has significant performance implications. +In Hydro, this can be achieved by placing the transaction and acknowledgment in the same atomic **tick**. Hydro guarantees that all the outputs of a tick will be computed before any are released. Importantly, Hydro's built-in atomic ticks cannot span multiple locations.Distributed atomicity requires distributed coordination protocols (e.g. two-phase commit) that can be built in Hydro, but which have significant performance implications and are not provided by default. + + From 23444540d385075c86245ad1eb4dd78944d33fbd Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Fri, 31 Jan 2025 10:05:42 -0800 Subject: [PATCH 23/29] docs: add links to Rustdoc for streams and singletons/optionals --- docs/docs/hydro/live-collections/singletons-optionals.md | 2 ++ docs/docs/hydro/live-collections/streams.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/docs/hydro/live-collections/singletons-optionals.md b/docs/docs/hydro/live-collections/singletons-optionals.md index 1679e6f536f..4e8859e569d 100644 --- a/docs/docs/hydro/live-collections/singletons-optionals.md +++ b/docs/docs/hydro/live-collections/singletons-optionals.md @@ -3,6 +3,8 @@ sidebar_position: 3 --- # Singletons and Optionals +You can view the full API documentation for Singletons [here](pathname:///rustdoc/hydro_lang/singleton/struct.Singleton) and Optionals [here](pathname:///rustdoc/hydro_lang/optional/struct.Optional). + :::caution The Hydro documentation is currently under active development! This page is a placeholder for future content. diff --git a/docs/docs/hydro/live-collections/streams.md b/docs/docs/hydro/live-collections/streams.md index 599999e0d88..2a379beafaf 100644 --- a/docs/docs/hydro/live-collections/streams.md +++ b/docs/docs/hydro/live-collections/streams.md @@ -3,6 +3,8 @@ sidebar_position: 2 --- # Streams +You can view the full API documentation for Streams [here](pathname:///rustdoc/hydro_lang/stream/struct.Stream). + :::caution The Hydro documentation is currently under active development! This page is a placeholder for future content. From f835528c8047f5274eaa401d90358c8174aa9a05 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Fri, 31 Jan 2025 10:14:36 -0800 Subject: [PATCH 24/29] docs: add demo video --- docs/docs/hydro/index.mdx | 17 +++++++ docs/src/pages/index.js | 104 +++++++++++++++++++++++++++----------- 2 files changed, 91 insertions(+), 30 deletions(-) diff --git a/docs/docs/hydro/index.mdx b/docs/docs/hydro/index.mdx index 8b571ffa7c7..26e04149a17 100644 --- a/docs/docs/hydro/index.mdx +++ b/docs/docs/hydro/index.mdx @@ -11,6 +11,23 @@ Hydro uses a two-stage compilation approach. Hydro programs are standard Rust pr Hydro has been used to write a variety of high-performance distributed systems, including implementations of classic distributed protocols such as two-phase commit and Paxos. Work is ongoing to develop a distributed systems standard library that will offer these protocols and more as reusable components. + + :::caution The docs for Hydro are still a work in progress. If you have any questions or run into bugs, please file an issue on the [Hydro GitHub repository](https://github.com/hydro-project/hydro). diff --git a/docs/src/pages/index.js b/docs/src/pages/index.js index 3223a4febf9..4b8cb5c7378 100644 --- a/docs/src/pages/index.js +++ b/docs/src/pages/index.js @@ -1,46 +1,90 @@ -import React from 'react'; -import Link from '@docusaurus/Link'; -import Layout from '@theme/Layout'; +import React from "react"; +import Link from "@docusaurus/Link"; +import Layout from "@theme/Layout"; -import styles from './index.module.css'; +import styles from "./index.module.css"; export default function Home() { return (
- Hydro Logo -

build for every scale

+ Hydro Logo +

+ build for{" "} + every scale +

-

The Hydro Project at UC Berkeley is developing cloud-native programming models that allow anyone to develop scalable and resilient distributed applications. Our research spans across databases, distributed systems, and programming languages to deliver a modern, end-to-end stack for cloud programming.

-
- Get Started +

+ The Hydro Project at UC Berkeley is developing cloud-native{" "} + programming models that allow anyone to develop{" "} + scalable and resilient distributed applications. Our + research spans across{" "} + databases, distributed systems, and programming languages{" "} + to deliver a modern, end-to-end stack for cloud programming. +

+
+ + Get Started + - + Latest Research
+ +
+ +
From 4fd7c453b8f1af7fdae66c209fc9ac904fb071bc Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Fri, 31 Jan 2025 10:48:01 -0800 Subject: [PATCH 25/29] docs: put Hydro above DFIR in the footer --- docs/docusaurus.config.js | 209 +++++++++++++++++++------------------- 1 file changed, 103 insertions(+), 106 deletions(-) diff --git a/docs/docusaurus.config.js b/docs/docusaurus.config.js index 6ffe3f4dac3..bee66e3b2c4 100644 --- a/docs/docusaurus.config.js +++ b/docs/docusaurus.config.js @@ -1,73 +1,70 @@ // @ts-check // Note: type annotations allow type checking and IDEs autocompletion -const lightCodeTheme = require('prism-react-renderer').themes.github; -const darkCodeTheme = require('prism-react-renderer').themes.dracula; +const lightCodeTheme = require("prism-react-renderer").themes.github; +const darkCodeTheme = require("prism-react-renderer").themes.dracula; -const math = require('remark-math'); -const katex = require('rehype-katex'); +const math = require("remark-math"); +const katex = require("rehype-katex"); /** @type {import('@docusaurus/types').Config} */ const config = { - title: 'Hydro - Build for Every Scale', - tagline: 'Dinosaurs are cool', - favicon: 'img/favicon.ico', + title: "Hydro - Build for Every Scale", + tagline: "Dinosaurs are cool", + favicon: "img/favicon.ico", // Set the production url of your site here - url: 'https://hydro.run', + url: "https://hydro.run", // Set the // pathname under which your site is served // For GitHub pages deployment, it is often '//' - baseUrl: '/', + baseUrl: "/", // GitHub pages deployment config. // If you aren't using GitHub pages, you don't need these. - organizationName: 'hydro-project', // Usually your GitHub org/user name. - projectName: 'hydroflow', // Usually your repo name. + organizationName: "hydro-project", // Usually your GitHub org/user name. + projectName: "hydroflow", // Usually your repo name. - onBrokenLinks: 'throw', - onBrokenMarkdownLinks: 'throw', + onBrokenLinks: "throw", + onBrokenMarkdownLinks: "throw", customFields: { - 'LOAD_PLAYGROUND': process.env.LOAD_PLAYGROUND || false, + LOAD_PLAYGROUND: process.env.LOAD_PLAYGROUND || false, }, markdown: { - mermaid: true + mermaid: true, }, - themes: [ - '@docusaurus/theme-mermaid' - ], + themes: ["@docusaurus/theme-mermaid"], // Even if you don't use internalization, you can use this field to set useful // metadata like html lang. For example, if your site is Chinese, you may want // to replace "en" with "zh-Hans". i18n: { - defaultLocale: 'en', - locales: ['en'], + defaultLocale: "en", + locales: ["en"], }, stylesheets: [ { - href: 'https://cdn.jsdelivr.net/npm/katex@0.13.24/dist/katex.min.css', - type: 'text/css', + href: "https://cdn.jsdelivr.net/npm/katex@0.13.24/dist/katex.min.css", + type: "text/css", integrity: - 'sha384-odtC+0UGzzFL/6PNoE8rX/SPcQDXBJ+uRepguP4QkPCm2LBxH3FA3y+fKSiJ+AmM', - crossorigin: 'anonymous', + "sha384-odtC+0UGzzFL/6PNoE8rX/SPcQDXBJ+uRepguP4QkPCm2LBxH3FA3y+fKSiJ+AmM", + crossorigin: "anonymous", }, ], presets: [ [ - 'classic', + "classic", /** @type {import('@docusaurus/preset-classic').Options} */ ({ docs: { - sidebarPath: require.resolve('./sidebars.js'), + sidebarPath: require.resolve("./sidebars.js"), // Please change this to your repo. // Remove this to remove the "edit this page" links. - editUrl: - 'https://github.com/hydro-project/hydro/tree/main/docs/', + editUrl: "https://github.com/hydro-project/hydro/tree/main/docs/", remarkPlugins: [math], rehypePlugins: [katex], }, @@ -79,7 +76,7 @@ const config = { // 'https://github.com/hydro-project/hydro/tree/main/docs/', // }, theme: { - customCss: require.resolve('./src/css/custom.css'), + customCss: require.resolve("./src/css/custom.css"), }, }), ], @@ -87,7 +84,7 @@ const config = { plugins: [ [ - '@docusaurus/plugin-ideal-image', + "@docusaurus/plugin-ideal-image", { quality: 75, max: 1080, @@ -96,115 +93,115 @@ const config = { disableInDev: false, }, ], - require.resolve("./wasm-plugin.js") + require.resolve("./wasm-plugin.js"), ], themeConfig: /** @type {import('@docusaurus/preset-classic').ThemeConfig} */ ({ // Replace with your project's social card - image: 'img/social-card.png', + image: "img/social-card.png", colorMode: { respectPrefersColorScheme: true, }, navbar: { - title: 'Hydro', + title: "Hydro", logo: { - alt: 'Hydro', - src: 'img/hydro-logo.svg', + alt: "Hydro", + src: "img/hydro-logo.svg", }, items: [ { - type: 'dropdown', - label: 'Docs', + type: "dropdown", + label: "Docs", items: [ { - type: 'docSidebar', - sidebarId: 'hydroSidebar', - label: 'Hydro', + type: "docSidebar", + sidebarId: "hydroSidebar", + label: "Hydro", }, { - type: 'docSidebar', - sidebarId: 'deploySidebar', - label: 'Hydro Deploy', + type: "docSidebar", + sidebarId: "deploySidebar", + label: "Hydro Deploy", }, { - href: 'pathname:///rustdoc/hydro_lang/', - label: 'Rustdoc', + href: "pathname:///rustdoc/hydro_lang/", + label: "Rustdoc", }, { - type: 'docSidebar', - sidebarId: 'dfirSidebar', - label: 'DFIR', + type: "docSidebar", + sidebarId: "dfirSidebar", + label: "DFIR", }, - ] + ], }, { - to: '/playground', - position: 'left', - label: 'Playground', + to: "/playground", + position: "left", + label: "Playground", }, { - to: '/research', - position: 'left', - label: 'Publications', + to: "/research", + position: "left", + label: "Publications", }, { - to: '/people', - position: 'left', - label: 'People', + to: "/people", + position: "left", + label: "People", }, // {to: '/blog', label: 'Blog', position: 'left'}, { - href: 'https://github.com/hydro-project/hydro', - label: 'GitHub', - position: 'right', + href: "https://github.com/hydro-project/hydro", + label: "GitHub", + position: "right", }, ], }, footer: { - style: 'dark', + style: "dark", links: [ { - title: 'Docs', + title: "Docs", items: [ { - label: 'DFIR', - to: '/docs/dfir/', + label: "Hydro", + to: "/docs/hydro/", }, { - label: 'Hydro', - to: '/docs/hydro/', + label: "DFIR", + to: "/docs/dfir/", }, { - label: 'Hydro Deploy', - to: '/docs/deploy/', - } + label: "Hydro Deploy", + to: "/docs/deploy/", + }, ], }, { - title: 'Research Group', + title: "Research Group", items: [ { - label: 'Publications', - to: '/research', + label: "Publications", + to: "/research", }, { - label: 'People', - to: '/people', - } + label: "People", + to: "/people", + }, ], }, { - title: 'More', + title: "More", items: [ // { // label: 'Blog', // to: '/blog', // }, { - label: 'GitHub', - href: 'https://github.com/hydro-project/hydro', + label: "GitHub", + href: "https://github.com/hydro-project/hydro", }, ], }, @@ -214,41 +211,41 @@ const config = { prism: { theme: lightCodeTheme, darkTheme: darkCodeTheme, - additionalLanguages: ['rust', 'bash'], + additionalLanguages: ["rust", "bash"], magicComments: [ { - className: 'theme-code-block-highlighted-line', - line: 'highlight-next-line', - block: {start: 'highlight-start', end: 'highlight-end'}, + className: "theme-code-block-highlighted-line", + line: "highlight-next-line", + block: { start: "highlight-start", end: "highlight-end" }, }, { - className: 'shell-command-line', - line: 'shell-command-next-line', - } - ] + className: "shell-command-line", + line: "shell-command-next-line", + }, + ], }, algolia: { - appId: 'C2TSTQAKIC', - apiKey: '38cef87035f42759bc1dd871e91e06ba', - indexName: 'hydro' + appId: "C2TSTQAKIC", + apiKey: "38cef87035f42759bc1dd871e91e06ba", + indexName: "hydro", }, }), - scripts: [ - { - id: "runllm-widget-script", - type: "module", - src: "https://widget.runllm.com", - "runllm-server-address": "https://api.runllm.com", - "runllm-assistant-id": "136", - "runllm-position": "BOTTOM_RIGHT", - "runllm-keyboard-shortcut": "Mod+j", - "runllm-preset": "docusaurus", - "runllm-slack-community-url": "", - "runllm-name": "Hydro", - "runllm-theme-color": "#005EEC", - async: true, - }, - ], + scripts: [ + { + id: "runllm-widget-script", + type: "module", + src: "https://widget.runllm.com", + "runllm-server-address": "https://api.runllm.com", + "runllm-assistant-id": "136", + "runllm-position": "BOTTOM_RIGHT", + "runllm-keyboard-shortcut": "Mod+j", + "runllm-preset": "docusaurus", + "runllm-slack-community-url": "", + "runllm-name": "Hydro", + "runllm-theme-color": "#005EEC", + async: true, + }, + ], }; module.exports = config; From 040c52901bb60515252c3cb5a469dbb9f6a30640 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Fri, 31 Jan 2025 12:20:50 -0800 Subject: [PATCH 26/29] docs: fix embed width on mobile --- docs/docs/hydro/index.mdx | 7 ++++--- docs/src/pages/index.js | 6 ++++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/docs/hydro/index.mdx b/docs/docs/hydro/index.mdx index 26e04149a17..41318f6e1ad 100644 --- a/docs/docs/hydro/index.mdx +++ b/docs/docs/hydro/index.mdx @@ -12,13 +12,14 @@ Hydro uses a two-stage compilation approach. Hydro programs are standard Rust pr Hydro has been used to write a variety of high-performance distributed systems, including implementations of classic distributed protocols such as two-phase commit and Paxos. Work is ongoing to develop a distributed systems standard library that will offer these protocols and more as reusable components.