diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index f84c08809e61..493322588c9d 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -395,7 +395,7 @@ jobs: - name: Run benchmark run: | - time cargo bench -p benches -- hydroflow --output-format bencher | tee output.txt + time cargo bench -p benches -- dfir --output-format bencher | tee output.txt time cargo bench -p benches -- micro/ops/ --output-format bencher | tee -a output.txt - name: Generate benchmark page diff --git a/.vscode/settings.json b/.vscode/settings.json index d650a5fd0e7c..b2db3df9b47d 100644 --- a/.vscode/settings.json +++ b/.vscode/settings.json @@ -13,6 +13,7 @@ } } ], + "rust-analyzer.cargo.features": ["deploy"], "editor.semanticTokenColorCustomizations": { "enabled": true, "rules": { diff --git a/Cargo.lock b/Cargo.lock index d1c902907469..bd721e82b1ef 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -846,9 +846,9 @@ dependencies = [ [[package]] name = "ctor" -version = "0.2.8" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" +checksum = "32a2785755761f3ddc1492979ce1e48d2c00d09311c39e4466429188f3dd6501" dependencies = [ "quote", "syn 2.0.75", @@ -944,6 +944,7 @@ name = "dfir_macro" version = "0.11.0" dependencies = [ "dfir_lang", + "glob", "itertools", "proc-macro-crate", "proc-macro2", @@ -1305,9 +1306,9 @@ checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" [[package]] name = "glob" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" +checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "gossip_kv" @@ -1635,6 +1636,8 @@ name = "hydro_test" version = "0.0.0" dependencies = [ "async-ssh2-lite", + "ctor", + "dfir_macro", "futures", "hydro_deploy", "hydro_lang", @@ -1893,10 +1896,11 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "js-sys" -version = "0.3.70" +version = "0.3.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1868808506b929d7b0cfa8f75951347aa71bb21144b7791bae35d9bccfcfe37a" +checksum = "1cfaf33c695fc6e08064efbc1f72ec937429614f25eef83af942d0e227c3a28f" dependencies = [ + "once_cell", "wasm-bindgen", ] @@ -3114,6 +3118,12 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "rustversion" +version = "1.0.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" + [[package]] name = "ryu" version = "1.0.18" @@ -4245,24 +4255,24 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.93" +version = "0.2.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a82edfc16a6c469f5f44dc7b571814045d60404b55a0ee849f9bcfa2e63dd9b5" +checksum = "1edc8929d7499fc4e8f0be2262a241556cfc54a0bea223790e71446f2aab1ef5" dependencies = [ "cfg-if", "once_cell", + "rustversion", "wasm-bindgen-macro", ] [[package]] name = "wasm-bindgen-backend" -version = "0.2.93" +version = "0.2.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9de396da306523044d3302746f1208fa71d7532227f15e347e2d93e4145dd77b" +checksum = "2f0a0651a5c2bc21487bde11ee802ccaf4c51935d0d3d42a6101f98161700bc6" dependencies = [ "bumpalo", "log", - "once_cell", "proc-macro2", "quote", "syn 2.0.75", @@ -4271,21 +4281,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.43" +version = "0.4.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61e9300f63a621e96ed275155c108eb6f843b6a26d053f122ab69724559dc8ed" +checksum = "555d470ec0bc3bb57890405e5d4322cc9ea83cebb085523ced7be4144dac1e61" dependencies = [ "cfg-if", "js-sys", + "once_cell", "wasm-bindgen", "web-sys", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.93" +version = "0.2.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "585c4c91a46b072c92e908d99cb1dcdf95c5218eeb6f3bf1efa991ee7a68cccf" +checksum = "7fe63fc6d09ed3792bd0897b314f53de8e16568c2b3f7982f468c0bf9bd0b407" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -4293,9 +4304,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.93" +version = "0.2.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" +checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" dependencies = [ "proc-macro2", "quote", @@ -4306,20 +4317,21 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.93" +version = "0.2.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c62a0a307cb4a311d3a07867860911ca130c3494e8c2719593806c08bc5d0484" +checksum = "1a05d73b933a847d6cccdda8f838a22ff101ad9bf93e33684f39c1f5f0eece3d" +dependencies = [ + "unicode-ident", +] [[package]] name = "wasm-bindgen-test" -version = "0.3.43" +version = "0.3.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68497a05fb21143a08a7d24fc81763384a3072ee43c44e86aad1744d6adef9d9" +checksum = "66c8d5e33ca3b6d9fa3b4676d774c5778031d27a578c2b007f905acf816152c3" dependencies = [ - "console_error_panic_hook", "js-sys", "minicov", - "scoped-tls", "wasm-bindgen", "wasm-bindgen-futures", "wasm-bindgen-test-macro", @@ -4327,9 +4339,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-test-macro" -version = "0.3.43" +version = "0.3.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b8220be1fa9e4c889b30fd207d4906657e7e90b12e0e6b0c8b8d8709f5de021" +checksum = "17d5042cc5fa009658f9a7333ef24291b1291a25b6382dd68862a7f3b969f69b" dependencies = [ "proc-macro2", "quote", @@ -4338,9 +4350,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.70" +version = "0.3.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26fdeaafd9bd129f65e7c031593c24d62186301e0c72c8978fa1678be7d532c0" +checksum = "33b6dd2ef9186f1f2072e409e99cd22a975331a6b3591b12c764e0e55c60d5d2" dependencies = [ "js-sys", "wasm-bindgen", @@ -4417,7 +4429,7 @@ version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.48.0", ] [[package]] diff --git a/benches/benches/words_diamond.rs b/benches/benches/words_diamond.rs index 82d2c279c3d1..34fd7e716774 100644 --- a/benches/benches/words_diamond.rs +++ b/benches/benches/words_diamond.rs @@ -29,10 +29,10 @@ fn hash_code(s: &str) -> u32 { .fold(0, |n, c| n.wrapping_mul(31).wrapping_add(c as u32)) } -fn hydroflow_diamond(c: &mut Criterion) { +fn dfir_rs_diamond(c: &mut Criterion) { let _ = *WORDS; - c.bench_function(name_of!(hydroflow_diamond), |b| { + c.bench_function(name_of!(dfir_rs_diamond), |b| { b.iter(|| { let words = words(); let mut df = dfir_syntax! { @@ -173,7 +173,7 @@ fn hydroflo2_diamond_iter_buffer_one(c: &mut Criterion) { criterion_group!( words_diamond, - hydroflow_diamond, + dfir_rs_diamond, hydroflo2_diamond_forloop, hydroflo2_diamond_iter_clone_chain, hydroflo2_diamond_iter_clone_interleave, diff --git a/dfir_lang/src/graph/eliminate_extra_unions_tees.rs b/dfir_lang/src/graph/eliminate_extra_unions_tees.rs index 400d120041c5..fa3a8ca786c8 100644 --- a/dfir_lang/src/graph/eliminate_extra_unions_tees.rs +++ b/dfir_lang/src/graph/eliminate_extra_unions_tees.rs @@ -13,7 +13,7 @@ fn find_unary_ops<'a>( .filter(move |&node_id| { graph .node_op_inst(node_id) - .map_or(false, |op_inst| op_name == op_inst.op_constraints.name) + .is_some_and(|op_inst| op_name == op_inst.op_constraints.name) }) .filter(|&node_id| { 1 == graph.node_degree_in(node_id) && 1 == graph.node_degree_out(node_id) diff --git a/dfir_lang/src/graph/hydroflow_graph.rs b/dfir_lang/src/graph/hydroflow_graph.rs index b3cb5c91e21c..c955f40e4707 100644 --- a/dfir_lang/src/graph/hydroflow_graph.rs +++ b/dfir_lang/src/graph/hydroflow_graph.rs @@ -718,7 +718,7 @@ impl DfirGraph { .iter() .position(|&node_id| { self.node_color(node_id) - .map_or(false, |color| Color::Pull != color) + .is_some_and(|color| Color::Pull != color) }) .unwrap_or(subgraph_nodes.len()) } diff --git a/dfir_macro/Cargo.toml b/dfir_macro/Cargo.toml index 290748ba5a1a..ac8efb48d01f 100644 --- a/dfir_macro/Cargo.toml +++ b/dfir_macro/Cargo.toml @@ -22,6 +22,7 @@ proc-macro2 = "1.0.74" proc-macro-crate = "1.0.0" quote = "1.0.35" syn = { version = "2.0.46", features = [ "parsing", "extra-traits" ] } +glob = "0.3.2" [build-dependencies] dfir_lang = { path = "../dfir_lang", version = "^0.11.0" } diff --git a/dfir_macro/src/lib.rs b/dfir_macro/src/lib.rs index cc1eb4808e16..f50d2549509e 100644 --- a/dfir_macro/src/lib.rs +++ b/dfir_macro/src/lib.rs @@ -122,19 +122,33 @@ pub fn dfir_parser(input: proc_macro::TokenStream) -> proc_macro::TokenStream { #[doc(hidden)] #[proc_macro] -pub fn surface_booktest_operators(input: proc_macro::TokenStream) -> proc_macro::TokenStream { - assert!(input.is_empty(), "Input must be empty"); - let each = dfir_lang::graph::ops::OPERATORS.iter().map(|op| { - let op_ident = Ident::new(op.name, Span::call_site()); - let op_filename = format!("../../docs/docgen/{}.md", op.name); - let lit_filename = LitStr::new(&op_filename, Span::call_site()); - quote! { - #[doc = include_str!(#lit_filename)] - mod #op_ident {} - } - }); +pub fn doctest_markdown_glob(input: proc_macro::TokenStream) -> proc_macro::TokenStream { + let current_dir = std::env::current_dir().unwrap(); + let input_glob = parse_macro_input!(input as LitStr); + let globbed_files = glob::glob(input_glob.value().as_str()) + .expect("Failed to read glob pattern") + .map(|entry| entry.expect("Failed to read glob entry")) + .map(|path| { + let path_abs = current_dir.join(path.clone()); + let path_abs_str = path_abs.to_str().expect("Failed to convert path to string"); + let file_name_without_extension = path.to_str().expect("Failed to get file stem"); + let lit = LitStr::new(path_abs_str, Span::call_site()); + let mut ident_string = file_name_without_extension + .chars() + .map(|c| if c.is_ascii_alphanumeric() { c } else { '_' }) + .collect::(); + if ident_string.chars().next().unwrap().is_ascii_digit() { + // Identifiers cannot start with a digit, prepend an underscore. + ident_string.insert(0, '_'); + } + let file_name_ident = Ident::new(&ident_string, Span::call_site()); + quote! { + #[doc = include_str!(#lit)] + mod #file_name_ident {} + } + }); let out = quote! { - #( #each )* + #( #globbed_files )* }; out.into() } diff --git a/dfir_rs/src/lib.rs b/dfir_rs/src/lib.rs index 20b32f75e6e8..96c94201864f 100644 --- a/dfir_rs/src/lib.rs +++ b/dfir_rs/src/lib.rs @@ -47,6 +47,6 @@ pub type Never = std::convert::Infallible; #[cfg(doctest)] mod booktest { mod surface_ops { - dfir_macro::surface_booktest_operators!(); + dfir_macro::doctest_markdown_glob!("docs/docgen/*.md"); } } diff --git a/dfir_rs/src/scheduled/context.rs b/dfir_rs/src/scheduled/context.rs index 982277d2b1c0..032d444bfff0 100644 --- a/dfir_rs/src/scheduled/context.rs +++ b/dfir_rs/src/scheduled/context.rs @@ -238,7 +238,8 @@ impl Default for Context { current_tick_start: SystemTime::now(), subgraph_last_tick_run_in: None, - subgraph_id: SubgraphId(0), + // Will be re-set before use. + subgraph_id: SubgraphId::from_raw(0), tasks_to_spawn: Vec::new(), task_join_handles: Vec::new(), diff --git a/dfir_rs/src/scheduled/graph.rs b/dfir_rs/src/scheduled/graph.rs index a7fd221024a5..484277543139 100644 --- a/dfir_rs/src/scheduled/graph.rs +++ b/dfir_rs/src/scheduled/graph.rs @@ -21,17 +21,18 @@ use super::port::{RecvCtx, RecvPort, SendCtx, SendPort, RECV, SEND}; use super::reactor::Reactor; use super::state::StateHandle; use super::subgraph::Subgraph; -use super::{HandoffId, SubgraphId}; +use super::{HandoffId, HandoffTag, SubgraphId, SubgraphTag}; use crate::scheduled::ticks::{TickDuration, TickInstant}; +use crate::util::slot_vec::SlotVec; use crate::Never; /// A DFIR graph. Owns, schedules, and runs the compiled subgraphs. #[derive(Default)] pub struct Dfir<'a> { - pub(super) subgraphs: Vec>, + pub(super) subgraphs: SlotVec>, pub(super) context: Context, - handoffs: Vec, + handoffs: SlotVec, #[cfg(feature = "meta")] /// See [`Self::meta_graph()`]. @@ -52,15 +53,32 @@ impl Dfir<'_> { where T: Clone, { - // Handoff ID of new tee output. - let new_hoff_id = HandoffId(self.handoffs.len()); - // If we're teeing from a child make sure to find root. - let tee_root = self.handoffs[tee_parent_port.handoff_id.0].pred_handoffs[0]; + let tee_root = self.handoffs[tee_parent_port.handoff_id].pred_handoffs[0]; // Set up teeing metadata. + let tee_root_data = &mut self.handoffs[tee_root]; + let tee_root_data_name = tee_root_data.name.clone(); + + // Insert new handoff output. + let teeing_handoff = tee_root_data + .handoff + .any_ref() + .downcast_ref::>() + .unwrap(); + let new_handoff = teeing_handoff.tee(); + + // Handoff ID of new tee output. + let new_hoff_id = self.handoffs.insert_with_key(|new_hoff_id| { + let new_name = Cow::Owned(format!("{} tee {:?}", tee_root_data_name, new_hoff_id)); + let mut new_handoff_data = HandoffData::new(new_name, new_handoff, new_hoff_id); + // Set self's predecessor as `tee_root`. + new_handoff_data.pred_handoffs = vec![tee_root]; + new_handoff_data + }); + // Go to `tee_root`'s successors and insert self (the new tee output). - let tee_root_data = &mut self.handoffs[tee_root.0]; + let tee_root_data = &mut self.handoffs[tee_root]; tee_root_data.succ_handoffs.push(new_hoff_id); // Add our new handoff id into the subgraph data if the send `tee_root` has already been @@ -70,22 +88,9 @@ impl Dfir<'_> { "Tee send side should only have one sender (or none set yet)." ); if let Some(&pred_sg_id) = tee_root_data.preds.first() { - self.subgraphs[pred_sg_id.0].succs.push(new_hoff_id); + self.subgraphs[pred_sg_id].succs.push(new_hoff_id); } - // Insert new handoff output. - let teeing_handoff = tee_root_data - .handoff - .any_ref() - .downcast_ref::>() - .unwrap(); - let new_handoff = teeing_handoff.tee(); - let new_name = Cow::Owned(format!("{} tee {:?}", tee_root_data.name, new_hoff_id)); - let mut new_handoff_data = HandoffData::new(new_name, new_handoff, new_hoff_id); - // Set self's predecessor as `tee_root`. - new_handoff_data.pred_handoffs = vec![tee_root]; - self.handoffs.push(new_handoff_data); - let output_port = RecvPort { handoff_id: new_hoff_id, _marker: PhantomData, @@ -101,7 +106,7 @@ impl Dfir<'_> { where T: Clone, { - let data = &self.handoffs[tee_port.handoff_id.0]; + let data = &self.handoffs[tee_port.handoff_id]; let teeing_handoff = data .handoff .any_ref() @@ -110,7 +115,7 @@ impl Dfir<'_> { teeing_handoff.drop(); let tee_root = data.pred_handoffs[0]; - let tee_root_data = &mut self.handoffs[tee_root.0]; + let tee_root_data = &mut self.handoffs[tee_root]; // Remove this output from the send succ handoff list. tee_root_data .succ_handoffs @@ -121,7 +126,7 @@ impl Dfir<'_> { "Tee send side should only have one sender (or none set yet)." ); if let Some(&pred_sg_id) = tee_root_data.preds.first() { - self.subgraphs[pred_sg_id.0] + self.subgraphs[pred_sg_id] .succs .retain(|&succ_hoff| succ_hoff != tee_port.handoff_id); } @@ -266,11 +271,11 @@ impl<'a> Dfir<'a> { { work_done = true; { - let sg_data = &mut self.subgraphs[sg_id.0]; + let sg_data = &mut self.subgraphs[sg_id]; // This must be true for the subgraph to be enqueued. assert!(sg_data.is_scheduled.take()); tracing::trace!( - sg_id = sg_id.0, + sg_id = sg_id.to_string(), sg_name = &*sg_data.name, "Running subgraph." ); @@ -281,12 +286,12 @@ impl<'a> Dfir<'a> { sg_data.last_tick_run_in = Some(current_tick); } - let sg_data = &self.subgraphs[sg_id.0]; + let sg_data = &self.subgraphs[sg_id]; for &handoff_id in sg_data.succs.iter() { - let handoff = &self.handoffs[handoff_id.0]; + let handoff = &self.handoffs[handoff_id]; if !handoff.handoff.is_bottom() { for &succ_id in handoff.succs.iter() { - let succ_sg_data = &self.subgraphs[succ_id.0]; + let succ_sg_data = &self.subgraphs[succ_id]; // If we have sent data to the next tick, then we can start the next tick. if succ_sg_data.stratum < self.context.current_stratum && !sg_data.is_lazy { self.context.can_start_tick = true; @@ -438,9 +443,9 @@ impl<'a> Dfir<'a> { pub fn try_recv_events(&mut self) -> usize { let mut enqueued_count = 0; while let Ok((sg_id, is_external)) = self.context.event_queue_recv.try_recv() { - let sg_data = &self.subgraphs[sg_id.0]; + let sg_data = &self.subgraphs[sg_id]; tracing::trace!( - sg_id = sg_id.0, + sg_id = sg_id.to_string(), is_external = is_external, sg_stratum = sg_data.stratum, "Event received." @@ -476,9 +481,9 @@ impl<'a> Dfir<'a> { let mut count = 0; loop { let (sg_id, is_external) = self.context.event_queue_recv.blocking_recv()?; - let sg_data = &self.subgraphs[sg_id.0]; + let sg_data = &self.subgraphs[sg_id]; tracing::trace!( - sg_id = sg_id.0, + sg_id = sg_id.to_string(), is_external = is_external, sg_stratum = sg_data.stratum, "Event received." @@ -521,9 +526,9 @@ impl<'a> Dfir<'a> { loop { tracing::trace!("Awaiting events (`event_queue_recv`)."); let (sg_id, is_external) = self.context.event_queue_recv.recv().await?; - let sg_data = &self.subgraphs[sg_id.0]; + let sg_data = &self.subgraphs[sg_id]; tracing::trace!( - sg_id = sg_id.0, + sg_id = sg_id.to_string(), is_external = is_external, sg_stratum = sg_data.stratum, "Event received." @@ -557,7 +562,7 @@ impl<'a> Dfir<'a> { /// Schedules a subgraph to be run. See also: [`Context::schedule_subgraph`]. pub fn schedule_subgraph(&mut self, sg_id: SubgraphId) -> bool { - let sg_data = &self.subgraphs[sg_id.0]; + let sg_data = &self.subgraphs[sg_id]; let already_scheduled = sg_data.is_scheduled.replace(true); if !already_scheduled { self.context.stratum_queues[sg_data.stratum].push_back(sg_id); @@ -602,26 +607,27 @@ impl<'a> Dfir<'a> { W: 'static + PortList, F: 'a + for<'ctx> FnMut(&'ctx mut Context, R::Ctx<'ctx>, W::Ctx<'ctx>), { - let sg_id = SubgraphId(self.subgraphs.len()); - - let (mut subgraph_preds, mut subgraph_succs) = Default::default(); - recv_ports.set_graph_meta(&mut self.handoffs, &mut subgraph_preds, sg_id, true); - send_ports.set_graph_meta(&mut self.handoffs, &mut subgraph_succs, sg_id, false); - - let subgraph = move |context: &mut Context, handoffs: &mut Vec| { - let recv = recv_ports.make_ctx(&*handoffs); - let send = send_ports.make_ctx(&*handoffs); - (subgraph)(context, recv, send); - }; - self.subgraphs.push(SubgraphData::new( - name.into(), - stratum, - subgraph, - subgraph_preds, - subgraph_succs, - true, - laziness, - )); + let sg_id = self.subgraphs.insert_with_key(|sg_id| { + let (mut subgraph_preds, mut subgraph_succs) = Default::default(); + recv_ports.set_graph_meta(&mut self.handoffs, &mut subgraph_preds, sg_id, true); + send_ports.set_graph_meta(&mut self.handoffs, &mut subgraph_succs, sg_id, false); + + let subgraph = + move |context: &mut Context, handoffs: &mut SlotVec| { + let recv = recv_ports.make_ctx(&*handoffs); + let send = send_ports.make_ctx(&*handoffs); + (subgraph)(context, recv, send); + }; + SubgraphData::new( + name.into(), + stratum, + subgraph, + subgraph_preds, + subgraph_succs, + true, + laziness, + ) + }); self.context.init_stratum(stratum); self.context.stratum_queues[stratum].push_back(sg_id); @@ -662,58 +668,60 @@ impl<'a> Dfir<'a> { F: 'static + for<'ctx> FnMut(&'ctx mut Context, &'ctx [&'ctx RecvCtx], &'ctx [&'ctx SendCtx]), { - let sg_id = SubgraphId(self.subgraphs.len()); + let sg_id = self.subgraphs.insert_with_key(|sg_id| { + let subgraph_preds = recv_ports.iter().map(|port| port.handoff_id).collect(); + let subgraph_succs = send_ports.iter().map(|port| port.handoff_id).collect(); - let subgraph_preds = recv_ports.iter().map(|port| port.handoff_id).collect(); - let subgraph_succs = send_ports.iter().map(|port| port.handoff_id).collect(); + for recv_port in recv_ports.iter() { + self.handoffs[recv_port.handoff_id].succs.push(sg_id); + } + for send_port in send_ports.iter() { + self.handoffs[send_port.handoff_id].preds.push(sg_id); + } - for recv_port in recv_ports.iter() { - self.handoffs[recv_port.handoff_id.0].succs.push(sg_id); - } - for send_port in send_ports.iter() { - self.handoffs[send_port.handoff_id.0].preds.push(sg_id); - } + let subgraph = + move |context: &mut Context, handoffs: &mut SlotVec| { + let recvs: Vec<&RecvCtx> = recv_ports + .iter() + .map(|hid| hid.handoff_id) + .map(|hid| handoffs.get(hid).unwrap()) + .map(|h_data| { + h_data + .handoff + .any_ref() + .downcast_ref() + .expect("Attempted to cast handoff to wrong type.") + }) + .map(RefCast::ref_cast) + .collect(); + + let sends: Vec<&SendCtx> = send_ports + .iter() + .map(|hid| hid.handoff_id) + .map(|hid| handoffs.get(hid).unwrap()) + .map(|h_data| { + h_data + .handoff + .any_ref() + .downcast_ref() + .expect("Attempted to cast handoff to wrong type.") + }) + .map(RefCast::ref_cast) + .collect(); + + (subgraph)(context, &recvs, &sends) + }; + SubgraphData::new( + name.into(), + stratum, + subgraph, + subgraph_preds, + subgraph_succs, + true, + false, + ) + }); - let subgraph = move |context: &mut Context, handoffs: &mut Vec| { - let recvs: Vec<&RecvCtx> = recv_ports - .iter() - .map(|hid| hid.handoff_id) - .map(|hid| handoffs.get(hid.0).unwrap()) - .map(|h_data| { - h_data - .handoff - .any_ref() - .downcast_ref() - .expect("Attempted to cast handoff to wrong type.") - }) - .map(RefCast::ref_cast) - .collect(); - - let sends: Vec<&SendCtx> = send_ports - .iter() - .map(|hid| hid.handoff_id) - .map(|hid| handoffs.get(hid.0).unwrap()) - .map(|h_data| { - h_data - .handoff - .any_ref() - .downcast_ref() - .expect("Attempted to cast handoff to wrong type.") - }) - .map(RefCast::ref_cast) - .collect(); - - (subgraph)(context, &recvs, &sends) - }; - self.subgraphs.push(SubgraphData::new( - name.into(), - stratum, - subgraph, - subgraph_preds, - subgraph_succs, - true, - false, - )); self.context.init_stratum(stratum); self.context.stratum_queues[stratum].push_back(sg_id); @@ -726,12 +734,11 @@ impl<'a> Dfir<'a> { Name: Into>, H: 'static + Handoff, { - let handoff_id = HandoffId(self.handoffs.len()); - // Create and insert handoff. let handoff = H::default(); - self.handoffs - .push(HandoffData::new(name.into(), handoff, handoff_id)); + let handoff_id = self + .handoffs + .insert_with_key(|hoff_id| HandoffData::new(name.into(), handoff, hoff_id)); // Make ports. let input_port = SendPort { diff --git a/dfir_rs/src/scheduled/handoff/handoff_list.rs b/dfir_rs/src/scheduled/handoff/handoff_list.rs index 87d4e4f3776f..08e759175b7a 100644 --- a/dfir_rs/src/scheduled/handoff/handoff_list.rs +++ b/dfir_rs/src/scheduled/handoff/handoff_list.rs @@ -7,7 +7,8 @@ use variadics::{variadic_trait, Variadic}; use super::Handoff; use crate::scheduled::graph::HandoffData; use crate::scheduled::port::{Polarity, Port, PortCtx}; -use crate::scheduled::{HandoffId, SubgraphId}; +use crate::scheduled::{HandoffId, HandoffTag, SubgraphId}; +use crate::util::slot_vec::SlotVec; /// Sealed trait for variadic lists of ports. /// @@ -29,7 +30,7 @@ where /// - `false`: Handoffs are successors (outputs) from subgraph `sg_id`. fn set_graph_meta( &self, - handoffs: &mut [HandoffData], + handoffs: &mut SlotVec, out_handoff_ids: &mut Vec, sg_id: SubgraphId, handoffs_are_preds: bool, @@ -41,7 +42,7 @@ where /// /// (Note that unlike [`Self::set_graph_meta`], this does not mess with pred/succ handoffs for /// teeing). - fn make_ctx<'a>(&self, handoffs: &'a [HandoffData]) -> Self::Ctx<'a>; + fn make_ctx<'a>(&self, handoffs: &'a SlotVec) -> Self::Ctx<'a>; } #[sealed] impl PortList for (Port, Rest) @@ -52,13 +53,13 @@ where { fn set_graph_meta( &self, - handoffs: &mut [HandoffData], + handoffs: &mut SlotVec, out_handoff_ids: &mut Vec, sg_id: SubgraphId, handoffs_are_preds: bool, ) { let (this, rest) = self; - let this_handoff = &mut handoffs[this.handoff_id.0]; + let this_handoff = &mut handoffs[this.handoff_id]; // Set subgraph's info (`out_handoff_ids`) about neighbor handoffs. // Use the "representative" handoff (pred or succ) for teeing handoffs, for the subgraph metadata. @@ -72,21 +73,21 @@ where // Set handoff's info (`preds`/`succs`) about neighbor subgraph (`sg_id`). if handoffs_are_preds { for succ_hoff in this_handoff.succ_handoffs.clone() { - handoffs[succ_hoff.0].succs.push(sg_id); + handoffs[succ_hoff].succs.push(sg_id); } } else { for pred_hoff in this_handoff.pred_handoffs.clone() { - handoffs[pred_hoff.0].preds.push(sg_id); + handoffs[pred_hoff].preds.push(sg_id); } } rest.set_graph_meta(handoffs, out_handoff_ids, sg_id, handoffs_are_preds); } type Ctx<'a> = (&'a PortCtx, Rest::Ctx<'a>); - fn make_ctx<'a>(&self, handoffs: &'a [HandoffData]) -> Self::Ctx<'a> { + fn make_ctx<'a>(&self, handoffs: &'a SlotVec) -> Self::Ctx<'a> { let (this, rest) = self; let handoff = handoffs - .get(this.handoff_id.0) + .get(this.handoff_id) .unwrap() .handoff .any_ref() @@ -105,7 +106,7 @@ where { fn set_graph_meta( &self, - _handoffs: &mut [HandoffData], + _handoffs: &mut SlotVec, _out_handoff_ids: &mut Vec, _sg_id: SubgraphId, _handoffs_are_preds: bool, @@ -113,7 +114,7 @@ where } type Ctx<'a> = (); - fn make_ctx<'a>(&self, _handoffs: &'a [HandoffData]) -> Self::Ctx<'a> {} + fn make_ctx<'a>(&self, _handoffs: &'a SlotVec) -> Self::Ctx<'a> {} } /// Trait for splitting a list of ports into two. diff --git a/dfir_rs/src/scheduled/mod.rs b/dfir_rs/src/scheduled/mod.rs index b322411cce39..0bf8003ea714 100644 --- a/dfir_rs/src/scheduled/mod.rs +++ b/dfir_rs/src/scheduled/mod.rs @@ -3,9 +3,7 @@ //! The most important item is the [`Hydroflow`](graph::Dfir) struct. Most of the items in this //! module are supporting the implementation of the `Hydroflow` struct and its operation. -use std::fmt::{Display, Formatter, Result}; - -use serde::Serialize; +use crate::util::slot_vec::Key; pub mod context; pub mod graph; @@ -21,27 +19,17 @@ pub(crate) mod subgraph; pub mod ticks; +/// Tag for [`SubgraphId`]. +pub enum SubgraphTag {} /// A subgraph's ID. Invalid if used in a different [`graph::Dfir`] /// instance than the original that created it. -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, Serialize)] -#[repr(transparent)] -pub struct SubgraphId(pub(crate) usize); -impl Display for SubgraphId { - fn fmt(&self, f: &mut Formatter) -> Result { - write!(f, "{}", self.0) - } -} +pub type SubgraphId = Key; +/// Tag for [`HandoffId`]. +pub enum HandoffTag {} /// A handoff's ID. Invalid if used in a different [`graph::Dfir`] /// instance than the original that created it. -#[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize)] -#[repr(transparent)] -pub struct HandoffId(pub(crate) usize); -impl Display for HandoffId { - fn fmt(&self, f: &mut Formatter) -> Result { - write!(f, "{}", self.0) - } -} +pub type HandoffId = Key; /// A staten handle's ID. Invalid if used in a different [`graph::Dfir`] /// instance than the original that created it. diff --git a/dfir_rs/src/scheduled/state.rs b/dfir_rs/src/scheduled/state.rs index 4567b8a1cdec..e327610cbf35 100644 --- a/dfir_rs/src/scheduled/state.rs +++ b/dfir_rs/src/scheduled/state.rs @@ -11,7 +11,7 @@ use super::StateId; /// If you need to store state handles in a data structure see [`StateHandleErased`] which hides /// the generic type parameter. #[must_use] -#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct StateHandle { pub(crate) state_id: StateId, pub(crate) _phantom: PhantomData<*mut T>, @@ -32,7 +32,7 @@ impl Clone for StateHandle { /// Use [`StateHandle::::try_from()`](StateHandle::try_from) to convert the `StateHandleErased` /// back into a `StateHandle` of the given type `T`. If `T` is the wrong type then the original /// `StateHandleErased` will be returned as the `Err`. -#[derive(Clone, Copy, Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash)] pub struct StateHandleErased { state_id: StateId, type_id: TypeId, diff --git a/dfir_rs/src/scheduled/subgraph.rs b/dfir_rs/src/scheduled/subgraph.rs index 990abbf348a1..5d1a27d9ac7f 100644 --- a/dfir_rs/src/scheduled/subgraph.rs +++ b/dfir_rs/src/scheduled/subgraph.rs @@ -1,16 +1,18 @@ use super::context::Context; use super::graph::HandoffData; +use super::HandoffTag; +use crate::util::slot_vec::SlotVec; /// Represents a compiled subgraph. Used internally by [Dataflow] to erase the input/output [Handoff] types. pub(crate) trait Subgraph { // TODO: pass in some scheduling info? - fn run(&mut self, context: &mut Context, handoffs: &mut Vec); + fn run(&mut self, context: &mut Context, handoffs: &mut SlotVec); } impl Subgraph for F where - F: FnMut(&mut Context, &mut Vec), + F: FnMut(&mut Context, &mut SlotVec), { - fn run(&mut self, context: &mut Context, handoffs: &mut Vec) { + fn run(&mut self, context: &mut Context, handoffs: &mut SlotVec) { (self)(context, handoffs); } } diff --git a/dfir_rs/src/util/mod.rs b/dfir_rs/src/util/mod.rs index e864d4282721..0c6d61ed29e4 100644 --- a/dfir_rs/src/util/mod.rs +++ b/dfir_rs/src/util/mod.rs @@ -6,6 +6,7 @@ pub mod clear; pub mod demux_enum; pub mod monotonic_map; pub mod multiset; +pub mod slot_vec; pub mod sparse_vec; pub mod unsync; diff --git a/dfir_rs/src/util/slot_vec.rs b/dfir_rs/src/util/slot_vec.rs new file mode 100644 index 000000000000..cca950e182b6 --- /dev/null +++ b/dfir_rs/src/util/slot_vec.rs @@ -0,0 +1,119 @@ +//! A Vec-based SlotMap-esque datastructure and corresponding Key type. + +use std::fmt::{Debug, Display, Formatter}; +use std::hash::{Hash, Hasher}; +use std::marker::PhantomData; +use std::ops::{Index, IndexMut}; + +/// A key into a SlotVec. +#[repr(transparent)] +pub struct Key { + index: usize, + _phantom: PhantomData, +} +impl Key { + /// Creates a Key from a raw index. Avoid using this function directly. + pub fn from_raw(index: usize) -> Self { + Key { + index, + _phantom: PhantomData, + } + } +} +impl Clone for Key { + fn clone(&self) -> Self { + *self + } +} +impl Copy for Key {} +impl PartialEq for Key { + fn eq(&self, other: &Self) -> bool { + self.index == other.index + } +} +impl Eq for Key {} +impl Hash for Key { + fn hash(&self, state: &mut H) { + self.index.hash(state); + } +} +impl Debug for Key { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + write!(f, "Key({})", self.index) + } +} +impl Display for Key { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + write!(f, "{}", self.index) + } +} + +/// A Vec-based SlotMap-esque datastructure without removes. +#[repr(transparent)] +pub struct SlotVec { + slots: Vec, + _phantom: PhantomData, +} +impl SlotVec { + /// Creates a new SlotVec. + pub fn new() -> Self { + Self { + slots: Vec::default(), + _phantom: PhantomData, + } + } + + /// Inserts a value into the SlotVec and returns the key. + pub fn insert(&mut self, value: Val) -> Key { + let key = Key::from_raw(self.slots.len()); + self.slots.push(value); + key + } + + /// Use the provided function to generate a value given the key and insert it into the SlotVec. + pub fn insert_with_key(&mut self, func: F) -> Key + where + F: FnOnce(Key) -> Val, + { + let key = Key::from_raw(self.slots.len()); + self.slots.push((func)(key)); + key + } + + /// Returns a reference to the value associated with the key. + pub fn get(&self, key: Key) -> Option<&Val> { + self.slots.get(key.index) + } + + /// Returns a mutable reference to the value associated with the key. + pub fn get_mut(&mut self, key: Key) -> Option<&mut Val> { + self.slots.get_mut(key.index) + } + + /// Returns the number of elements in the SlotVec. + pub fn len(&self) -> usize { + self.slots.len() + } + + /// Returns true if the SlotVec is empty. + pub fn is_empty(&self) -> bool { + self.slots.is_empty() + } +} +impl Index> for SlotVec { + type Output = Val; + + fn index(&self, key: Key) -> &Self::Output { + self.get(key).unwrap() + } +} +impl IndexMut> for SlotVec { + fn index_mut(&mut self, key: Key) -> &mut Self::Output { + self.get_mut(key).unwrap() + } +} +impl Default for SlotVec { + fn default() -> Self { + Self::new() + } +} diff --git a/dfir_rs/src/util/unsync/mpsc.rs b/dfir_rs/src/util/unsync/mpsc.rs index b4b8d802fb43..6a4732db596a 100644 --- a/dfir_rs/src/util/unsync/mpsc.rs +++ b/dfir_rs/src/util/unsync/mpsc.rs @@ -25,7 +25,7 @@ impl Sender { let mut shared = strong.borrow_mut(); if shared .capacity - .map_or(false, |cap| cap.get() <= shared.buffer.len()) + .is_some_and(|cap| cap.get() <= shared.buffer.len()) { // Full. shared.send_wakers.push(ctx.waker().clone()); @@ -53,7 +53,7 @@ impl Sender { let mut shared = strong.borrow_mut(); if shared .capacity - .map_or(false, |cap| cap.get() <= shared.buffer.len()) + .is_some_and(|cap| cap.get() <= shared.buffer.len()) { Err(TrySendError::Full(item)) } else { @@ -104,7 +104,7 @@ impl Sink for Sender { let mut shared = strong.borrow_mut(); if shared .capacity - .map_or(false, |cap| cap.get() <= shared.buffer.len()) + .is_some_and(|cap| cap.get() <= shared.buffer.len()) { // Full. shared.send_wakers.push(ctx.waker().clone()); diff --git a/dfir_rs/tests/surface_examples.rs b/dfir_rs/tests/surface_examples.rs index 36acb582b6c6..15d7ac777e92 100644 --- a/dfir_rs/tests/surface_examples.rs +++ b/dfir_rs/tests/surface_examples.rs @@ -11,11 +11,7 @@ fn test_all() { let examples_files = read_dir("examples/") .unwrap() .flat_map(Result::ok) - .filter(|entry| { - entry - .file_type() - .map_or(false, |file_type| file_type.is_file()) - }) + .filter(|entry| entry.file_type().is_ok_and(|file_type| file_type.is_file())) .map(|entry| entry.file_name()) .map(|filename| filename.into_string().unwrap()) .filter(|filename| filename.starts_with("example_") && filename.ends_with(".rs")) diff --git a/docs/docs/hydro/dataflow-programming.mdx b/docs/docs/hydro/dataflow-programming.mdx index e4aa44b89b43..cbb3c69f9dbb 100644 --- a/docs/docs/hydro/dataflow-programming.mdx +++ b/docs/docs/hydro/dataflow-programming.mdx @@ -9,5 +9,10 @@ Programs in Hydro describe how to **transform** entire collections of data using 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. +The core of the Hydro API are the following types and their methods: +- [Stream](https://hydro.run/rustdoc/hydro_lang/stream/struct.Stream) +- [Singleton](https://hydro.run/rustdoc/hydro_lang/singleton/struct.Singleton) +- [Optional](https://hydro.run/rustdoc/hydro_lang/optional/struct.Optional) + diff --git a/docs/docs/hydro/quickstart/first-dataflow.mdx b/docs/docs/hydro/quickstart/first-dataflow.mdx index d34d8820f74c..286c4879d9f6 100644 --- a/docs/docs/hydro/quickstart/first-dataflow.mdx +++ b/docs/docs/hydro/quickstart/first-dataflow.mdx @@ -35,7 +35,7 @@ We'll write our first dataflow in `src/first_ten.rs`. This program will run on a You'll notice that the arguments to `source_iter` and `for_each` are wrapped in `q!` macros. The top-level Hydro program (`first_ten`) is responsible for setting up the dataflow structure, whereas the `q!` macro is used to mark the Rust code that will be executed at **runtime**. Generally, runtime code in a `q!` macro is a snippet of Rust code that defines a static source of data or a closure. If you forget to wrap a block in `q!` when that is required, you'll see an error like: -``` +```text closure is expected to take 5 arguments, but it takes X arguments ``` diff --git a/hydro_deploy/core/src/deployment.rs b/hydro_deploy/core/src/deployment.rs index bcec6996e3d5..f2608108bd56 100644 --- a/hydro_deploy/core/src/deployment.rs +++ b/hydro_deploy/core/src/deployment.rs @@ -1,3 +1,8 @@ +#![allow( + unexpected_cfgs, + reason = "https://github.com/BrynCooke/buildstructor/issues/192" +)] + use std::collections::HashMap; use std::future::Future; use std::sync::{Arc, Weak}; diff --git a/hydro_deploy/core/src/hydroflow_crate/build.rs b/hydro_deploy/core/src/hydroflow_crate/build.rs index b6612ca288a1..edd3ced5389f 100644 --- a/hydro_deploy/core/src/hydroflow_crate/build.rs +++ b/hydro_deploy/core/src/hydroflow_crate/build.rs @@ -2,7 +2,7 @@ use std::error::Error; use std::fmt::Display; use std::io::BufRead; use std::path::{Path, PathBuf}; -use std::process::{Command, Stdio}; +use std::process::{Command, ExitStatus, Stdio}; use std::sync::OnceLock; use cargo_metadata::diagnostic::Diagnostic; @@ -179,14 +179,20 @@ pub async fn build_crate_memoized(params: BuildParams) -> Result<&'static BuildO ProgressTracker::println(msg.message.rendered.as_deref().unwrap()); diagnostics.push(msg.message); } + cargo_metadata::Message::TextLine(line) => { + ProgressTracker::println(&line); + } + cargo_metadata::Message::BuildFinished(_) => {} + cargo_metadata::Message::BuildScriptExecuted(_) => {} _ => {} } } - if spawned.wait().unwrap().success() { + let exit_code = spawned.wait().unwrap(); + if exit_code.success() { Err(BuildError::NoBinaryEmitted) } else { - Err(BuildError::FailedToBuildCrate(diagnostics)) + Err(BuildError::FailedToBuildCrate(exit_code, diagnostics)) } }) .await @@ -198,7 +204,7 @@ pub async fn build_crate_memoized(params: BuildParams) -> Result<&'static BuildO #[derive(Clone, Debug)] pub enum BuildError { - FailedToBuildCrate(Vec), + FailedToBuildCrate(ExitStatus, Vec), TokioJoinError, NoBinaryEmitted, } @@ -206,8 +212,8 @@ pub enum BuildError { impl Display for BuildError { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { - Self::FailedToBuildCrate(diagnostics) => { - writeln!(f, "Failed to build crate:")?; + Self::FailedToBuildCrate(exit_status, diagnostics) => { + writeln!(f, "Failed to build crate (exit status {}):", exit_status)?; for diagnostic in diagnostics { write!(f, "{}", diagnostic)?; } diff --git a/hydro_deploy/core/src/hydroflow_crate/tracing_options.rs b/hydro_deploy/core/src/hydroflow_crate/tracing_options.rs index 096ae9da0ff0..522831a79f76 100644 --- a/hydro_deploy/core/src/hydroflow_crate/tracing_options.rs +++ b/hydro_deploy/core/src/hydroflow_crate/tracing_options.rs @@ -1,4 +1,8 @@ #![allow(clippy::too_many_arguments, reason = "buildstructor")] +#![allow( + unexpected_cfgs, + reason = "https://github.com/BrynCooke/buildstructor/issues/192" +)] use std::path::PathBuf; diff --git a/hydro_lang/Cargo.toml b/hydro_lang/Cargo.toml index 66a4e6b0cceb..c102a46d01c4 100644 --- a/hydro_lang/Cargo.toml +++ b/hydro_lang/Cargo.toml @@ -22,7 +22,6 @@ build = [ "dep:dfir_lang" ] [dependencies] bincode = "1.3.1" -ctor = "0.2.8" hydro_deploy = { path = "../hydro_deploy/core", version = "^0.11.0", optional = true } dfir_rs = { path = "../dfir_rs", version = "^0.11.0", default-features = false, features = ["deploy_integration"] } dfir_lang = { path = "../dfir_lang", version = "^0.11.0", optional = true } @@ -41,6 +40,7 @@ syn = { version = "2.0.46", features = [ "parsing", "extra-traits", "visit-mut" tokio = { version = "1.29.0", features = [ "full" ] } toml = { version = "0.8.0", optional = true } trybuild-internals-api = { version = "1.0.99", optional = true } +ctor = "0.2" [build-dependencies] stageleft_tool = { path = "../stageleft_tool", version = "^0.5.0" } diff --git a/hydro_lang/src/builder/deploy.rs b/hydro_lang/src/builder/deploy.rs index 448c739ccf32..0b3c71890e42 100644 --- a/hydro_lang/src/builder/deploy.rs +++ b/hydro_lang/src/builder/deploy.rs @@ -116,8 +116,6 @@ impl<'a, D: Deploy<'a>> DeployFlow<'a, D> { } fn extra_stmts(&self, env: &>::CompileEnv) -> BTreeMap> { - let all_locations_count = self.nodes.len() + self.clusters.len(); - let mut extra_stmts: BTreeMap> = BTreeMap::new(); for &c_id in self.clusters.keys() { let self_id_ident = syn::Ident::new( @@ -132,14 +130,14 @@ impl<'a, D: Deploy<'a>> DeployFlow<'a, D> { let #self_id_ident = #self_id_expr; }); - for other_location in 0..all_locations_count { + for other_location in self.nodes.keys().chain(self.clusters.keys()) { let other_id_ident = syn::Ident::new( &format!("__hydro_lang_cluster_ids_{}", c_id), Span::call_site(), ); let other_id_expr = D::cluster_ids(env, c_id).splice_untyped(); extra_stmts - .entry(other_location) + .entry(*other_location) .or_default() .push(syn::parse_quote! { let #other_id_ident = #other_id_expr; @@ -176,26 +174,34 @@ impl<'a, D: Deploy<'a, CompileEnv = ()>> DeployFlow<'a, D> { let (mut processes, mut clusters, mut externals) = ( std::mem::take(&mut self.nodes) .into_iter() - .map(|(node_id, node)| { - node.instantiate( - env, - &mut meta, - compiled.remove(&node_id).unwrap(), - extra_stmts.remove(&node_id).unwrap_or_default(), - ); - (node_id, node) + .filter_map(|(node_id, node)| { + if let Some(ir) = compiled.remove(&node_id) { + node.instantiate( + env, + &mut meta, + ir, + extra_stmts.remove(&node_id).unwrap_or_default(), + ); + Some((node_id, node)) + } else { + None + } }) .collect::>(), std::mem::take(&mut self.clusters) .into_iter() - .map(|(cluster_id, cluster)| { - cluster.instantiate( - env, - &mut meta, - compiled.remove(&cluster_id).unwrap(), - extra_stmts.remove(&cluster_id).unwrap_or_default(), - ); - (cluster_id, cluster) + .filter_map(|(cluster_id, cluster)| { + if let Some(ir) = compiled.remove(&cluster_id) { + cluster.instantiate( + env, + &mut meta, + ir, + extra_stmts.remove(&cluster_id).unwrap_or_default(), + ); + Some((cluster_id, cluster)) + } else { + None + } }) .collect::>(), std::mem::take(&mut self.externals) diff --git a/hydro_lang/src/deploy/deploy_graph.rs b/hydro_lang/src/deploy/deploy_graph.rs index 3ef6aee4733c..ff17da80da39 100644 --- a/hydro_lang/src/deploy/deploy_graph.rs +++ b/hydro_lang/src/deploy/deploy_graph.rs @@ -370,7 +370,7 @@ impl<'a> Deploy<'a> for HydroDeploy { fn cluster_ids( _env: &Self::CompileEnv, of_cluster: usize, - ) -> impl QuotedWithContext<'a, &'a Vec, ()> + Copy + 'a { + ) -> impl QuotedWithContext<'a, &'a [u32], ()> + Copy + 'a { cluster_members(RuntimeData::new("__hydro_lang_trybuild_cli"), of_cluster) } diff --git a/hydro_lang/src/deploy/macro_runtime.rs b/hydro_lang/src/deploy/macro_runtime.rs index 0cd134b46bd1..e6d858b7fde2 100644 --- a/hydro_lang/src/deploy/macro_runtime.rs +++ b/hydro_lang/src/deploy/macro_runtime.rs @@ -170,7 +170,7 @@ impl<'a> Deploy<'a> for DeployRuntime { fn cluster_ids( env: &Self::CompileEnv, of_cluster: usize, - ) -> impl QuotedWithContext<'a, &'a Vec, ()> + Copy + 'a { + ) -> impl QuotedWithContext<'a, &'a [u32], ()> + Copy + 'a { crate::deploy_runtime::cluster_members(*env, of_cluster) } diff --git a/hydro_lang/src/deploy/mod.rs b/hydro_lang/src/deploy/mod.rs index cd19b556243a..3e76af6ffa53 100644 --- a/hydro_lang/src/deploy/mod.rs +++ b/hydro_lang/src/deploy/mod.rs @@ -170,7 +170,7 @@ pub trait Deploy<'a> { fn cluster_ids( env: &Self::CompileEnv, of_cluster: usize, - ) -> impl QuotedWithContext<'a, &'a Vec, ()> + Copy + 'a; + ) -> impl QuotedWithContext<'a, &'a [u32], ()> + Copy + 'a; fn cluster_self_id(env: &Self::CompileEnv) -> impl QuotedWithContext<'a, u32, ()> + Copy + 'a; } diff --git a/hydro_lang/src/deploy/trybuild.rs b/hydro_lang/src/deploy/trybuild.rs index a79f0c9118ae..b40c250a3417 100644 --- a/hydro_lang/src/deploy/trybuild.rs +++ b/hydro_lang/src/deploy/trybuild.rs @@ -10,7 +10,7 @@ use trybuild_internals_api::env::Update; use trybuild_internals_api::run::{PathDependency, Project}; use trybuild_internals_api::{dependencies, features, path, Runner}; -use super::trybuild_rewriters::{ReplaceCrateNameWithStaged, ReplaceCrateWithOrig}; +use super::trybuild_rewriters::ReplaceCrateNameWithStaged; static IS_TEST: std::sync::atomic::AtomicBool = std::sync::atomic::AtomicBool::new(false); @@ -44,19 +44,19 @@ pub fn create_graph_trybuild( ReplaceCrateNameWithStaged { crate_name: crate_name.clone(), + is_test, } .visit_file_mut(&mut generated_code); - let mut inlined_staged = stageleft_tool::gen_staged_trybuild( - &path!(source_dir / "src" / "lib.rs"), - crate_name.clone(), - is_test, - ); - - ReplaceCrateWithOrig { - crate_name: crate_name.clone(), - } - .visit_file_mut(&mut inlined_staged); + let inlined_staged = if is_test { + stageleft_tool::gen_staged_trybuild( + &path!(source_dir / "src" / "lib.rs"), + crate_name.clone(), + is_test, + ) + } else { + syn::parse_quote!() + }; let source = prettyplease::unparse(&syn::parse_quote! { #generated_code diff --git a/hydro_lang/src/deploy/trybuild_rewriters.rs b/hydro_lang/src/deploy/trybuild_rewriters.rs index 6a9a14cafe73..6af050a51835 100644 --- a/hydro_lang/src/deploy/trybuild_rewriters.rs +++ b/hydro_lang/src/deploy/trybuild_rewriters.rs @@ -2,6 +2,7 @@ use syn::visit_mut::VisitMut; pub struct ReplaceCrateNameWithStaged { pub crate_name: String, + pub is_test: bool, } impl VisitMut for ReplaceCrateNameWithStaged { @@ -9,27 +10,22 @@ impl VisitMut for ReplaceCrateNameWithStaged { if let Some(first) = i.path.segments.first() { if first.ident == self.crate_name { let tail = i.path.segments.iter().skip(1).collect::>(); - *i = syn::parse_quote!(crate::__staged #(::#tail)*); + + if self.is_test { + *i = syn::parse_quote!(crate::__staged #(::#tail)*); + } else { + let crate_ident = syn::Ident::new(&self.crate_name, first.ident.span()); + *i = syn::parse_quote!(#crate_ident::__staged #(::#tail)*); + } } } syn::visit_mut::visit_type_path_mut(self, i); } -} -pub struct ReplaceCrateWithOrig { - pub crate_name: String, -} - -impl VisitMut for ReplaceCrateWithOrig { - fn visit_item_use_mut(&mut self, i: &mut syn::ItemUse) { - if let syn::UseTree::Path(p) = &mut i.tree { - if p.ident == "crate" { - p.ident = syn::Ident::new(&self.crate_name, p.ident.span()); - i.leading_colon = Some(Default::default()); - } + fn visit_use_path_mut(&mut self, i: &mut syn::UsePath) { + if i.ident == "crate" && !self.is_test { + i.ident = syn::Ident::new(&self.crate_name, i.ident.span()); } - - syn::visit_mut::visit_item_use_mut(self, i); } } diff --git a/hydro_lang/src/deploy_runtime.rs b/hydro_lang/src/deploy_runtime.rs index ce5b08eb95a5..2f007fd5d62e 100644 --- a/hydro_lang/src/deploy_runtime.rs +++ b/hydro_lang/src/deploy_runtime.rs @@ -16,8 +16,13 @@ pub struct HydroflowPlusMeta { pub fn cluster_members( cli: RuntimeData<&DeployPorts>, of_cluster: usize, -) -> impl QuotedWithContext<&Vec, ()> + Copy { - q!(cli.meta.clusters.get(&of_cluster).unwrap()) +) -> impl QuotedWithContext<&[u32], ()> + Copy { + q!(cli + .meta + .clusters + .get(&of_cluster) + .map(|v| v.as_slice()) + .unwrap_or(&[])) // we default to empty slice because this is the scenario where the cluster is unused in the graph } pub fn cluster_self_id( diff --git a/hydro_lang/src/lib.rs b/hydro_lang/src/lib.rs index 438ffcc73bf2..aaad68cb9023 100644 --- a/hydro_lang/src/lib.rs +++ b/hydro_lang/src/lib.rs @@ -54,7 +54,7 @@ fn add_private_reexports() { #[stageleft::runtime] #[cfg(test)] -mod tests { +mod test_init { #[ctor::ctor] fn init() { crate::deploy::init_test(); diff --git a/hydro_lang/src/location/cluster/mod.rs b/hydro_lang/src/location/cluster/mod.rs index 884633bedc85..7e9ee8e76ed6 100644 --- a/hydro_lang/src/location/cluster/mod.rs +++ b/hydro_lang/src/location/cluster/mod.rs @@ -94,7 +94,7 @@ impl<'a, C: 'a, Ctx> FreeVariableWithContext for ClusterIds<'a, C> { ( None, Some( - quote! { unsafe { ::std::mem::transmute::<_, &::std::vec::Vec<#root::ClusterId<#c_type>>>(#ident) } }, + quote! { unsafe { ::std::mem::transmute::<_, &[#root::ClusterId<#c_type>]>(#ident) } }, ), ) } diff --git a/hydro_lang/src/stream.rs b/hydro_lang/src/stream.rs index 7d8ec7d1a8f4..51944eaa1d9a 100644 --- a/hydro_lang/src/stream.rs +++ b/hydro_lang/src/stream.rs @@ -223,21 +223,21 @@ impl<'a, T: Clone, L: Location<'a>, B, Order> Clone for Stream { } impl<'a, T, L: Location<'a>, B, Order> Stream { - /// Transforms the stream by applying a function (`f`) to each element, - /// emitting the output elements in the same order as the input. + /// Produces a stream based on invoking `f` on each element in order. + /// If you do not want to modify the stream and instead only want to view + /// each item use [`Stream::inspect`] instead. /// /// # Example /// ```rust /// # use hydro_lang::*; /// # use dfir_rs::futures::StreamExt; /// # tokio_test::block_on(test_util::stream_transform_test(|process| { - /// let numbers = process.source_iter(q!(0..10)); - /// let mapped = numbers.map(q!(|n| n * 2)); - /// # mapped + /// let words = process.source_iter(q!(vec!["hello", "world"])); + /// words.map(q!(|x| x.to_uppercase())) /// # }, |mut stream| async move { - /// // 2, 4, 6, 8, ... - /// # for i in 0..10 { - /// # assert_eq!(stream.next().await.unwrap(), i * 2); + /// // HELLO, WORLD + /// # for w in vec!["HELLO", "WORLD"] { + /// # assert_eq!(stream.next().await.unwrap(), w); /// # } /// # })); /// ``` @@ -256,6 +256,21 @@ 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, @@ -263,6 +278,28 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { 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. + /// + /// For example, `U` could be a `Vec`, but not a `HashSet`. If the order of the items in `U` is + /// not deterministic, use [`Stream::flat_map_unordered`] instead. + /// + /// # 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![vec![1, 2], vec![3, 4]])) + /// .flat_map_ordered(q!(|x| x)) + /// # }, |mut stream| async move { + /// // 1, 2, 3, 4 + /// # for w in (1..5) { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); + /// ``` pub fn flat_map_ordered, F: Fn(T) -> I + 'a>( self, f: impl IntoQuotedMut<'a, F, L>, @@ -278,6 +315,30 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } + /// Like [`Stream::flat_map_ordered`], but allows the implementation of [`Iterator`] + /// for the output type `U` to produce items in any order. + /// + /// # Example + /// ```rust + /// # use hydro_lang::*; + /// # use dfir_rs::futures::StreamExt; + /// # tokio_test::block_on(test_util::stream_transform_test::<_, _, NoOrder>(|process| { + /// process + /// .source_iter(q!(vec![ + /// std::collections::HashSet::::from_iter(vec![1, 2]), + /// std::collections::HashSet::from_iter(vec![3, 4]), + /// ])) + /// .flat_map_unordered(q!(|x| x)) + /// # }, |mut stream| async move { + /// // 1, 2, 3, 4, but in no particular order + /// # let mut results = Vec::new(); + /// # for w in (1..5) { + /// # results.push(stream.next().await.unwrap()); + /// # } + /// # results.sort(); + /// # assert_eq!(results, vec![1, 2, 3, 4]); + /// # })); + /// ``` pub fn flat_map_unordered, F: Fn(T) -> I + 'a>( self, f: impl IntoQuotedMut<'a, F, L>, @@ -293,6 +354,26 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } + /// For each item `i` in the input stream, treat `i` as an [`Iterator`] and produce its items one by one. + /// The implementation for [`Iterator`] for the element type `T` must produce items in a **deterministic** order. + /// + /// For example, `T` could be a `Vec`, but not a `HashSet`. If the order of the items in `T` is + /// not deterministic, use [`Stream::flatten_unordered`] instead. + /// + /// ```rust + /// # use hydro_lang::*; + /// # use dfir_rs::futures::StreamExt; + /// # tokio_test::block_on(test_util::stream_transform_test(|process| { + /// process + /// .source_iter(q!(vec![vec![1, 2], vec![3, 4]])) + /// .flatten_ordered() + /// # }, |mut stream| async move { + /// // 1, 2, 3, 4 + /// # for w in (1..5) { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); + /// ``` pub fn flatten_ordered(self) -> Stream where T: IntoIterator, @@ -300,6 +381,29 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { self.flat_map_ordered(q!(|d| d)) } + /// Like [`Stream::flatten_ordered`], but allows the implementation of [`Iterator`] + /// for the element type `T` to produce items in any order. + /// + /// # Example + /// ```rust + /// # use hydro_lang::*; + /// # use dfir_rs::futures::StreamExt; + /// # tokio_test::block_on(test_util::stream_transform_test::<_, _, NoOrder>(|process| { + /// process + /// .source_iter(q!(vec![ + /// std::collections::HashSet::::from_iter(vec![1, 2]), + /// std::collections::HashSet::from_iter(vec![3, 4]), + /// ])) + /// .flatten_unordered() + /// # }, |mut stream| async move { + /// // 1, 2, 3, 4, but in no particular order + /// # let mut results = Vec::new(); + /// # for w in (1..5) { + /// # results.push(stream.next().await.unwrap()); + /// # } + /// # results.sort(); + /// # assert_eq!(results, vec![1, 2, 3, 4]); + /// # })); pub fn flatten_unordered(self) -> Stream where T: IntoIterator, @@ -307,6 +411,28 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { self.flat_map_unordered(q!(|d| d)) } + /// Creates a stream containing only the elements of the input stream that satisfy a predicate + /// `f`, preserving the order of the elements. + /// + /// The closure `f` receives a reference `&T` rather than an owned value `T` because filtering does + /// not modify or take ownership of the values. If you need to modify the values while filtering + /// use [`Stream::filter_map`] instead. + /// + /// # 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, 2, 3, 4])) + /// .filter(q!(|&x| x > 2)) + /// # }, |mut stream| async move { + /// // 3, 4 + /// # for w in (3..5) { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); + /// ``` pub fn filter bool + 'a>( self, f: impl IntoQuotedMut<'a, F, L>, @@ -322,6 +448,22 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } + /// An operator that both filters and maps. It yields only the items for which the supplied closure `f` returns `Some(value)`. + /// + /// # 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", "hello", "world", "2"])) + /// .filter_map(q!(|s| s.parse::().ok())) + /// # }, |mut stream| async move { + /// // 1, 2 + /// # for w in (1..3) { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); pub fn filter_map Option + 'a>( self, f: impl IntoQuotedMut<'a, F, L>, @@ -337,6 +479,29 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } + /// Generates a stream that maps each input element `i` to a tuple `(i, x)`, + /// where `x` is the final value of `other`, a bounded [`Singleton`]. + /// + /// # 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 batch = unsafe { + /// process + /// .source_iter(q!(vec![1, 2, 3, 4])) + /// .timestamped(&tick) + /// .tick_batch() + /// }; + /// let count = batch.clone().count(); + /// batch.cross_singleton(count).all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // (1, 4), (2, 4), (3, 4), (4, 4) + /// # for w in vec![(1, 4), (2, 4), (3, 4), (4, 4)] { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); pub fn cross_singleton( self, other: impl Into>, @@ -357,7 +522,7 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } - /// Allow this stream through if the other stream has elements, otherwise the output is empty. + /// Allow this stream through if the argument (a Bounded Optional) is non-empty, otherwise the output is empty. pub fn continue_if(self, signal: Optional) -> Stream { self.cross_singleton(signal.map(q!(|_u| ()))) .map(q!(|(d, _signal)| d)) @@ -368,6 +533,8 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { self.continue_if(other.into_stream().count().filter(q!(|c| *c == 0))) } + /// Forms the cross-product (Cartesian product, cross-join) of the items in the 2 input streams, returning all + /// tupled pairs. pub fn cross_product(self, other: Stream) -> Stream<(T, O), L, B, Order> where T: Clone, @@ -385,6 +552,8 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } + /// Takes one stream as input and filters out any duplicate occurrences. The output + /// contains all unique values from the input. pub fn unique(self) -> Stream where T: Eq + Hash, @@ -398,6 +567,10 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } + /// Outputs everything in this stream that is *not* contained in the `other` stream. + /// + /// The `other` stream must be [`Bounded`], since this function will wait until + /// all its elements are available before producing any output. pub fn filter_not_in(self, other: Stream) -> Stream where T: Eq + Hash, @@ -414,6 +587,9 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) } + /// An operator which allows you to "inspect" each element of a stream without + /// modifying it. The closure `f` is called on a reference to each item. This is + /// mainly useful for debugging, and should not be used to generate side-effects. pub fn inspect( self, f: impl IntoQuotedMut<'a, F, L>, @@ -453,7 +629,7 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { /// /// # Safety /// This function is used as an escape hatch, and any mistakes in the - /// provided ordering guarantee will propogate into the guarantees + /// provided ordering guarantee will propagate into the guarantees /// for the rest of the program. pub unsafe fn assume_ordering(self) -> Stream { Stream::new(self.location, self.ir_node.into_inner()) @@ -464,6 +640,29 @@ impl<'a, T, L: Location<'a>, B, Order> Stream where Order: MinOrder, { + /// Combines elements of the stream into a [`Singleton`], by starting with an intitial value, + /// generated by the `init` closure, and then applying the `comb` closure to each element in the stream. + /// Unlike iterators, `comb` takes the accumulator by `&mut` reference, so that it can be modified in place. + /// + /// The `comb` closure must be **commutative**, as the order of input items is not guaranteed. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch + /// .fold_commutative(q!(|| 0), q!(|acc, x| *acc += x)) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // 10 + /// # assert_eq!(stream.next().await.unwrap(), 10); + /// # })); + /// ``` pub fn fold_commutative A + 'a, F: Fn(&mut A, T)>( self, init: impl IntoQuotedMut<'a, I, L>, @@ -492,6 +691,30 @@ where Singleton::new(self.location, core) } + /// Combines elements of the stream into a [`Optional`], by starting with the first element in the stream, + /// and then applying the `comb` closure to each element in the stream. The [`Optional`] will be empty + /// until the first element in the input arrives. Unlike iterators, `comb` takes the accumulator by `&mut` + /// reference, so that it can be modified in place. + /// + /// The `comb` closure must be **commutative**, as the order of input items is not guaranteed. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch + /// .reduce_commutative(q!(|curr, new| *curr += new)) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // 10 + /// # assert_eq!(stream.next().await.unwrap(), 10); + /// # })); + /// ``` pub fn reduce_commutative( self, comb: impl IntoQuotedMut<'a, F, L>, @@ -513,6 +736,23 @@ where Optional::new(self.location, core) } + /// Computes the maximum element in the stream as an [`Optional`], which + /// will be empty until the first element in the input arrives. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch.max().all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 4 + /// # assert_eq!(stream.next().await.unwrap(), 4); + /// # })); + /// ``` pub fn max(self) -> Optional where T: Ord, @@ -524,6 +764,24 @@ where })) } + /// Computes the maximum element in the stream as an [`Optional`], where the + /// maximum is determined according to the `key` function. The [`Optional`] will + /// be empty until the first element in the input arrives. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch.max_by_key(q!(|x| -x)).all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 1 + /// # assert_eq!(stream.next().await.unwrap(), 1); + /// # })); + /// ``` pub fn max_by_key K + 'a>( self, key: impl IntoQuotedMut<'a, F, L> + Copy, @@ -555,6 +813,23 @@ where Optional::new(self.location, core) } + /// Computes the minimum element in the stream as an [`Optional`], which + /// will be empty until the first element in the input arrives. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch.min().all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 1 + /// # assert_eq!(stream.next().await.unwrap(), 1); + /// # })); + /// ``` pub fn min(self) -> Optional where T: Ord, @@ -566,6 +841,22 @@ where })) } + /// Computes the number of elements in the stream as a [`Singleton`]. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch.count().all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 4 + /// # assert_eq!(stream.next().await.unwrap(), 4); + /// # })); + /// ``` pub fn count(self) -> Singleton { self.fold_commutative(q!(|| 0usize), q!(|count, _| *count += 1)) } @@ -600,14 +891,78 @@ impl<'a, T, L: Location<'a>, B> Stream { } } + /// Computes the first element in the stream as an [`Optional`], which + /// will be empty until the first element in the input arrives. + /// + /// This requires the stream to have a [`TotalOrder`] guarantee, otherwise + /// re-ordering of elements may cause the first element to change. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch.first().all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 1 + /// # assert_eq!(stream.next().await.unwrap(), 1); + /// # })); + /// ``` pub fn first(self) -> Optional { Optional::new(self.location, self.ir_node.into_inner()) } + /// Computes the last element in the stream as an [`Optional`], which + /// will be empty until an element in the input arrives. + /// + /// This requires the stream to have a [`TotalOrder`] guarantee, otherwise + /// re-ordering of elements may cause the last element to change. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch.last().all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 4 + /// # assert_eq!(stream.next().await.unwrap(), 4); + /// # })); + /// ``` pub fn last(self) -> Optional { self.reduce(q!(|curr, new| *curr = new)) } + /// Combines elements of the stream into a [`Singleton`], by starting with an intitial value, + /// generated by the `init` closure, and then applying the `comb` closure to each element in the stream. + /// Unlike iterators, `comb` takes the accumulator by `&mut` reference, so that it can be modified in place. + /// + /// The input stream must have a [`TotalOrder`] guarantee, which means that the `comb` closure is allowed + /// to depend on the order of elements in the stream. + /// + /// # 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 words = process.source_iter(q!(vec!["HELLO", "WORLD"])); + /// let batch = unsafe { words.timestamped(&tick).tick_batch() }; + /// batch + /// .fold(q!(|| String::new()), q!(|acc, x| acc.push_str(x))) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // "HELLOWORLD" + /// # assert_eq!(stream.next().await.unwrap(), "HELLOWORLD"); + /// # })); + /// ``` pub fn fold A + 'a, F: Fn(&mut A, T)>( self, init: impl IntoQuotedMut<'a, I, L>, @@ -636,6 +991,31 @@ impl<'a, T, L: Location<'a>, B> Stream { Singleton::new(self.location, core) } + /// Combines elements of the stream into a [`Optional`], by starting with the first element in the stream, + /// and then applying the `comb` closure to each element in the stream. The [`Optional`] will be empty + /// until the first element in the input arrives. + /// + /// The input stream must have a [`TotalOrder`] guarantee, which means that the `comb` closure is allowed + /// to depend on the order of elements in the stream. + /// + /// # 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 words = process.source_iter(q!(vec!["HELLO", "WORLD"])); + /// let batch = unsafe { words.timestamped(&tick).tick_batch() }; + /// batch + /// .map(q!(|x| x.to_string())) + /// .reduce(q!(|curr, new| curr.push_str(&new))) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // "HELLOWORLD" + /// # assert_eq!(stream.next().await.unwrap(), "HELLOWORLD"); + /// # })); + /// ``` pub fn reduce( self, comb: impl IntoQuotedMut<'a, F, L>, @@ -658,43 +1038,72 @@ impl<'a, T, L: Location<'a>, B> Stream { } } -impl<'a, T, L: Location<'a>> Stream { - pub fn chain( - self, - other: Stream, - ) -> Stream { - check_matching_location(&self.location, &other.location); - - Stream::new( - 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::(), - }, - ) - } -} - -impl<'a, T, L: Location<'a> + NoTick + NoTimestamp> Stream { - pub fn union( - self, - other: Stream, - ) -> Stream { +impl<'a, T, L: Location<'a> + NoTick + NoTimestamp, O> Stream { + /// Produces a new stream that interleaves the elements of the two input streams. + /// The result has [`NoOrder`] because the order of interleaving is not guaranteed. + /// + /// Currently, both input streams must be [`Unbounded`]. When the streams are + /// [`Bounded`], you can use [`Stream::chain`] instead. + /// + /// # Example + /// ```rust + /// # use hydro_lang::*; + /// # use dfir_rs::futures::StreamExt; + /// # tokio_test::block_on(test_util::stream_transform_test(|process| { + /// let numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// numbers.clone().map(q!(|x| x + 1)).union(numbers) + /// # }, |mut stream| async move { + /// // 2, 3, 4, 5, and 1, 2, 3, 4 interleaved in unknown order + /// # for w in vec![2, 3, 4, 5, 1, 2, 3, 4] { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); + /// ``` + pub fn union(self, other: Stream) -> Stream { let tick = self.location.tick(); unsafe { - // SAFETY: Because the inputs and outputs are unordered, + // SAFETY: Because the outputs are unordered, // we can interleave batches from both streams. self.timestamped(&tick) .tick_batch() - .union(other.timestamped(&tick).tick_batch()) + .assume_ordering::() + .chain( + other + .timestamped(&tick) + .tick_batch() + .assume_ordering::(), + ) .all_ticks() .drop_timestamp() + .assume_ordering() } } } impl<'a, T, L: Location<'a>, Order> Stream { + /// Produces a new stream that emits the input elements in sorted order. + /// + /// The input stream can have any ordering guarantee, but the output stream + /// will have a [`TotalOrder`] guarantee. This operator will block until all + /// elements in the input stream are available, so it requires the input stream + /// to be [`Bounded`]. + /// + /// # 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 numbers = process.source_iter(q!(vec![4, 2, 3, 1])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch.sort().all_ticks().drop_timestamp() + /// # }, |mut stream| async move { + /// // 1, 2, 3, 4 + /// # for w in (1..5) { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); + /// ``` pub fn sort(self) -> Stream where T: Ord, @@ -708,7 +1117,37 @@ impl<'a, T, L: Location<'a>, Order> Stream { ) } - pub fn union(self, other: Stream) -> Stream + /// Produces a new stream that first emits the elements of the `self` stream, + /// and then emits the elements of the `other` stream. The output stream has + /// a [`TotalOrder`] guarantee if and only if both input streams have a + /// [`TotalOrder`] guarantee. + /// + /// Currently, both input streams must be [`Bounded`]. This operator will block + /// on the first stream until all its elements are available. In a future version, + /// we will relax the requirement on the `other` stream. + /// + /// # 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 numbers = process.source_iter(q!(vec![1, 2, 3, 4])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch + /// .clone() + /// .map(q!(|x| x + 1)) + /// .chain(batch) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // 2, 3, 4, 5, 1, 2, 3, 4 + /// # for w in vec![2, 3, 4, 5, 1, 2, 3, 4] { + /// # assert_eq!(stream.next().await.unwrap(), w); + /// # } + /// # })); + /// ``` + pub fn chain(self, other: Stream) -> Stream where Order: MinOrder, { @@ -726,6 +1165,8 @@ impl<'a, T, L: Location<'a>, Order> Stream { } impl<'a, K, V1, L: Location<'a>, B, Order> Stream<(K, V1), L, B, Order> { + /// Given two streams of pairs `(K, V1)` and `(K, V2)`, produces a new stream of nested pairs `(K, (V1, V2))` + /// by equi-joining the two streams on the key attribute `K`. pub fn join(self, n: Stream<(K, V2), L, B, O2>) -> Stream<(K, (V1, V2)), L, B, NoOrder> where K: Eq + Hash, @@ -742,6 +1183,10 @@ impl<'a, K, V1, L: Location<'a>, B, Order> Stream<(K, V1), L, B, Order> { ) } + /// Given two streams of pairs `(K, V1)` and `(K, V2)`, + /// computes the anti-join of the items in the input -- i.e. returns + /// unique items in the first input that do not have a matching key + /// in the second input. pub fn anti_join(self, n: Stream) -> Stream<(K, V1), L, B, Order> where K: Eq + Hash, @@ -760,6 +1205,34 @@ impl<'a, K, V1, L: Location<'a>, B, Order> Stream<(K, V1), L, B, Order> { } impl<'a, K: Eq + Hash, V, L: Location<'a>> Stream<(K, V), Tick, Bounded> { + /// A special case of [`Stream::fold`], in the spirit of SQL's GROUP BY and aggregation constructs. The input + /// tuples are partitioned into groups by the first element ("keys"), and for each group the values + /// in the second element are accumulated via the `comb` closure. + /// + /// The input stream must have a [`TotalOrder`] guarantee, which means that the `comb` closure is allowed + /// to depend on the order of elements in the stream. + /// + /// If the input and output value types are the same and do not require initialization then use + /// [`Stream::reduce_keyed`]. + /// + /// # 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 numbers = process.source_iter(q!(vec![(1, 2), (2, 3), (1, 3), (2, 4)])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch + /// .fold_keyed(q!(|| 0), q!(|acc, x| *acc += x)) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // (1, 5), (2, 7) + /// # assert_eq!(stream.next().await.unwrap(), (1, 5)); + /// # assert_eq!(stream.next().await.unwrap(), (2, 7)); + /// # })); + /// ``` pub fn fold_keyed A + 'a, F: Fn(&mut A, V) + 'a>( self, init: impl IntoQuotedMut<'a, I, Tick>, @@ -779,6 +1252,33 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>> Stream<(K, V), Tick, Bounded> { ) } + /// A special case of [`Stream::reduce`], in the spirit of SQL's GROUP BY and aggregation constructs. The input + /// tuples are partitioned into groups by the first element ("keys"), and for each group the values + /// in the second element are accumulated via the `comb` closure. + /// + /// The input stream must have a [`TotalOrder`] guarantee, which means that the `comb` closure is allowed + /// to depend on the order of elements in the stream. + /// + /// If you need the accumulated value to have a different type than the input, use [`Stream::fold_keyed`]. + /// + /// # 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 numbers = process.source_iter(q!(vec![(1, 2), (2, 3), (1, 3), (2, 4)])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch + /// .reduce_keyed(q!(|acc, x| *acc += x)) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // (1, 5), (2, 7) + /// # assert_eq!(stream.next().await.unwrap(), (1, 5)); + /// # assert_eq!(stream.next().await.unwrap(), (2, 7)); + /// # })); + /// ``` pub fn reduce_keyed( self, comb: impl IntoQuotedMut<'a, F, Tick>, @@ -797,6 +1297,33 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>> Stream<(K, V), Tick, Bounded> { } impl<'a, K: Eq + Hash, V, L: Location<'a>, Order> Stream<(K, V), Tick, Bounded, Order> { + /// A special case of [`Stream::fold_commutative`], in the spirit of SQL's GROUP BY and aggregation constructs. The input + /// tuples are partitioned into groups by the first element ("keys"), and for each group the values + /// in the second element are accumulated via the `comb` closure. + /// + /// The `comb` closure must be **commutative**, as the order of input items is not guaranteed. + /// + /// If the input and output value types are the same and do not require initialization then use + /// [`Stream::reduce_keyed_commutative`]. + /// + /// # 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 numbers = process.source_iter(q!(vec![(1, 2), (2, 3), (1, 3), (2, 4)])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch + /// .fold_keyed_commutative(q!(|| 0), q!(|acc, x| *acc += x)) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // (1, 5), (2, 7) + /// # assert_eq!(stream.next().await.unwrap(), (1, 5)); + /// # assert_eq!(stream.next().await.unwrap(), (2, 7)); + /// # })); + /// ``` pub fn fold_keyed_commutative A + 'a, F: Fn(&mut A, V) + 'a>( self, init: impl IntoQuotedMut<'a, I, Tick>, @@ -816,11 +1343,38 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>, Order> Stream<(K, V), Tick, Bounde ) } + /// Given a stream of pairs `(K, V)`, produces a new stream of unique keys `K`. pub fn keys(self) -> Stream, Bounded, Order> { self.fold_keyed_commutative(q!(|| ()), q!(|_, _| {})) .map(q!(|(k, _)| k)) } + /// A special case of [`Stream::reduce_commutative`], in the spirit of SQL's GROUP BY and aggregation constructs. The input + /// tuples are partitioned into groups by the first element ("keys"), and for each group the values + /// in the second element are accumulated via the `comb` closure. + /// + /// The `comb` closure must be **commutative**, as the order of input items is not guaranteed. + /// + /// If you need the accumulated value to have a different type than the input, use [`Stream::fold_keyed_commutative`]. + /// + /// # 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 numbers = process.source_iter(q!(vec![(1, 2), (2, 3), (1, 3), (2, 4)])); + /// let batch = unsafe { numbers.timestamped(&tick).tick_batch() }; + /// batch + /// .reduce_keyed_commutative(q!(|acc, x| *acc += x)) + /// .all_ticks() + /// .drop_timestamp() + /// # }, |mut stream| async move { + /// // (1, 5), (2, 7) + /// # assert_eq!(stream.next().await.unwrap(), (1, 5)); + /// # assert_eq!(stream.next().await.unwrap(), (2, 7)); + /// # })); + /// ``` pub fn reduce_keyed_commutative( self, comb: impl IntoQuotedMut<'a, F, Tick>, diff --git a/hydro_lang/src/test_util.rs b/hydro_lang/src/test_util.rs index f79e4ce476e1..42d0cabc979b 100644 --- a/hydro_lang/src/test_util.rs +++ b/hydro_lang/src/test_util.rs @@ -10,8 +10,9 @@ pub async fn stream_transform_test< 'a, O: Serialize + DeserializeOwned + 'static, C: Future, + OutOrder, >( - thunk: impl FnOnce(&Process<'a>) -> Stream, Unbounded>, + thunk: impl FnOnce(&Process<'a>) -> Stream, Unbounded, OutOrder>, check: impl FnOnce(Pin>>) -> C, ) { let mut deployment = hydro_deploy::Deployment::new(); diff --git a/hydro_std/Cargo.toml b/hydro_std/Cargo.toml index 20599f0fae36..fdb4eaf80ffc 100644 --- a/hydro_std/Cargo.toml +++ b/hydro_std/Cargo.toml @@ -26,4 +26,4 @@ hydro_lang = { path = "../hydro_lang", version = "^0.11.0" } insta = "1.39" hydro_deploy = { path = "../hydro_deploy/core", version = "^0.11.0" } async-ssh2-lite = { version = "0.5.0", features = ["vendored-openssl"] } -ctor = "0.2.8" +ctor = "0.2" diff --git a/hydro_std/src/lib.rs b/hydro_std/src/lib.rs index 624c2f33af7a..bec454464388 100644 --- a/hydro_std/src/lib.rs +++ b/hydro_std/src/lib.rs @@ -5,7 +5,7 @@ pub mod request_response; #[stageleft::runtime] #[cfg(test)] -mod tests { +mod test_init { #[ctor::ctor] fn init() { hydro_lang::deploy::init_test(); diff --git a/hydro_std/src/quorum.rs b/hydro_std/src/quorum.rs index 200173fb73ce..beae281c90b1 100644 --- a/hydro_std/src/quorum.rs +++ b/hydro_std/src/quorum.rs @@ -22,7 +22,7 @@ pub fn collect_quorum_with_response< let tick = responses.timestamp_source(); let (not_all_complete_cycle, not_all) = tick.cycle::>(); - let current_responses = not_all.union(unsafe { + let current_responses = not_all.chain(unsafe { // SAFETY: we always persist values that have not reached quorum, so even // with arbitrary batching we always produce deterministic quorum results responses.clone().tick_batch() @@ -111,7 +111,7 @@ pub fn collect_quorum<'a, L: Location<'a> + NoTick, Order, K: Clone + Eq + Hash, let tick = responses.timestamp_source(); let (not_all_complete_cycle, not_all) = tick.cycle::>(); - let current_responses = not_all.union(unsafe { + let current_responses = not_all.chain(unsafe { // SAFETY: we always persist values that have not reached quorum, so even // with arbitrary batching we always produce deterministic quorum results responses.clone().tick_batch() diff --git a/hydro_std/src/request_response.rs b/hydro_std/src/request_response.rs index 230f1effa5a0..7532fd88996b 100644 --- a/hydro_std/src/request_response.rs +++ b/hydro_std/src/request_response.rs @@ -19,7 +19,7 @@ pub fn join_responses<'a, K: Clone + Eq + Hash, M: Clone, V: Clone, L: Location< let (remaining_to_join_complete_cycle, remaining_to_join) = tick.cycle::>(); - let remaining_and_new: Stream<(K, M), Tick, Bounded, _> = remaining_to_join.union(metadata); + let remaining_and_new: Stream<(K, M), Tick, Bounded, _> = remaining_to_join.chain(metadata); let responses = unsafe { // SAFETY: because we persist the metadata, delays resulting from diff --git a/hydro_test/Cargo.toml b/hydro_test/Cargo.toml index 83d1f911c96c..8e860bba9669 100644 --- a/hydro_test/Cargo.toml +++ b/hydro_test/Cargo.toml @@ -23,8 +23,10 @@ serde = { version = "1.0.197", features = [ "derive" ] } stageleft_tool = { path = "../stageleft_tool", version = "^0.5.0" } [dev-dependencies] +ctor = "0.2" insta = "1.39" hydro_deploy = { path = "../hydro_deploy/core", version = "^0.11.0" } 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" } diff --git a/hydro_test/examples/paxos.rs b/hydro_test/examples/paxos.rs index 07d24c0ba00f..bd8ca2d69ce9 100644 --- a/hydro_test/examples/paxos.rs +++ b/hydro_test/examples/paxos.rs @@ -3,6 +3,7 @@ use std::sync::Arc; use hydro_deploy::gcp::GcpNetwork; use hydro_deploy::{Deployment, Host}; use hydro_lang::deploy::TrybuildHost; +use hydro_test::cluster::paxos::PaxosConfig; use tokio::sync::RwLock; type HostCreator = Box Arc>; @@ -43,13 +44,15 @@ async fn main() { let (proposers, acceptors, clients, replicas) = hydro_test::cluster::paxos_bench::paxos_bench( &builder, - f, num_clients_per_node, median_latency_window_size, checkpoint_frequency, - i_am_leader_send_timeout, - i_am_leader_check_timeout, - i_am_leader_check_timeout_delay_multiplier, + PaxosConfig { + f, + i_am_leader_send_timeout, + i_am_leader_check_timeout, + i_am_leader_check_timeout_delay_multiplier, + }, ); let rustflags = "-C opt-level=3 -C codegen-units=1 -C strip=none -C debuginfo=2 -C lto=off"; diff --git a/hydro_test/src/cluster/bench_client.rs b/hydro_test/src/cluster/bench_client.rs new file mode 100644 index 000000000000..788a61c8da2b --- /dev/null +++ b/hydro_test/src/cluster/bench_client.rs @@ -0,0 +1,164 @@ +use std::cell::RefCell; +use std::rc::Rc; +use std::time::Duration; + +use hydro_lang::*; +use tokio::time::Instant; + +pub struct Client {} + +pub fn bench_client<'a>( + clients: &Cluster<'a, Client>, + transaction_cycle: impl FnOnce( + Stream<(u32, u32), Cluster<'a, Client>, Unbounded>, + ) -> Stream<(u32, u32), Cluster<'a, Client>, Unbounded, NoOrder>, + num_clients_per_node: usize, + median_latency_window_size: usize, +) { + let client_tick = clients.tick(); + // 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 c_new_payloads_on_start = start_this_tick.clone().flat_map_ordered(q!(move |_| (0 + ..num_clients_per_node) + .map(move |i| ( + (CLUSTER_SELF_ID.raw_id * (num_clients_per_node as u32)) + i as u32, + 0 + )))); + + let (c_to_proposers_complete_cycle, c_to_proposers) = + clients.forward_ref::>(); + let c_received_quorum_payloads = unsafe { + // SAFETY: because the transaction processor is required to handle arbitrary reordering + // across *different* keys, we are safe because delaying a transaction result for a key + // will only affect when the next request for that key is emitted with respect to other + // keys + transaction_cycle(c_to_proposers) + .timestamped(&client_tick) + .tick_batch() + }; + + // Whenever all replicas confirm that a payload was committed, send another payload + let c_new_payloads_when_committed = c_received_quorum_payloads + .clone() + .map(q!(|payload| (payload.0, payload.1 + 1))); + c_to_proposers_complete_cycle.complete( + c_new_payloads_on_start + .chain(unsafe { + // SAFETY: we don't send a new write for the same key until the previous one is committed, + // so this contains only a single write per key, and we don't care about order + // across keys + c_new_payloads_when_committed.assume_ordering::() + }) + .all_ticks() + .drop_timestamp(), + ); + + // Track statistics + let (c_timers_complete_cycle, c_timers) = + client_tick.cycle::>(); + let c_new_timers_when_leader_elected = start_this_tick + .map(q!(|_| Instant::now())) + .flat_map_ordered(q!( + move |now| (0..num_clients_per_node).map(move |virtual_id| (virtual_id, now)) + )); + let c_updated_timers = c_received_quorum_payloads + .clone() + .map(q!(|(key, _prev_count)| (key as usize, Instant::now()))); + let c_new_timers = c_timers + .clone() // Update c_timers in tick+1 so we can record differences during this tick (to track latency) + .chain(c_new_timers_when_leader_elected) + .chain(c_updated_timers.clone()) + .reduce_keyed_commutative(q!(|curr_time, new_time| { + if new_time > *curr_time { + *curr_time = new_time; + } + })); + c_timers_complete_cycle.complete_next_tick(c_new_timers); + + let c_stats_output_timer = unsafe { + // SAFETY: intentionally sampling statistics + clients + .source_interval(q!(Duration::from_secs(1))) + .timestamped(&client_tick) + .tick_batch() + } + .first(); + + let c_latency_reset = c_stats_output_timer.clone().map(q!(|_| None)).defer_tick(); + + let c_latencies = c_timers + .join(c_updated_timers) + .map(q!(|(_virtual_id, (prev_time, curr_time))| Some( + curr_time.duration_since(prev_time) + ))) + .chain(c_latency_reset.into_stream()) + .all_ticks() + .flatten_ordered() + .fold_commutative( + // Create window with ring buffer using vec + wraparound index + // TODO: Would be nice if I could use vec![] instead, but that doesn't work in Hydro with RuntimeData *median_latency_window_size + q!(move || ( + Rc::new(RefCell::new(Vec::::with_capacity( + median_latency_window_size + ))), + 0usize, + )), + q!(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); + } + // Increment write index and wrap around + *write_index = (*write_index + 1) % median_latency_window_size; + }), + ) + .map(q!(|(latencies, _)| latencies)); + + let c_throughput_new_batch = c_received_quorum_payloads + .clone() + .count() + .continue_unless(c_stats_output_timer.clone()) + .map(q!(|batch_size| (batch_size, false))); + + let c_throughput_reset = c_stats_output_timer + .clone() + .map(q!(|_| (0, true))) + .defer_tick(); + + let c_throughput = c_throughput_new_batch + .union(c_throughput_reset) + .all_ticks() + .fold( + q!(|| 0), + q!(|total, (batch_size, reset)| { + if reset { + *total = 0; + } else { + *total += batch_size; + } + }), + ); + + unsafe { + // SAFETY: intentionally sampling statistics + c_latencies.zip(c_throughput).latest_tick() + } + .continue_if(c_stats_output_timer) + .all_ticks() + .for_each(q!(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); + })); + // End track statistics +} diff --git a/hydro_test/src/cluster/paxos_kv.rs b/hydro_test/src/cluster/kv_replica.rs similarity index 73% rename from hydro_test/src/cluster/paxos_kv.rs rename to hydro_test/src/cluster/kv_replica.rs index 8a072017ef83..946a618182bd 100644 --- a/hydro_test/src/cluster/paxos_kv.rs +++ b/hydro_test/src/cluster/kv_replica.rs @@ -6,8 +6,6 @@ use hydro_lang::*; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; -use super::paxos::{paxos_core, Acceptor, Ballot, Proposer}; - pub struct Replica {} pub trait KvKey: Serialize + DeserializeOwned + Hash + Eq + Clone + Debug {} @@ -41,70 +39,22 @@ impl PartialOrd for SequencedKv { } } -/// Sets up a linearizable key-value store using Paxos. -/// -/// # Safety -/// Notifications for leader election are non-deterministic. When the leader is changing, -/// writes may be dropped by the old leader. -#[expect( - clippy::type_complexity, - clippy::too_many_arguments, - reason = "internal paxos code // TODO" -)] -pub unsafe fn paxos_kv<'a, K: KvKey, V: KvValue>( - proposers: &Cluster<'a, Proposer>, - acceptors: &Cluster<'a, Acceptor>, - replicas: &Cluster<'a, Replica>, - c_to_proposers: Stream, Cluster<'a, Proposer>, Unbounded>, - f: usize, - i_am_leader_send_timeout: u64, - i_am_leader_check_timeout: u64, - i_am_leader_check_timeout_delay_multiplier: usize, - checkpoint_frequency: usize, -) -> ( - Stream, Unbounded>, - Stream, Cluster<'a, Replica>, Unbounded>, -) { - let (r_to_acceptors_checkpoint_complete_cycle, r_to_acceptors_checkpoint) = - replicas.forward_ref::>(); - - let (p_to_clients_new_leader_elected, p_to_replicas) = unsafe { - // SAFETY: Leader election non-determinism and non-deterministic dropping of writes is documented. - paxos_core( - proposers, - acceptors, - r_to_acceptors_checkpoint.broadcast_bincode(acceptors), - c_to_proposers, - f, - i_am_leader_send_timeout, - i_am_leader_check_timeout, - i_am_leader_check_timeout_delay_multiplier, - ) - }; - - let (r_to_acceptors_checkpoint_new, r_new_processed_payloads) = replica( - replicas, - p_to_replicas - .map(q!(|(slot, kv)| SequencedKv { seq: slot, kv })) - .broadcast_bincode_interleaved(replicas), - checkpoint_frequency, - ); - - r_to_acceptors_checkpoint_complete_cycle.complete(r_to_acceptors_checkpoint_new); - - (p_to_clients_new_leader_elected, r_new_processed_payloads) -} - // Replicas. All relations for replicas will be prefixed with r. Expects ReplicaPayload on p_to_replicas, outputs a stream of (client address, ReplicaPayload) after processing. #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -pub fn replica<'a, K: KvKey, V: KvValue>( +pub fn kv_replica<'a, K: KvKey, V: KvValue>( replicas: &Cluster<'a, Replica>, - p_to_replicas: Stream, Cluster<'a, Replica>, Unbounded, NoOrder>, + p_to_replicas: impl Into< + Stream<(usize, Option>), Cluster<'a, Replica>, Unbounded, NoOrder>, + >, checkpoint_frequency: usize, ) -> ( Stream, Unbounded>, Stream, Cluster<'a, Replica>, Unbounded>, ) { + let p_to_replicas = p_to_replicas + .into() + .map(q!(|(slot, kv)| SequencedKv { seq: slot, kv })); + let replica_tick = replicas.tick(); let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = replica_tick.cycle(); @@ -116,7 +66,7 @@ pub fn replica<'a, K: KvKey, V: KvValue>( .timestamped(&replica_tick) .tick_batch() } - .union(r_buffered_payloads) // Combine with all payloads that we've received and not processed yet + .chain(r_buffered_payloads) // Combine with all payloads that we've received and not processed yet .sort(); // Create a cycle since we'll use this seq before we define it let (r_highest_seq_complete_cycle, r_highest_seq) = diff --git a/hydro_test/src/cluster/mod.rs b/hydro_test/src/cluster/mod.rs index b4e5be9bb006..9b3e74eff9dc 100644 --- a/hydro_test/src/cluster/mod.rs +++ b/hydro_test/src/cluster/mod.rs @@ -1,8 +1,10 @@ +pub mod bench_client; pub mod compute_pi; +pub mod kv_replica; pub mod many_to_many; pub mod map_reduce; pub mod paxos; pub mod paxos_bench; -pub mod paxos_kv; +pub mod paxos_with_client; pub mod simple_cluster; pub mod two_pc; diff --git a/hydro_test/src/cluster/paxos.rs b/hydro_test/src/cluster/paxos.rs index 93f4f0bb06f5..2d40915ea9fa 100644 --- a/hydro_test/src/cluster/paxos.rs +++ b/hydro_test/src/cluster/paxos.rs @@ -12,6 +12,18 @@ use serde::{Deserialize, Serialize}; pub struct Proposer {} pub struct Acceptor {} +#[derive(Clone, Copy)] +pub struct PaxosConfig { + /// Maximum number of faulty nodes + pub f: usize, + /// How often to send "I am leader" heartbeats + pub i_am_leader_send_timeout: u64, + /// How often to check if the leader has expired + pub i_am_leader_check_timeout: u64, + /// Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts + pub i_am_leader_check_timeout_delay_multiplier: usize, +} + pub trait PaxosPayload: Serialize + DeserializeOwned + PartialEq + Eq + Clone + Debug {} impl PaxosPayload for T {} @@ -64,11 +76,7 @@ struct P2a

{ /// in deterministic order. However, when the leader is changing, payloads may be /// non-deterministically dropped. The stream of ballots is also non-deterministic because /// leaders are elected in a non-deterministic process. -#[expect( - clippy::too_many_arguments, - clippy::type_complexity, - reason = "internal paxos code // TODO" -)] +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] pub unsafe fn paxos_core<'a, P: PaxosPayload, R>( proposers: &Cluster<'a, Proposer>, acceptors: &Cluster<'a, Acceptor>, @@ -78,15 +86,20 @@ pub unsafe fn paxos_core<'a, P: PaxosPayload, R>( Unbounded, NoOrder, >, - c_to_proposers: Stream, Unbounded>, - f: usize, - i_am_leader_send_timeout: u64, - i_am_leader_check_timeout: u64, - i_am_leader_check_timeout_delay_multiplier: usize, + c_to_proposers: impl FnOnce( + Stream, Unbounded>, + ) -> Stream, Unbounded>, + config: PaxosConfig, ) -> ( Stream, Unbounded>, Stream<(usize, Option

), Cluster<'a, Proposer>, Unbounded, NoOrder>, ) { + let f = config.f; + let i_am_leader_send_timeout = config.i_am_leader_send_timeout; + let i_am_leader_check_timeout = config.i_am_leader_check_timeout; + let i_am_leader_check_timeout_delay_multiplier = + config.i_am_leader_check_timeout_delay_multiplier; + proposers .source_iter(q!(["Proposers say hello"])) .for_each(q!(|s| println!("{}", s))); @@ -127,6 +140,14 @@ pub unsafe fn paxos_core<'a, P: PaxosPayload, R>( .clone() .continue_unless(p_is_leader.clone().defer_tick()); + let c_to_proposers = c_to_proposers( + just_became_leader + .clone() + .then(p_ballot.clone()) + .all_ticks() + .drop_timestamp(), + ); + let (p_to_replicas, a_log, sequencing_max_ballots) = unsafe { // SAFETY: The relevant p1bs are non-deterministic because they come from a arbitrary quorum, but because // we use a quorum, if we remain the leader there are no missing committed values when we combine the logs. @@ -485,8 +506,9 @@ fn recommit_after_leader_election<'a, P: PaxosPayload>( ) { let p_p1b_max_checkpoint = accepted_logs .clone() - .map(q!(|(checkpoint, _log)| checkpoint)) - .max(); + .filter_map(q!(|(checkpoint, _log)| checkpoint)) + .max() + .into_singleton(); let p_p1b_highest_entries_and_count = accepted_logs .map(q!(|(_checkpoint, log)| log)) .flatten_unordered() // Convert HashMap log back to stream @@ -555,7 +577,7 @@ fn recommit_after_leader_election<'a, P: PaxosPayload>( value: None })); - (p_log_to_try_commit.union(p_log_holes), p_max_slot) + (p_log_to_try_commit.chain(p_log_holes), p_max_slot) } #[expect( @@ -618,7 +640,7 @@ unsafe fn sequence_payload<'a, P: PaxosPayload, R>( (slot, ballot), Some(payload) ))) - .union(p_log_to_recommit.map(q!(|p2a| ((p2a.slot, p2a.ballot), p2a.value)))) + .chain(p_log_to_recommit.map(q!(|p2a| ((p2a.slot, p2a.ballot), p2a.value)))) .continue_if(p_is_leader) .all_ticks(); @@ -769,7 +791,7 @@ fn acceptor_p2<'a, P: PaxosPayload, R>( } )); let a_log = a_p2as_to_place_in_log - .union(a_new_checkpoint.into_stream()) + .chain(a_new_checkpoint.into_stream()) .all_ticks() .fold_commutative( q!(|| (None, HashMap::new())), diff --git a/hydro_test/src/cluster/paxos_bench.rs b/hydro_test/src/cluster/paxos_bench.rs index 52f6f5ebc752..1c06048491d5 100644 --- a/hydro_test/src/cluster/paxos_bench.rs +++ b/hydro_test/src/cluster/paxos_bench.rs @@ -1,28 +1,17 @@ -use std::cell::RefCell; -use std::rc::Rc; -use std::time::Duration; - use hydro_lang::*; use hydro_std::quorum::collect_quorum; -use tokio::time::Instant; - -use super::paxos::{Acceptor, Ballot, Proposer}; -use super::paxos_kv::{paxos_kv, KvPayload, Replica}; -pub struct Client {} +use super::bench_client::{bench_client, Client}; +use super::kv_replica::{kv_replica, KvPayload, Replica}; +use super::paxos::{Acceptor, PaxosConfig, Proposer}; +use super::paxos_with_client::paxos_with_client; -// Important: By convention, all relations that represent booleans either have a single "true" value or nothing. -// This allows us to use the continue_if_exists() and continue_if_empty() operators as if they were if (true) and if (false) statements. -#[expect(clippy::too_many_arguments, reason = "internal paxos code // TODO")] pub fn paxos_bench<'a>( flow: &FlowBuilder<'a>, - f: usize, num_clients_per_node: usize, median_latency_window_size: usize, /* How many latencies to keep in the window for calculating the median */ checkpoint_frequency: usize, // How many sequence numbers to commit before checkpointing - i_am_leader_send_timeout: u64, // How often to heartbeat - i_am_leader_check_timeout: u64, // How often to check if heartbeat expired - i_am_leader_check_timeout_delay_multiplier: usize, /* Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts */ + paxos_config: PaxosConfig, ) -> ( Cluster<'a, Proposer>, Cluster<'a, Acceptor>, @@ -34,83 +23,40 @@ pub fn paxos_bench<'a>( let clients = flow.cluster::(); let replicas = flow.cluster::(); - let (new_leader_elected_complete, new_leader_elected) = - clients.forward_ref::>(); - - let client_tick = clients.tick(); - let cur_leader_id = new_leader_elected - .inspect(q!(|ballot| println!( - "Client notified that leader was elected: {:?}", - ballot - ))) - .max() - .map(q!(|ballot: Ballot| ballot.proposer_id)); - - let leader_changed = unsafe { - // SAFETY: we are okay if we miss a transient leader ID, because we - // will eventually get the latest one and can restart requests then - cur_leader_id - .clone() - .timestamped(&client_tick) - .latest_tick() - .delta() - .map(q!(|_| ())) - .all_ticks() - .drop_timestamp() - }; - bench_client( &clients, - leader_changed, |c_to_proposers| { - let to_proposers = unsafe { - // SAFETY: the risk here is that we send a batch of requests - // with a stale leader ID, but because the leader ID comes from the - // network there is no way to guarantee that it is up to date + let payloads = c_to_proposers.map(q!(move |(key, value)| KvPayload { + key, + // we use our ID as part of the value and use that so the replica only notifies us + value: (CLUSTER_SELF_ID, value) + })); - // TODO(shadaj): we should retry if we get an error due to sending - // to a stale leader - c_to_proposers - .timestamped(&client_tick) - .tick_batch() - .cross_singleton(cur_leader_id.timestamped(&client_tick).latest_tick()) - .all_ticks() - } - .map(q!(move |((key, value), leader_id)| ( - leader_id, - KvPayload { - key, - // we use our ID as part of the value and use that so the replica only notifies us - value: (CLUSTER_SELF_ID, value) - } - ))) - .send_bincode_interleaved(&proposers); + let (replica_checkpoint_complete, replica_checkpoint) = + replicas.forward_ref::>(); - let to_proposers = unsafe { + let sequenced_payloads = unsafe { // SAFETY: clients "own" certain keys, so interleaving elements from clients will not affect // the order of writes to the same key - to_proposers.assume_ordering() - }; - let (new_leader_elected, processed_payloads) = unsafe { - // SAFETY: Non-deterministic leader notifications are handled in `to_proposers`. We do not - // care about the order in which key writes are processed, which is the non-determinism in - // `processed_payloads`. - paxos_kv( + // TODO(shadaj): we should retry when a payload is dropped due to stale leader + paxos_with_client( &proposers, &acceptors, - &replicas, - to_proposers, - f, - i_am_leader_send_timeout, - i_am_leader_check_timeout, - i_am_leader_check_timeout_delay_multiplier, - checkpoint_frequency, + &clients, + payloads, + replica_checkpoint.broadcast_bincode(&acceptors), + paxos_config, ) }; - new_leader_elected_complete - .complete(new_leader_elected.broadcast_bincode_interleaved(&clients)); + let sequenced_to_replicas = sequenced_payloads.broadcast_bincode_interleaved(&replicas); + + // Replicas + let (replica_checkpoint, processed_payloads) = + kv_replica(&replicas, sequenced_to_replicas, checkpoint_frequency); + + replica_checkpoint_complete.complete(replica_checkpoint); let c_received_payloads = processed_payloads .map(q!(|payload| ( @@ -120,13 +66,13 @@ pub fn paxos_bench<'a>( .send_bincode_interleaved(&clients); // we only mark a transaction as committed when all replicas have applied it - let (c_quorum_payloads, _) = collect_quorum::<_, _, _, ()>( - c_received_payloads.timestamped(&client_tick), - f + 1, - f + 1, - ); - - c_quorum_payloads.drop_timestamp() + collect_quorum::<_, _, _, ()>( + c_received_payloads.timestamped(&clients.tick()), + paxos_config.f + 1, + paxos_config.f + 1, + ) + .0 + .drop_timestamp() }, num_clients_per_node, median_latency_window_size, @@ -135,180 +81,28 @@ pub fn paxos_bench<'a>( (proposers, acceptors, clients, replicas) } -fn bench_client<'a>( - clients: &Cluster<'a, Client>, - trigger_restart: Stream<(), Cluster<'a, Client>, Unbounded>, - transaction_cycle: impl FnOnce( - Stream<(u32, u32), Cluster<'a, Client>, Unbounded>, - ) -> Stream<(u32, u32), Cluster<'a, Client>, Unbounded, NoOrder>, - num_clients_per_node: usize, - median_latency_window_size: usize, -) { - let client_tick = clients.tick(); - // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); - - // Whenever the leader changes, make all clients send a message - let restart_this_tick = unsafe { - // SAFETY: non-deterministic delay in restarting requests - // is okay because once it is restarted statistics should reach - // steady state regardless of when the restart happes - trigger_restart - .timestamped(&client_tick) - .tick_batch() - .last() - }; - - let c_new_payloads_when_restart = restart_this_tick.clone().flat_map_ordered(q!(move |_| (0 - ..num_clients_per_node) - .map(move |i| ( - (CLUSTER_SELF_ID.raw_id * (num_clients_per_node as u32)) + i as u32, - 0 - )))); - - let (c_to_proposers_complete_cycle, c_to_proposers) = - clients.forward_ref::>(); - let c_received_quorum_payloads = unsafe { - // SAFETY: because the transaction processor is required to handle arbitrary reordering - // across *different* keys, we are safe because delaying a transaction result for a key - // will only affect when the next request for that key is emitted with respect to other - // keys - transaction_cycle(c_to_proposers) - .timestamped(&client_tick) - .tick_batch() - }; - - // Whenever all replicas confirm that a payload was committed, send another payload - let c_new_payloads_when_committed = c_received_quorum_payloads - .clone() - .map(q!(|payload| (payload.0, payload.1 + 1))); - c_to_proposers_complete_cycle.complete( - c_new_payloads_when_restart - .chain(unsafe { - // SAFETY: we don't send a new write for the same key until the previous one is committed, - // so this contains only a single write per key, and we don't care about order - // across keys - c_new_payloads_when_committed.assume_ordering() - }) - .all_ticks() - .drop_timestamp(), - ); - - // Track statistics - let (c_timers_complete_cycle, c_timers) = - client_tick.cycle::>(); - let c_new_timers_when_leader_elected = restart_this_tick - .map(q!(|_| Instant::now())) - .flat_map_ordered(q!( - move |now| (0..num_clients_per_node).map(move |virtual_id| (virtual_id, now)) - )); - let c_updated_timers = c_received_quorum_payloads - .clone() - .map(q!(|(key, _prev_count)| (key as usize, Instant::now()))); - let c_new_timers = c_timers - .clone() // Update c_timers in tick+1 so we can record differences during this tick (to track latency) - .union(c_new_timers_when_leader_elected) - .union(c_updated_timers.clone()) - .reduce_keyed_commutative(q!(|curr_time, new_time| { - if new_time > *curr_time { - *curr_time = new_time; - } - })); - c_timers_complete_cycle.complete_next_tick(c_new_timers); - - let c_stats_output_timer = unsafe { - // SAFETY: intentionally sampling statistics - clients - .source_interval(q!(Duration::from_secs(1))) - .timestamped(&client_tick) - .tick_batch() - } - .first(); - - let c_latency_reset = c_stats_output_timer.clone().map(q!(|_| None)).defer_tick(); - - let c_latencies = c_timers - .join(c_updated_timers) - .map(q!(|(_virtual_id, (prev_time, curr_time))| Some( - curr_time.duration_since(prev_time) - ))) - .union(c_latency_reset.into_stream()) - .all_ticks() - .flatten_ordered() - .fold_commutative( - // Create window with ring buffer using vec + wraparound index - // TODO: Would be nice if I could use vec![] instead, but that doesn't work in Hydro with RuntimeData *median_latency_window_size - q!(move || ( - Rc::new(RefCell::new(Vec::::with_capacity( - median_latency_window_size - ))), - 0usize, - )), - q!(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); - } - // Increment write index and wrap around - *write_index = (*write_index + 1) % median_latency_window_size; - }), - ) - .map(q!(|(latencies, _)| latencies)); - - let c_throughput_new_batch = c_received_quorum_payloads - .clone() - .count() - .continue_unless(c_stats_output_timer.clone()) - .map(q!(|batch_size| (batch_size, false))); - - let c_throughput_reset = c_stats_output_timer - .clone() - .map(q!(|_| (0, true))) - .defer_tick(); - - let c_throughput = c_throughput_new_batch - .union(c_throughput_reset) - .all_ticks() - .fold( - q!(|| 0), - q!(|total, (batch_size, reset)| { - if reset { - *total = 0; - } else { - *total += batch_size; - } - }), - ); - - unsafe { - // SAFETY: intentionally sampling statistics - c_latencies.zip(c_throughput).latest_tick() - } - .continue_if(c_stats_output_timer) - .all_ticks() - .for_each(q!(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); - })); - // End track statistics -} - #[cfg(test)] mod tests { use hydro_lang::deploy::DeployRuntime; use stageleft::RuntimeData; + use crate::cluster::paxos::PaxosConfig; + #[test] fn paxos_ir() { let builder = hydro_lang::FlowBuilder::new(); - let _ = super::paxos_bench(&builder, 1, 1, 1, 1, 1, 1, 1); + let _ = super::paxos_bench( + &builder, + 1, + 1, + 1, + PaxosConfig { + f: 1, + i_am_leader_send_timeout: 1, + i_am_leader_check_timeout: 1, + i_am_leader_check_timeout_delay_multiplier: 1, + }, + ); let built = builder.with_default_optimize::(); hydro_lang::ir::dbg_dedup_tee(|| { diff --git a/hydro_test/src/cluster/paxos_with_client.rs b/hydro_test/src/cluster/paxos_with_client.rs new file mode 100644 index 000000000000..f850ed088e36 --- /dev/null +++ b/hydro_test/src/cluster/paxos_with_client.rs @@ -0,0 +1,75 @@ +use hydro_lang::*; + +use super::paxos::{paxos_core, Acceptor, Ballot, PaxosConfig, PaxosPayload, Proposer}; + +/// Wraps the core Paxos algorithm with logic to send payloads from clients to the current +/// leader. +/// +/// # Safety +/// Clients may send payloads to a stale leader if the leader changes between the time the +/// payload is sent and the time it is processed. This will result in the payload being dropped. +/// Payloads sent from multiple clients may be interleaved in a non-deterministic order. +pub unsafe fn paxos_with_client<'a, C: 'a, R, P: PaxosPayload>( + proposers: &Cluster<'a, Proposer>, + acceptors: &Cluster<'a, Acceptor>, + clients: &Cluster<'a, C>, + payloads: Stream, Unbounded>, + replica_checkpoint: Stream<(ClusterId, usize), Cluster<'a, Acceptor>, Unbounded, NoOrder>, + paxos_config: PaxosConfig, +) -> Stream<(usize, Option

), Cluster<'a, Proposer>, Unbounded, NoOrder> { + unsafe { + // SAFETY: Non-deterministic leader notifications are handled in `cur_leader_id`. We do not + // care about the order in which key writes are processed, which is the non-determinism in + // `sequenced_payloads`. + + paxos_core( + proposers, + acceptors, + replica_checkpoint, + |new_leader_elected| { + let cur_leader_id = new_leader_elected + .broadcast_bincode_interleaved(clients) + .inspect(q!(|ballot| println!( + "Client notified that leader was elected: {:?}", + ballot + ))) + .max() + .map(q!(|ballot: Ballot| ballot.proposer_id)); + + let payloads_at_proposer = { + // SAFETY: the risk here is that we send a batch of requests + // with a stale leader ID, but because the leader ID comes from the + // network there is no way to guarantee that it is up to date. This + // is documented non-determinism. + + let client_tick = clients.tick(); + let payload_batch = payloads.timestamped(&client_tick).tick_batch(); + + let latest_leader = cur_leader_id.timestamped(&client_tick).latest_tick(); + + let (unsent_payloads_complete, unsent_payloads) = + client_tick.cycle::>(); + + let all_payloads = unsent_payloads.chain(payload_batch); + + unsent_payloads_complete.complete_next_tick( + all_payloads.clone().continue_unless(latest_leader.clone()), + ); + + all_payloads.cross_singleton(latest_leader).all_ticks() + } + .map(q!(move |(payload, leader_id)| (leader_id, payload))) + .send_bincode_interleaved(proposers); + + let payloads_at_proposer = { + // SAFETY: documented non-determinism in interleaving of client payloads + payloads_at_proposer.assume_ordering() + }; + + payloads_at_proposer + }, + paxos_config, + ) + .1 + } +} 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 ebf8e1ad2786..74c0e31cdf69 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 @@ -22,7 +22,7 @@ expression: built.ir() | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < () > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }, ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < () > > > (__hydro_lang_cluster_ids_0) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < () >] > (__hydro_lang_cluster_ids_0) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Source { source: Iter( { use crate :: __staged :: cluster :: many_to_many :: * ; 0 .. 2 }, 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 1d18a502c2c7..64648d904d46 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 @@ -50,7 +50,7 @@ expression: built.ir() | res | { hydro_lang :: runtime_support :: bincode :: deserialize :: < std :: string :: String > (& res . unwrap ()) . unwrap () }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , std :: string :: String) > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > > > (__hydro_lang_cluster_ids_1) } ; | (i , w) | (ids__free [i % ids__free . len ()] , w) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , std :: string :: String) > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker >] > (__hydro_lang_cluster_ids_1) } ; | (i , w) | (ids__free [i % ids__free . len ()] , w) }), input: Enumerate { is_static: true, input: Map { diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap index e45bca3cc7be..697ebd1f3180 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap @@ -5,7 +5,7 @@ expression: ir.surface_syntax_string() 1v1 = source_iter ({ use crate :: __staged :: cluster :: map_reduce :: * ; vec ! ["abc" , "abc" , "xyz" , "abc"] }); 2v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < & str , std :: string :: String > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | s | s . to_string () })); 3v1 = enumerate :: < 'static > (); -4v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , std :: string :: String) > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > > > (__hydro_lang_cluster_ids_1) } ; | (i , w) | (ids__free [i % ids__free . len ()] , w) })); +4v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , std :: string :: String) > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker >] > (__hydro_lang_cluster_ids_1) } ; | (i , w) | (ids__free [i % ids__free . len ()] , w) })); 5v1 = map (| (id , data) : (hydro_lang :: ClusterId < _ > , std :: string :: String) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < std :: string :: String > (& data) . unwrap () . into ()) }); 6v1 = dest_sink ({ use hydro_lang :: __staged :: deploy_runtime :: * ; let env__free = FAKE ; let p1_port__free = "port_0" ; { env__free . port (p1_port__free) . connect_local_blocking :: < ConnectedDemux < ConnectedDirect > > () . into_sink () } }); 7v1 = source_stream ({ use hydro_lang :: __staged :: deploy_runtime :: * ; let env__free = FAKE ; let p2_port__free = "port_1" ; { env__free . port (p2_port__free) . connect_local_blocking :: < ConnectedTagged < ConnectedDirect > > () . into_source () } }); 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 d53ab94f81b1..eb4f676ebc95 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 @@ -30,7 +30,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -93,7 +93,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -142,7 +142,7 @@ expression: built.ir() | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }, ), input: FlatMap { - 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 :: < _ , & :: std :: vec :: Vec < 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))) }), + 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( @@ -165,7 +165,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -198,7 +198,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -210,7 +210,7 @@ expression: built.ir() 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 ; } } }), + 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 { @@ -218,7 +218,7 @@ expression: built.ir() sym: cycle_5, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -226,9 +226,9 @@ expression: built.ir() }, 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), + 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 { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , (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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) , (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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , (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 >)) , (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 hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 1, @@ -239,14 +239,14 @@ expression: built.ir() ), to_key: None, serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , (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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < (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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > (& data) . unwrap () . into ()) }, + | (id , data) : (hydro_lang :: ClusterId < _ > , (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 >)) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < (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 >) > (& data) . unwrap () . into ()) }, ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < (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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < (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 >) > (& b) . unwrap ()) }, ), 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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) })) }), + 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 { @@ -269,7 +269,7 @@ expression: built.ir() | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }, ), input: FlatMap { - 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 :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + 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 :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor >] > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Inspect { 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 { @@ -365,7 +365,7 @@ expression: built.ir() sym: cycle_0, }, location_kind: Tick( - 3, + 2, Cluster( 1, ), @@ -398,7 +398,7 @@ expression: built.ir() sym: cycle_5, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -419,7 +419,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -429,19 +429,19 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), input: CrossSingleton( 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), + 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | ((quorum_ballot , quorum_accepted) , my_ballot) | if quorum_ballot == my_ballot { Some (quorum_accepted) } else { None } }), + 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 { - 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 ; } } }, + 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | logs , log | { logs . push (log) ; } }), + 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 { - 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >)) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (v) => Some ((key , v)) , Err (_) => None , } }), + 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 { @@ -459,7 +459,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -510,19 +510,145 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , ballot) | ballot }), input: 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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)) , } }), + 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: , }, }, }, }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Tick( + 7, + Cluster( + 2, + ), + ), + input: DeferTick( + 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 { + ident: Ident { + sym: cycle_1, + }, + location_kind: Tick( + 7, + Cluster( + 2, + ), + ), + }, + 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 { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + ), + }, + 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 }), + 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 , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: : Map { + 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 { + 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 }), + input: Network { + from_location: Cluster( + 0, + ), + from_key: None, + to_location: Cluster( + 2, + ), + to_key: None, + serialize_fn: Some( + | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }, + ), + instantiate_fn: , + deserialize_fn: Some( + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }, + ), + input: FlatMap { + 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 { + inner: , + }, + 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 { + inner: , + }, + 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 { + inner: , + }, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + ), + }, + ), + }, CycleSink { ident: Ident { sym: cycle_7, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -532,20 +658,20 @@ expression: built.ir() 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 { - inner: : Fold { + 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , usize) , (usize , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((index , payload) , base_slot) | (base_slot + index , payload) }), + 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 { is_static: false, input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , ()) , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), + 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 { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + 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( 2, @@ -556,43 +682,20 @@ expression: built.ir() ), to_key: None, serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& data) . unwrap () . into ()) }, + | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > (& data) . unwrap () . into ()) }, ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > (& b) . unwrap ()) }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; move | ((key , value) , leader_id) | (leader_id , KvPayload { key , value : (CLUSTER_SELF_ID__free , value) }) }), + 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( - CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), + Tee { + inner: , }, Tee { - inner: : Map { - 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_bench :: * ; | 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 { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - ), - }, - }, + inner: , }, ), }, @@ -608,27 +711,27 @@ expression: built.ir() }, }, Tee { - inner: : Chain( + inner: : Chain( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), input: Tee { - inner: : Reduce { + 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 :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + 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: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >)) , (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , (count , entry)) | (slot , (count , entry . unwrap ())) }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (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) > > >)) , (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) > >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , (count , entry)) | (slot , (count , entry . unwrap ())) }), input: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , None) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { if let Some (curr_entry_payload) = & mut curr_entry . 1 { let same_values = new_entry . value == curr_entry_payload . value ; let higher_ballot = new_entry . ballot > curr_entry_payload . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry_payload . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry_payload . value = new_entry . value ; } } } else { * curr_entry = (1 , Some (new_entry)) ; } } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (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) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , None) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (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) > > >) , 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 :: * ; | curr_entry , new_entry | { if let Some (curr_entry_payload) = & mut curr_entry . 1 { let same_values = new_entry . value == curr_entry_payload . value ; let higher_ballot = new_entry . ballot > curr_entry_payload . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry_payload . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry_payload . value = new_entry . value ; } } } else { * curr_entry = (1 , Some (new_entry)) ; } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use hydro_lang :: __staged :: stream :: * ; | d | d }), + f: stageleft :: runtime_support :: fn1_type_hint :: < 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 :: 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 :: stream :: * ; | d | d }), input: Map { - 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_checkpoint , log) | log }), + 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) > > >) , 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 :: * ; | (_checkpoint , log) | log }), input: Tee { - inner: : FlatMap { - 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > > ({ use hydro_lang :: __staged :: optional :: * ; | v | v }), + inner: : FlatMap { + 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: , }, @@ -649,7 +752,7 @@ expression: built.ir() sym: cycle_7, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -674,7 +777,7 @@ expression: built.ir() }, }, Tee { - inner: , + inner: , }, ), }, @@ -685,7 +788,7 @@ expression: built.ir() sym: cycle_9, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -693,27 +796,27 @@ expression: built.ir() input: DeferTick( Difference( Tee { - inner: : FilterMap { + 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 { - inner: : FoldKeyed { + 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 < () , 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( + inner: : Chain( CycleSource { ident: Ident { sym: cycle_8, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), ), }, Tee { - inner: : Map { + 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 { from_location: Cluster( @@ -732,11 +835,11 @@ expression: built.ir() | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) > (& b) . unwrap ()) }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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) })) }), + 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 { - 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + 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 { from_location: Cluster( 0, @@ -747,26 +850,26 @@ expression: built.ir() ), to_key: None, serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > (& data) . unwrap () . into ()) }, + | (id , data) : (hydro_lang :: ClusterId < _ > , 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) > >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < 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) > > > (& data) . unwrap () . into ()) }, ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < 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) > > > (& b) . unwrap ()) }, ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + 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) > > , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor >] > (__hydro_lang_cluster_ids_1) } ; | 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 :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , ballot) , value) | P2a { ballot , slot , value } }), + 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) > >) , 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) , value) | P2a { ballot , slot , value } }), input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , ()) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), + 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 { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , payload) , ballot) | ((slot , ballot) , Some (payload)) }), + 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 { - inner: , + inner: , }, Tee { inner: , @@ -774,51 +877,66 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2a | ((p2a . slot , p2a . ballot) , p2a . value) }), + 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 { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < usize >) , core :: option :: Option < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 , }) } }), + 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 { - inner: , + inner: , }, Tee { inner: , }, ), Tee { - inner: : Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , core :: option :: Option < usize > , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Map { - 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (checkpoint , _log) | checkpoint }), - input: Tee { - inner: , + inner: : Chain( + 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: FilterMap { + 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: , + }, + }, }, }, - }, + Persist( + Source { + source: Iter( + [:: std :: option :: Option :: None], + ), + location_kind: Cluster( + 0, + ), + }, + ), + ), }, ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , ballot) | P2a { ballot , slot , value : None } }), + 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 { 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 { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + 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: , + inner: , }, }, ), @@ -859,10 +977,10 @@ expression: built.ir() }, }, Tee { - inner: : FilterMap { + 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: , + inner: , }, }, }, @@ -874,7 +992,7 @@ expression: built.ir() sym: cycle_8, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -882,10 +1000,10 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), ), @@ -895,7 +1013,7 @@ expression: built.ir() sym: cycle_10, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -903,38 +1021,38 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Chain( + inner: : Chain( CycleSource { ident: Ident { sym: cycle_10, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), ), }, Tee { - inner: , + inner: , }, ), }, 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 { + 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 { - inner: , + inner: , }, CycleSource { ident: Ident { sym: cycle_9, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -952,21 +1070,21 @@ expression: built.ir() sym: cycle_0, }, location_kind: Tick( - 3, + 2, Cluster( 1, ), ), 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 , } ,) ; } } } } }), + 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 { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), + 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 { - inner: , + inner: , }, Tee { inner: , @@ -974,17 +1092,17 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), + 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 { 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 :: paxos_kv :: Replica > , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + 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 :: paxos_kv :: Replica > , usize) , ()) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , usize) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), + 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 { - inner: : ReduceKeyed { + 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 { @@ -1001,10 +1119,10 @@ expression: built.ir() ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < usize > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: kv_replica :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < usize > (& b) . unwrap ()) }, ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor >] > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: CycleSource { ident: Ident { sym: cycle_0, @@ -1024,9 +1142,9 @@ expression: built.ir() 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 } }), 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 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , usize) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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: , + inner: , }, }, }, @@ -1053,7 +1171,7 @@ expression: built.ir() input: FilterMap { 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: , + inner: , }, }, }, @@ -1070,45 +1188,45 @@ expression: built.ir() ), input: DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), + 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 :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), + 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( + inner: : Sort( Chain( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - from_key: None, - to_location: Cluster( - 3, - ), - to_key: None, - serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& data) . unwrap () . into ()) }, - ), - instantiate_fn: , - deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& b) . unwrap ()) }, - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > > > (__hydro_lang_cluster_ids_3) } ; | 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 :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (slot , kv) | SequencedKv { seq : slot , kv } }), + 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 }), + input: Network { + from_location: Cluster( + 0, + ), + from_key: None, + to_location: Cluster( + 3, + ), + to_key: None, + serialize_fn: Some( + | (id , data) : (hydro_lang :: ClusterId < _ > , (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) > >)) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < (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) > >) > (& data) . unwrap () . into ()) }, + ), + instantiate_fn: , + deserialize_fn: Some( + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < (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) > >) > (& b) . unwrap ()) }, + ), + input: FlatMap { + 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) > >) , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica >] > (__hydro_lang_cluster_ids_3) } ; | 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 :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ())) , (usize , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , _ballot) , (value , _)) | (slot , value) }), + 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 :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ())) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ())) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , (meta , resp)) | (key , (meta , resp)) }), + 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 { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1132,14 +1250,14 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < usize > , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | v | v }), + 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 :: paxos_kv :: * ; | | None }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | 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) ; } } }), + 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 { - inner: , + inner: , }, Chain( Map { @@ -1188,23 +1306,23 @@ expression: built.ir() ), input: DeferTick( 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 :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (_kv_store , highest_seq) | highest_seq }), + 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 :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | | (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 :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (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) ; } }), + 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 { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), + 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 :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), + 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: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1228,8 +1346,8 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , usize) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; 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 } }), + 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 { @@ -1263,7 +1381,7 @@ expression: built.ir() ), ), Tee { - inner: , + inner: , }, ), }, @@ -1278,72 +1396,7 @@ expression: built.ir() 3, ), input: Tee { - inner: , - }, - }, - CycleSink { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - 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 }), - input: Network { - from_location: Cluster( - 0, - ), - from_key: None, - to_location: Cluster( - 2, - ), - to_key: None, - serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }, - ), - instantiate_fn: , - deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }, - ), - input: FlatMap { - 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_bench :: Client > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < hydro_test :: cluster :: paxos_bench :: 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 { - inner: , - }, - 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( - Tee { - inner: , - }, - 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 { - inner: , - }, - ), - }, - }, - }, - ), - }, - }, - ), - }, - }, - }, + inner: , }, }, CycleSink { @@ -1351,7 +1404,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Tick( - 0, + 9, Cluster( 2, ), @@ -1359,21 +1412,21 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Chain( + inner: : Chain( CycleSource { ident: Ident { sym: cycle_2, }, location_kind: Tick( - 0, + 9, Cluster( 2, ), ), }, Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , ((u32 , u32) , core :: result :: Result < () , () >)) , ((u32 , u32) , core :: result :: Result < () , () >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + 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 { from_location: Cluster( 3, @@ -1388,14 +1441,14 @@ expression: built.ir() ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < ((u32 , u32) , core :: result :: Result < () , () >) > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: kv_replica :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < ((u32 , u32) , core :: result :: Result < () , () >) > (& b) . unwrap ()) }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , ((u32 , u32) , core :: result :: Result < () , () >)) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . 0 , ((payload . key , payload . value . 1) , Ok (()))) }), + 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 :: bench_client :: Client > , ((u32 , u32) , core :: result :: Result < () , () >)) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . 0 , ((payload . key , payload . value . 1) , Ok (()))) }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | payload | payload . kv }), + 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: , + inner: , }, }, }, @@ -1405,14 +1458,14 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { + 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 { - inner: : FoldKeyed { + 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 < () , () > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { - inner: , + inner: , }, }, }, @@ -1423,33 +1476,30 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_1, + sym: cycle_0, }, location_kind: Cluster( 2, ), input: Chain( FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_bench :: 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)) }), + 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: : Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < () , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | * curr = new }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | () }), - input: Delta( - Tee { - inner: , - }, - ), - }, + inner: : Source { + source: Iter( + { use hydro_lang :: __staged :: location :: tick :: * ; let e__free = { use crate :: __staged :: cluster :: bench_client :: * ; () } ; [e__free] }, + ), + location_kind: Cluster( + 2, + ), }, }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . 0 , payload . 1 + 1) }), + 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: , + inner: : Tee { + inner: , }, }, }, @@ -1460,23 +1510,23 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 1, + 0, Cluster( 2, ), ), input: DeferTick( ReduceKeyed { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < tokio :: time :: Instant , tokio :: time :: Instant , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_time , new_time | { if new_time > * curr_time { * curr_time = new_time ; } } }), + 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 { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_3, }, location_kind: Tick( - 1, + 0, Cluster( 2, ), @@ -1484,20 +1534,20 @@ expression: built.ir() }, }, FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let num_clients_per_node__free = 1usize ; move | now | (0 .. num_clients_per_node__free) . map (move | virtual_id | (virtual_id , now)) }), + 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 :: paxos_bench :: * ; | _ | Instant :: now () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < () , tokio :: time :: Instant > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | Instant :: now () }), input: Tee { - inner: , + inner: , }, }, }, ), Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (usize , tokio :: time :: Instant) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (key , _prev_count) | (key as usize , Instant :: now ()) }), + 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: , + inner: , }, }, }, @@ -1506,38 +1556,38 @@ expression: built.ir() ), }, 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 :: paxos_bench :: * ; 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) ; } }), + 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 { - 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 :: paxos_bench :: * ; | (latencies , _) | latencies }), + 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 :: paxos_bench :: * ; 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 :: paxos_bench :: * ; 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 ; } }), + 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 { 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 { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (tokio :: time :: Instant , tokio :: time :: Instant)) , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (_virtual_id , (prev_time , curr_time)) | Some (curr_time . duration_since (prev_time)) }), + 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 { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), + 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 { + inner: : Source { source: Stream( - { use hydro_lang :: __staged :: location :: * ; let interval__free = { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval (interval__free)) }, + { use hydro_lang :: __staged :: location :: * ; let interval__free = { use crate :: __staged :: cluster :: bench_client :: * ; Duration :: from_secs (1) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval (interval__free)) }, ), location_kind: Cluster( 2, @@ -1552,12 +1602,12 @@ expression: built.ir() }, }, Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , bool) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | total , (batch_size , reset) | { if reset { * total = 0 ; } else { * total += batch_size ; } } }), + 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 { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | batch_size | (batch_size , false) }), + 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( @@ -1565,7 +1615,7 @@ expression: built.ir() 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: , + inner: , }, }, Map { @@ -1576,7 +1626,7 @@ expression: built.ir() init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1586,9 +1636,9 @@ expression: built.ir() }, DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , (usize , bool) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | (0 , true) }), input: Tee { - inner: , + inner: , }, }, ), @@ -1599,7 +1649,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ), 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 308e2cef810f..7c48e8e03c43 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 @@ -48,7 +48,7 @@ expression: built.ir() 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( - unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydro_lang :: ClusterId < () > > > (__hydro_lang_cluster_ids_1) }, + unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < () >] > (__hydro_lang_cluster_ids_1) }, ), location_kind: Process( 0, diff --git a/hydro_test/src/lib.rs b/hydro_test/src/lib.rs index 0054f03d4c5b..0eacd8499872 100644 --- a/hydro_test/src/lib.rs +++ b/hydro_test/src/lib.rs @@ -5,7 +5,16 @@ pub mod distributed; #[doc(hidden)] #[stageleft::runtime] +#[cfg(doctest)] mod docs { - #[doc = include_str!("../../docs/docs/hydro/consistency.md")] - mod consistency {} + dfir_macro::doctest_markdown_glob!("docs/docs/hydro/**/*.md*"); +} + +#[stageleft::runtime] +#[cfg(test)] +mod test_init { + #[ctor::ctor] + fn init() { + hydro_lang::deploy::init_test(); + } } diff --git a/hydro_test_local/src/local/graph_reachability.rs b/hydro_test_local/src/local/graph_reachability.rs index 2d361431c3aa..2728c55bf6d0 100644 --- a/hydro_test_local/src/local/graph_reachability.rs +++ b/hydro_test_local/src/local/graph_reachability.rs @@ -25,7 +25,7 @@ pub fn graph_reachability<'a>( roots .timestamped(&reachability_tick) .tick_batch() - .union(reached_cycle) + .chain(reached_cycle) }; let reachable = reached .clone() @@ -35,7 +35,7 @@ pub fn graph_reachability<'a>( edges.timestamped(&reachability_tick).tick_batch().persist() }) .map(q!(|(_from, (_, to))| to)); - set_reached_cycle.complete_next_tick(reached.clone().union(reachable)); + set_reached_cycle.complete_next_tick(reached.clone().chain(reachable)); reached.all_ticks().unique().for_each(q!(|v| { reached_out.send(v).unwrap(); diff --git a/lattices/src/with_bot.rs b/lattices/src/with_bot.rs index a30d8d3b1ad6..b5243fde63ec 100644 --- a/lattices/src/with_bot.rs +++ b/lattices/src/with_bot.rs @@ -138,7 +138,7 @@ where Inner: IsTop, { fn is_top(&self) -> bool { - self.0.as_ref().map_or(false, IsTop::is_top) + self.0.as_ref().is_some_and(IsTop::is_top) } } diff --git a/lattices/src/with_top.rs b/lattices/src/with_top.rs index d8c122a1e686..ef12965ee680 100644 --- a/lattices/src/with_top.rs +++ b/lattices/src/with_top.rs @@ -124,7 +124,7 @@ where Inner: IsBot, { fn is_bot(&self) -> bool { - self.0.as_ref().map_or(false, IsBot::is_bot) + self.0.as_ref().is_some_and(IsBot::is_bot) } } diff --git a/multiplatform_test/src/lib.rs b/multiplatform_test/src/lib.rs index a1826fc960a3..88021fb8b7cd 100644 --- a/multiplatform_test/src/lib.rs +++ b/multiplatform_test/src/lib.rs @@ -51,9 +51,7 @@ impl Platform { Platform::Tokio => quote! { #[tokio::test ] }, Platform::AsyncStd => quote! { #[async_std::test] }, Platform::Dfir => quote! { #[dfir_rs::test] }, - Platform::Wasm => { - quote! { #[wasm_bindgen_test::wasm_bindgen_test] } - } + Platform::Wasm => quote! { #[wasm_bindgen_test::wasm_bindgen_test] }, Platform::EnvLogging | Platform::EnvTracing => Default::default(), } } diff --git a/stageleft/src/lib.rs b/stageleft/src/lib.rs index b8c662be384e..7de48bef0409 100644 --- a/stageleft/src/lib.rs +++ b/stageleft/src/lib.rs @@ -54,16 +54,17 @@ macro_rules! stageleft_crate { "lib_macro.rs" )); - #[cfg(not(feature = "stageleft_devel"))] #[cfg(not(stageleft_macro))] #[doc(hidden)] #[allow( unused, ambiguous_glob_reexports, + unexpected_cfgs, clippy::suspicious_else_formatting, reason = "generated code" )] pub mod __staged { + #[cfg(not(feature = "stageleft_devel"))] include!(concat!( env!("OUT_DIR"), $crate::PATH_SEPARATOR!(), @@ -76,16 +77,17 @@ macro_rules! stageleft_crate { #[macro_export] macro_rules! stageleft_no_entry_crate { () => { - #[cfg(not(feature = "stageleft_devel"))] #[doc(hidden)] #[allow( unused, ambiguous_glob_reexports, + unexpected_cfgs, clippy::suspicious_else_formatting, clippy::type_complexity, reason = "generated code" )] pub mod __staged { + #[cfg(not(feature = "stageleft_devel"))] include!(concat!( env!("OUT_DIR"), $crate::PATH_SEPARATOR!(), diff --git a/stageleft/src/type_name.rs b/stageleft/src/type_name.rs index 55546afdb481..dc02417d44c4 100644 --- a/stageleft/src/type_name.rs +++ b/stageleft/src/type_name.rs @@ -17,6 +17,10 @@ static PRIVATE_REEXPORTS: ReexportsSet = LazyLock::new(|| { vec!["std", "collections", "hash", "map"], vec!["std", "collections", "hash_map"], ), + ( + vec!["std", "collections", "hash", "set"], + vec!["std", "collections", "hash_set"], + ), (vec!["std", "vec", "into_iter"], vec!["std", "vec"]), ]) }); diff --git a/stageleft_tool/src/lib.rs b/stageleft_tool/src/lib.rs index a925245d7eb3..6f1190fbe0ab 100644 --- a/stageleft_tool/src/lib.rs +++ b/stageleft_tool/src/lib.rs @@ -157,6 +157,22 @@ impl VisitMut for GenFinalPubVistor { syn::visit_mut::visit_use_path_mut(self, i); } + fn visit_vis_restricted_mut(&mut self, _i: &mut syn::VisRestricted) { + // don't treat the restriction as a path, we don't want to rewrite that to `__staged` + } + + fn visit_path_mut(&mut self, i: &mut syn::Path) { + if !i.segments.is_empty() && i.segments[0].ident == "crate" { + i.segments.insert( + 1, + syn::PathSegment { + ident: parse_quote!(__staged), + arguments: Default::default(), + }, + ); + } + } + fn visit_item_mod_mut(&mut self, i: &mut syn::ItemMod) { let is_runtime_or_test = i.attrs.iter().any(|a| { a.path().to_token_stream().to_string() == "stageleft :: runtime" @@ -321,7 +337,6 @@ pub fn gen_final_helper() { ) .unwrap(); - println!("cargo::rustc-check-cfg=cfg(stageleft_macro)"); println!("cargo::rerun-if-changed=build.rs"); println!("cargo::rerun-if-changed=src"); } @@ -329,7 +344,15 @@ pub fn gen_final_helper() { #[macro_export] macro_rules! gen_final { () => { - #[cfg(not(feature = "stageleft_devel"))] - $crate::gen_final_helper() + #[allow( + unexpected_cfgs, + reason = "Consumer crates may optionally add the `stageleft_devel` feature." + )] + { + println!("cargo::rustc-check-cfg=cfg(stageleft_macro)"); + + #[cfg(not(feature = "stageleft_devel"))] + $crate::gen_final_helper() + } }; } diff --git a/template/hydro/Cargo.toml b/template/hydro/Cargo.toml index 386d6f0d1764..7d3f480e54de 100644 --- a/template/hydro/Cargo.toml +++ b/template/hydro/Cargo.toml @@ -19,6 +19,7 @@ stageleft_tool = { git = "{{ hydro_git | default: 'https://github.com/hydro-proj [dev-dependencies] async-ssh2-lite = { version = "0.5.0", features = ["vendored-openssl"] } +ctor = "0.2" hydro_deploy = { git = "{{ hydro_git | default: 'https://github.com/hydro-project/hydro.git' }}", branch = "{{ hydro_branch | default: 'main' }}" } hydro_lang = { git = "{{ hydro_git | default: 'https://github.com/hydro-project/hydro.git' }}", branch = "{{ hydro_branch | default: 'main' }}", features = [ "deploy", diff --git a/template/hydro/src/lib.rs b/template/hydro/src/lib.rs index 4e3abdc4f4cf..d12b97d76249 100644 --- a/template/hydro/src/lib.rs +++ b/template/hydro/src/lib.rs @@ -3,3 +3,12 @@ stageleft::stageleft_no_entry_crate!(); pub mod first_ten; pub mod first_ten_cluster; pub mod first_ten_distributed; + +#[stageleft::runtime] +#[cfg(test)] +mod test_init { + #[ctor::ctor] + fn init() { + hydro_lang::deploy::init_test(); + } +}