From 536e6442d68b0947da5bfef9991825003e6867fc Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Sun, 25 Aug 2024 20:55:17 -0700 Subject: [PATCH] feat(hydroflow_plus)!: refactor API to have no-tick semantics by default (#1421) Now, by default streams exist at a "top-level" where there are no ticks and operators run over the entire collection. To perform iterative computations, developers must explicitly entire a tick domain (using `tick_batch`), and return to the outer domain (using `all_ticks`). --- hydroflow_plus/src/builder/mod.rs | 83 +- hydroflow_plus/src/cycle.rs | 25 +- hydroflow_plus/src/ir.rs | 6 + hydroflow_plus/src/persist_pullup.rs | 225 +- hydroflow_plus/src/profiler.rs | 5 +- hydroflow_plus/src/properties.rs | 2 + ...p__tests__persist_pullup_behind_tee-2.snap | 32 +- ...lup__tests__persist_pullup_behind_tee.snap | 68 +- ...ist_pullup_behind_tee@surface_graph_0.snap | 15 +- ...__tests__persist_pullup_through_map-2.snap | 22 +- ...up__tests__persist_pullup_through_map.snap | 28 +- ...st_pullup_through_map@surface_graph_0.snap | 5 +- ...ts__profiler_wrapping_all_operators-2.snap | 23 +- ...ests__profiler_wrapping_all_operators.snap | 30 +- hydroflow_plus/src/stream.rs | 274 +- hydroflow_plus_test/src/cluster/compute_pi.rs | 7 +- hydroflow_plus_test/src/cluster/map_reduce.rs | 11 +- hydroflow_plus_test/src/cluster/paxos.rs | 218 +- .../src/cluster/simple_cluster.rs | 2 + ...test__cluster__paxos__tests__paxos_ir.snap | 5416 +++++++++++++++++ ...simple_cluster__tests__simple_cluster.snap | 42 +- .../src/local/chat_app.rs | 9 +- .../src/local/compute_pi.rs | 3 +- .../src/local/count_elems.rs | 3 +- .../src/local/graph_reachability.rs | 8 +- .../src/local/negation.rs | 30 +- ...ity__tests__reachability@graphvis_dot.snap | 2 +- ..._tests__reachability@graphvis_mermaid.snap | 2 +- .../src/local/teed_join.rs | 6 +- 29 files changed, 6190 insertions(+), 412 deletions(-) create mode 100644 hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap diff --git a/hydroflow_plus/src/builder/mod.rs b/hydroflow_plus/src/builder/mod.rs index d9b6e1ce9b7d..88ff13c56142 100644 --- a/hydroflow_plus/src/builder/mod.rs +++ b/hydroflow_plus/src/builder/mod.rs @@ -5,6 +5,7 @@ use std::rc::Rc; use std::time::Duration; use hydroflow::futures::stream::Stream as FuturesStream; +use hydroflow::{tokio, tokio_stream}; use internal::TokenStream; use proc_macro2::Span; use quote::quote; @@ -13,7 +14,7 @@ use stageleft::*; use crate::ir::{HfPlusLeaf, HfPlusNode, HfPlusSource}; use crate::location::{Cluster, Location, LocationId, Process}; -use crate::stream::{Async, Windowed}; +use crate::stream::{Bounded, NoTick, Tick, Unbounded}; use crate::{HfCycle, RuntimeContext, Stream}; pub mod built; @@ -189,14 +190,14 @@ impl<'a> FlowBuilder<'a> { } } - pub fn spin(&self, on: &L) -> Stream<'a, (), Async, L> { + pub fn spin(&self, on: &L) -> Stream<'a, (), Unbounded, NoTick, L> { Stream::new( on.id(), self.ir_leaves().clone(), - HfPlusNode::Source { + HfPlusNode::Persist(Box::new(HfPlusNode::Source { source: HfPlusSource::Spin(), location_kind: on.id(), - }, + })), ) } @@ -204,7 +205,7 @@ impl<'a> FlowBuilder<'a> { &self, on: &L, batch_size: impl Quoted<'a, usize> + Copy + 'a, - ) -> Stream<'a, (), Windowed, L> { + ) -> Stream<'a, (), Bounded, Tick, L> { self.spin(on) .flat_map(q!(move |_| 0..batch_size)) .map(q!(|_| ())) @@ -215,16 +216,16 @@ impl<'a> FlowBuilder<'a> { &self, on: &L, e: impl Quoted<'a, E>, - ) -> Stream<'a, T, Async, L> { + ) -> Stream<'a, T, Unbounded, NoTick, L> { let e = e.splice(); Stream::new( on.id(), self.ir_leaves().clone(), - HfPlusNode::Source { + HfPlusNode::Persist(Box::new(HfPlusNode::Source { source: HfPlusSource::Stream(e.into()), location_kind: on.id(), - }, + })), ) } @@ -232,16 +233,16 @@ impl<'a> FlowBuilder<'a> { &self, on: &L, e: impl Quoted<'a, E>, - ) -> Stream<'a, T, Windowed, L> { + ) -> Stream<'a, T, Bounded, NoTick, L> { let e = e.splice(); Stream::new( on.id(), self.ir_leaves().clone(), - HfPlusNode::Source { + HfPlusNode::Persist(Box::new(HfPlusNode::Source { source: HfPlusSource::Iter(e.into()), location_kind: on.id(), - }, + })), ) } @@ -249,16 +250,16 @@ impl<'a> FlowBuilder<'a> { &self, on: &L, interval: impl Quoted<'a, Duration> + Copy + 'a, - ) -> Stream<'a, (), Async, L> { + ) -> Stream<'a, (), Unbounded, NoTick, L> { let interval = interval.splice(); Stream::new( on.id(), self.ir_leaves().clone(), - HfPlusNode::Source { + HfPlusNode::Persist(Box::new(HfPlusNode::Source { source: HfPlusSource::Interval(interval.into()), location_kind: on.id(), - }, + })), ) } @@ -267,19 +268,59 @@ impl<'a> FlowBuilder<'a> { on: &L, delay: impl Quoted<'a, Duration> + Copy + 'a, interval: impl Quoted<'a, Duration> + Copy + 'a, - ) -> Stream<'a, hydroflow::tokio::time::Instant, Async, L> { + ) -> Stream<'a, tokio::time::Instant, Unbounded, NoTick, L> { self.source_stream( on, - q!(hydroflow::tokio_stream::wrappers::IntervalStream::new( - hydroflow::tokio::time::interval_at( - hydroflow::tokio::time::Instant::now() + delay, - interval - ) + q!(tokio_stream::wrappers::IntervalStream::new( + tokio::time::interval_at(tokio::time::Instant::now() + delay, interval) )), ) } - pub fn cycle(&self, on: &L) -> (HfCycle<'a, T, W, L>, Stream<'a, T, W, L>) { + #[allow(clippy::type_complexity)] + pub fn cycle( + &self, + on: &L, + ) -> (HfCycle<'a, T, W, NoTick, L>, Stream<'a, T, W, NoTick, L>) { + let next_id = { + let on_id = match on.id() { + LocationId::Process(id) => id, + LocationId::Cluster(id) => id, + }; + + let mut cycle_ids = self.cycle_ids.borrow_mut(); + let next_id_entry = cycle_ids.entry(on_id).or_default(); + + let id = *next_id_entry; + *next_id_entry += 1; + id + }; + + let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); + + ( + HfCycle { + ident: ident.clone(), + location_kind: on.id(), + ir_leaves: self.ir_leaves().clone(), + _phantom: PhantomData, + }, + Stream::new( + on.id(), + self.ir_leaves().clone(), + HfPlusNode::Persist(Box::new(HfPlusNode::CycleSource { + ident, + location_kind: on.id(), + })), + ), + ) + } + + #[allow(clippy::type_complexity)] + pub fn tick_cycle( + &self, + on: &L, + ) -> (HfCycle<'a, T, W, Tick, L>, Stream<'a, T, W, Tick, L>) { let next_id = { let on_id = match on.id() { LocationId::Process(id) => id, diff --git a/hydroflow_plus/src/cycle.rs b/hydroflow_plus/src/cycle.rs index 5a6f82e78bcb..66a8f68cf827 100644 --- a/hydroflow_plus/src/cycle.rs +++ b/hydroflow_plus/src/cycle.rs @@ -1,23 +1,24 @@ use std::marker::PhantomData; use crate::builder::FlowLeaves; -use crate::ir::HfPlusLeaf; +use crate::ir::{HfPlusLeaf, HfPlusNode}; use crate::location::{Location, LocationId}; +use crate::stream::{NoTick, Tick}; use crate::Stream; /// Represents a fixpoint cycle in the graph that will be fulfilled /// by a stream that is not yet known. /// /// See [`Stream`] for an explainer on the type parameters. -pub struct HfCycle<'a, T, W, N: Location> { +pub struct HfCycle<'a, T, W, C, N: Location> { pub(crate) ident: syn::Ident, pub(crate) location_kind: LocationId, pub(crate) ir_leaves: FlowLeaves<'a>, - pub(crate) _phantom: PhantomData<(N, &'a mut &'a (), T, W)>, + pub(crate) _phantom: PhantomData<(N, &'a mut &'a (), T, W, C)>, } -impl<'a, T, W, N: Location> HfCycle<'a, T, W, N> { - pub fn complete(self, stream: Stream<'a, T, W, N>) { +impl<'a, T, W, N: Location> HfCycle<'a, T, W, Tick, N> { + pub fn complete(self, stream: Stream<'a, T, W, Tick, N>) { let ident = self.ident; self.ir_leaves.borrow_mut().as_mut().expect("Attempted to add a cycle to a flow that has already been finalized. No cycles can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { @@ -27,3 +28,17 @@ impl<'a, T, W, N: Location> HfCycle<'a, T, W, N> { }); } } + +impl<'a, T, W, N: Location> HfCycle<'a, T, W, NoTick, N> { + pub fn complete(self, stream: Stream<'a, T, W, NoTick, N>) { + let ident = self.ident; + + self.ir_leaves.borrow_mut().as_mut().expect("Attempted to add a cycle to a flow that has already been finalized. No cycles can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + ident, + location_kind: self.location_kind, + input: Box::new(HfPlusNode::Unpersist( + Box::new(stream.ir_node.into_inner()) + )), + }); + } +} diff --git a/hydroflow_plus/src/ir.rs b/hydroflow_plus/src/ir.rs index 8c76689640ee..5a78ba485e1d 100644 --- a/hydroflow_plus/src/ir.rs +++ b/hydroflow_plus/src/ir.rs @@ -226,6 +226,7 @@ pub enum HfPlusNode<'a> { }, Persist(Box>), + Unpersist(Box>), Delta(Box>), Union(Box>, Box>), @@ -519,6 +520,7 @@ impl<'a> HfPlusNode<'a> { } HfPlusNode::Persist(inner) => transform(inner.as_mut(), seen_tees), + HfPlusNode::Unpersist(inner) => transform(inner.as_mut(), seen_tees), HfPlusNode::Delta(inner) => transform(inner.as_mut(), seen_tees), HfPlusNode::Union(left, right) => { @@ -623,6 +625,10 @@ impl<'a> HfPlusNode<'a> { (persist_ident, location) } + HfPlusNode::Unpersist(_) => { + panic!("Unpersist is a marker node and should have been optimized away. This is likely a compiler bug.") + } + HfPlusNode::Delta(inner) => { let (inner_ident, location) = inner.emit(graph_builders, built_tees, next_stmt_id); diff --git a/hydroflow_plus/src/persist_pullup.rs b/hydroflow_plus/src/persist_pullup.rs index eab612c05dbb..45806b714026 100644 --- a/hydroflow_plus/src/persist_pullup.rs +++ b/hydroflow_plus/src/persist_pullup.rs @@ -1,31 +1,202 @@ +use std::cell::RefCell; +use std::collections::HashSet; +use std::ops::Deref; + use crate::ir::*; -fn persist_pullup_node<'a>(node: &mut HfPlusNode<'a>, seen_tees: &mut SeenTees<'a>) { - node.transform_children(persist_pullup_node, seen_tees); - if let HfPlusNode::Map { - f: _, - input: box HfPlusNode::Persist(_), - } = node - { - if let HfPlusNode::Map { - f, - input: box HfPlusNode::Persist(behind_persist), - } = std::mem::replace(node, HfPlusNode::Placeholder) - { - *node = HfPlusNode::Persist(Box::new(HfPlusNode::Map { +fn persist_pullup_node<'a>( + node: &mut HfPlusNode<'a>, + seen_tees: &mut SeenTees<'a>, + persist_pulled_tees: &mut HashSet<*const RefCell>>, +) { + node.transform_children( + |n, s| persist_pullup_node(n, s, persist_pulled_tees), + seen_tees, + ); + + match node { + HfPlusNode::Unpersist(box HfPlusNode::Persist(_)) => { + if let HfPlusNode::Unpersist(box HfPlusNode::Persist(box behind_persist)) = + std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = behind_persist; + } else { + unreachable!() + } + } + + HfPlusNode::Delta(box HfPlusNode::Persist(_)) => { + if let HfPlusNode::Delta(box HfPlusNode::Persist(box behind_persist)) = + std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = behind_persist; + } else { + unreachable!() + } + } + + HfPlusNode::Tee { inner } => { + if persist_pulled_tees.contains(&(inner.as_ref() as *const RefCell>)) { + *node = HfPlusNode::Persist(Box::new(HfPlusNode::Tee { + inner: inner.clone(), + })); + } else { + let inner_borrow = inner.borrow(); + if let HfPlusNode::Persist(_) = inner_borrow.deref() { + drop(inner_borrow); + persist_pulled_tees.insert(inner.as_ref() as *const RefCell>); + if let HfPlusNode::Persist(box behind_persist) = + inner.replace(HfPlusNode::Placeholder) + { + *inner.borrow_mut() = behind_persist; + } else { + unreachable!() + } + + *node = HfPlusNode::Persist(Box::new(HfPlusNode::Tee { + inner: inner.clone(), + })); + } + } + } + + HfPlusNode::Map { + f: _, + input: box HfPlusNode::Persist(_), + } => { + if let HfPlusNode::Map { + f, + input: box HfPlusNode::Persist(behind_persist), + } = std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = HfPlusNode::Persist(Box::new(HfPlusNode::Map { + f, + input: behind_persist, + })); + } else { + unreachable!() + } + } + + HfPlusNode::FlatMap { + f: _, + input: box HfPlusNode::Persist(_), + } => { + if let HfPlusNode::FlatMap { f, - input: behind_persist, - })); - } else { - unreachable!() + input: box HfPlusNode::Persist(behind_persist), + } = std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = HfPlusNode::Persist(Box::new(HfPlusNode::FlatMap { + f, + input: behind_persist, + })); + } else { + unreachable!() + } + } + + HfPlusNode::Filter { + f: _, + input: box HfPlusNode::Persist(_), + } => { + if let HfPlusNode::Filter { + f, + input: box HfPlusNode::Persist(behind_persist), + } = std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = HfPlusNode::Persist(Box::new(HfPlusNode::Filter { + f, + input: behind_persist, + })); + } else { + unreachable!() + } + } + + HfPlusNode::Network { + input: box HfPlusNode::Persist(_), + .. + } => { + if let HfPlusNode::Network { + from_location, + to_location, + serialize_pipeline, + instantiate_fn, + deserialize_pipeline, + input: box HfPlusNode::Persist(behind_persist), + } = std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = HfPlusNode::Persist(Box::new(HfPlusNode::Network { + from_location, + to_location, + serialize_pipeline, + instantiate_fn, + deserialize_pipeline, + input: behind_persist, + })); + } else { + unreachable!() + } + } + + HfPlusNode::Union(box HfPlusNode::Persist(_), box HfPlusNode::Persist(_)) => { + if let HfPlusNode::Union( + box HfPlusNode::Persist(left), + box HfPlusNode::Persist(right), + ) = std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = HfPlusNode::Persist(Box::new(HfPlusNode::Union(left, right))); + } else { + unreachable!() + } + } + + HfPlusNode::CrossProduct(box HfPlusNode::Persist(_), box HfPlusNode::Persist(_)) => { + if let HfPlusNode::CrossProduct( + box HfPlusNode::Persist(left), + box HfPlusNode::Persist(right), + ) = std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = HfPlusNode::Persist(Box::new(HfPlusNode::Delta(Box::new( + HfPlusNode::CrossProduct( + Box::new(HfPlusNode::Persist(left)), + Box::new(HfPlusNode::Persist(right)), + ), + )))); + } else { + unreachable!() + } + } + + HfPlusNode::Join(box HfPlusNode::Persist(_), box HfPlusNode::Persist(_)) => { + if let HfPlusNode::Join(box HfPlusNode::Persist(left), box HfPlusNode::Persist(right)) = + std::mem::replace(node, HfPlusNode::Placeholder) + { + *node = + HfPlusNode::Persist(Box::new(HfPlusNode::Delta(Box::new(HfPlusNode::Join( + Box::new(HfPlusNode::Persist(left)), + Box::new(HfPlusNode::Persist(right)), + ))))); + } else { + unreachable!() + } } + + _ => {} } } pub fn persist_pullup(ir: Vec) -> Vec { let mut seen_tees = Default::default(); + let mut persist_pulled_tees = Default::default(); ir.into_iter() - .map(|l| l.transform_children(persist_pullup_node, &mut seen_tees)) + .map(|l| { + l.transform_children( + |n, s| persist_pullup_node(n, s, &mut persist_pulled_tees), + &mut seen_tees, + ) + }) .collect() } @@ -41,7 +212,6 @@ mod tests { let process = flow.process::<()>(); flow.source_iter(&process, q!(0..10)) - .all_ticks() .map(q!(|v| v + 1)) .for_each(q!(|n| println!("{}", n))); @@ -64,14 +234,19 @@ mod tests { let flow = crate::builder::FlowBuilder::new(); let process = flow.process::<()>(); - let before_tee = flow - .source_iter(&process, q!(0..10)) - .all_ticks() - .map(q!(|v| v + 1)); + let before_tee = flow.source_iter(&process, q!(0..10)).tick_batch().persist(); - before_tee.clone().for_each(q!(|n| println!("{}", n))); + before_tee + .clone() + .map(q!(|v| v + 1)) + .all_ticks() + .for_each(q!(|n| println!("{}", n))); - before_tee.for_each(q!(|n| println!("{}", n))); + before_tee + .clone() + .map(q!(|v| v + 1)) + .all_ticks() + .for_each(q!(|n| println!("{}", n))); let built = flow.finalize(); diff --git a/hydroflow_plus/src/profiler.rs b/hydroflow_plus/src/profiler.rs index 853fbf5560eb..8adc5fcc7b6b 100644 --- a/hydroflow_plus/src/profiler.rs +++ b/hydroflow_plus/src/profiler.rs @@ -87,7 +87,6 @@ mod tests { let process = flow.process::<()>(); flow.source_iter(&process, q!(0..10)) - .all_ticks() .map(q!(|v| v + 1)) .for_each(q!(|n| println!("{}", n))); @@ -106,8 +105,8 @@ mod tests { let counter_queue = RuntimeData::new("Fake"); let pushed_down = built - .optimize_with(|ir| super::profiling(ir, runtime_context, counters, counter_queue)) - .with_default_optimize(); + .with_default_optimize() + .optimize_with(|ir| super::profiling(ir, runtime_context, counters, counter_queue)); insta::assert_debug_snapshot!(&pushed_down.ir()); diff --git a/hydroflow_plus/src/properties.rs b/hydroflow_plus/src/properties.rs index 9d0977e3b210..3d8b34349d94 100644 --- a/hydroflow_plus/src/properties.rs +++ b/hydroflow_plus/src/properties.rs @@ -102,7 +102,9 @@ mod tests { flow.source_iter(&process, q!(vec![])) .map(q!(|string: String| (string, ()))) + .tick_batch() .fold_keyed(q!(|| 0), counter_func) + .all_ticks() .for_each(q!(|(string, count)| println!("{}: {}", string, count))); let built = flow diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee-2.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee-2.snap index cc43d5edc9a9..2406b4f16ae4 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee-2.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee-2.snap @@ -5,12 +5,12 @@ expression: optimized.ir() [ ForEach { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }, - input: Tee { - inner: RefCell { - value: Persist( - Map { - f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, - input: Source { + input: Persist( + Map { + f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, + input: Tee { + inner: RefCell { + value: Source { source: Iter( { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, ), @@ -19,18 +19,18 @@ expression: optimized.ir() ), }, }, - ), + }, }, - }, + ), }, ForEach { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }, - input: Tee { - inner: RefCell { - value: Persist( - Map { - f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, - input: Source { + input: Persist( + Map { + f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, + input: Tee { + inner: RefCell { + value: Source { source: Iter( { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, ), @@ -39,8 +39,8 @@ expression: optimized.ir() ), }, }, - ), + }, }, - }, + ), }, ] diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee.snap index 7d933e1dfeed..a8736418c3e1 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee.snap @@ -5,42 +5,58 @@ expression: built.ir() [ ForEach { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }, - input: Tee { - inner: RefCell { - value: Map { + input: Unpersist( + Persist( + Map { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, - input: Persist( - Source { - source: Iter( - { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, - ), - location_kind: Process( - 0, + input: Tee { + inner: RefCell { + value: Persist( + Unpersist( + Persist( + Source { + source: Iter( + { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, + ), + location_kind: Process( + 0, + ), + }, + ), + ), ), }, - ), + }, }, - }, - }, + ), + ), }, ForEach { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }, - input: Tee { - inner: RefCell { - value: Map { + input: Unpersist( + Persist( + Map { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, - input: Persist( - Source { - source: Iter( - { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, - ), - location_kind: Process( - 0, + input: Tee { + inner: RefCell { + value: Persist( + Unpersist( + Persist( + Source { + source: Iter( + { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, + ), + location_kind: Process( + 0, + ), + }, + ), + ), ), }, - ), + }, }, - }, - }, + ), + ), }, ] diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee@surface_graph_0.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee@surface_graph_0.snap index 753b400ee716..0b04e1c416a1 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee@surface_graph_0.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_behind_tee@surface_graph_0.snap @@ -3,15 +3,18 @@ source: hydroflow_plus/src/persist_pullup.rs expression: graph.surface_syntax_string() --- 1v1 = source_iter ({ use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }); -2v1 = map ({ use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }); -3v1 = persist :: < 'static > (); -4v1 = tee (); +2v1 = tee (); +3v1 = map ({ use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }); +4v1 = persist :: < 'static > (); 5v1 = for_each ({ use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }); -6v1 = for_each ({ use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }); +6v1 = map ({ use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }); +7v1 = persist :: < 'static > (); +8v1 = for_each ({ use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }); 1v1 -> 2v1; 2v1 -> 3v1; 3v1 -> 4v1; 4v1 -> 5v1; -4v1 -> 6v1; - +2v1 -> 6v1; +6v1 -> 7v1; +7v1 -> 8v1; diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map-2.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map-2.snap index d8085779f6c6..0e9e69ef04c8 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map-2.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map-2.snap @@ -5,18 +5,16 @@ expression: optimized.ir() [ ForEach { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }, - input: Persist( - Map { - f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, - input: Source { - source: Iter( - { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, - ), - location_kind: Process( - 0, - ), - }, + input: Map { + f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, + input: Source { + source: Iter( + { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, + ), + location_kind: Process( + 0, + ), }, - ), + }, }, ] diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map.snap index 744ad98be98c..0f59809684ab 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map.snap @@ -5,18 +5,20 @@ expression: built.ir() [ ForEach { f: { use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }, - input: Map { - f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, - input: Persist( - Source { - source: Iter( - { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, - ), - location_kind: Process( - 0, - ), - }, - ), - }, + input: Unpersist( + Map { + f: { use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }, + input: Persist( + Source { + source: Iter( + { use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }, + ), + location_kind: Process( + 0, + ), + }, + ), + }, + ), }, ] diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map@surface_graph_0.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map@surface_graph_0.snap index 0923db298219..a7d3574562bb 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map@surface_graph_0.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__persist_pullup__tests__persist_pullup_through_map@surface_graph_0.snap @@ -4,10 +4,7 @@ expression: graph.surface_syntax_string() --- 1v1 = source_iter ({ use crate :: __staged :: persist_pullup :: tests :: * ; 0 .. 10 }); 2v1 = map ({ use crate :: __staged :: persist_pullup :: tests :: * ; | v | v + 1 }); -3v1 = persist :: < 'static > (); -4v1 = for_each ({ use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }); +3v1 = for_each ({ use crate :: __staged :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }); 1v1 -> 2v1; 2v1 -> 3v1; -3v1 -> 4v1; - diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators-2.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators-2.snap index ce8a3921555f..82ff7a928d8d 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators-2.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators-2.snap @@ -1,6 +1,6 @@ --- source: hydroflow_plus/src/profiler.rs -expression: "&pushed_down.ir" +expression: "&pushed_down.ir()" --- [ ForEach { @@ -11,19 +11,14 @@ expression: "&pushed_down.ir" f: { use crate :: __staged :: profiler :: tests :: * ; | v | v + 1 }, input: Inspect { f: { use crate :: __staged :: profiler :: * ; let counter_queue = Fake ; let counters = Fake ; let my_id = 1u32 ; { counter_queue . borrow () . unbounded_send ((my_id as usize , counters . borrow () [my_id as usize])) . unwrap () ; counters . borrow_mut () [my_id as usize] = 0 ; move | _ | { hydroflow_plus :: profiler :: increment_counter (& mut counters . borrow_mut () [my_id as usize] ,) ; } } }, - input: Persist( - Inspect { - f: { use crate :: __staged :: profiler :: * ; let counter_queue = Fake ; let counters = Fake ; let my_id = 2u32 ; { counter_queue . borrow () . unbounded_send ((my_id as usize , counters . borrow () [my_id as usize])) . unwrap () ; counters . borrow_mut () [my_id as usize] = 0 ; move | _ | { hydroflow_plus :: profiler :: increment_counter (& mut counters . borrow_mut () [my_id as usize] ,) ; } } }, - input: Source { - source: Iter( - { use crate :: __staged :: profiler :: tests :: * ; 0 .. 10 }, - ), - location_kind: Process( - 0, - ), - }, - }, - ), + input: Source { + source: Iter( + { use crate :: __staged :: profiler :: tests :: * ; 0 .. 10 }, + ), + location_kind: Process( + 0, + ), + }, }, }, }, diff --git a/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators.snap b/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators.snap index 34446c6173cc..6aefb7099aba 100644 --- a/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators.snap +++ b/hydroflow_plus/src/snapshots/hydroflow_plus__profiler__tests__profiler_wrapping_all_operators.snap @@ -1,22 +1,24 @@ --- source: hydroflow_plus/src/profiler.rs -expression: "&built.ir" +expression: "&built.ir()" --- [ ForEach { f: { use crate :: __staged :: profiler :: tests :: * ; | n | println ! ("{}" , n) }, - input: Map { - f: { use crate :: __staged :: profiler :: tests :: * ; | v | v + 1 }, - input: Persist( - Source { - source: Iter( - { use crate :: __staged :: profiler :: tests :: * ; 0 .. 10 }, - ), - location_kind: Process( - 0, - ), - }, - ), - }, + input: Unpersist( + Map { + f: { use crate :: __staged :: profiler :: tests :: * ; | v | v + 1 }, + input: Persist( + Source { + source: Iter( + { use crate :: __staged :: profiler :: tests :: * ; 0 .. 10 }, + ), + location_kind: Process( + 0, + ), + }, + ), + }, + ), }, ] diff --git a/hydroflow_plus/src/stream.rs b/hydroflow_plus/src/stream.rs index 0c0a61284ff6..52f654275e29 100644 --- a/hydroflow_plus/src/stream.rs +++ b/hydroflow_plus/src/stream.rs @@ -19,18 +19,18 @@ use crate::ir::{DebugInstantiate, HfPlusLeaf, HfPlusNode, HfPlusSource}; use crate::location::{CanSend, Location, LocationId}; use crate::Cluster; -/// Marks the stream as being asynchronous, which means the presence -/// of all elements is directly influenced by the runtime's batching -/// behavior. Aggregation operations are not permitted on streams -/// with this tag because the developer has not explicitly specified -/// if they want to aggregate over the entire stream or just the -/// current batch. -pub struct Async {} - -/// Marks the stream as being windowed, which means the developer has -/// opted-into either a batched or persistent windowing semantics. -/// Aggregation operations are permitted on streams with this tag. -pub struct Windowed {} +/// Marks the stream as being unbounded, which means that it is not +/// guaranteed to be complete in finite time. +pub enum Unbounded {} + +/// Marks the stream as being bounded, which means that it is guaranteed +/// to be complete in finite time. +pub enum Bounded {} + +/// Marks the stream as existing outside of a clock domain. +pub enum NoTick {} +/// Marks the stream as being inside the single global clock domain. +pub enum Tick {} /// An infinite stream of elements of type `T`. /// @@ -38,19 +38,21 @@ pub struct Windowed {} /// - `'a`: the lifetime of the final Hydroflow graph, which constraints /// which values can be captured in closures passed to operators /// - `T`: the type of elements in the stream -/// - `W`: the windowing semantics of the stream, which is either [`Async`] -/// or [`Windowed`] +/// - `W`: the boundedness of the stream, which is either [`Bounded`] +/// or [`Unbounded`] +/// - `C`: the tick domain of the stream, which is either [`Tick`] or +/// [`NoTick`] /// - `N`: the type of the node that the stream is materialized on -pub struct Stream<'a, T, W, N: Location> { +pub struct Stream<'a, T, W, C, N: Location> { location_kind: LocationId, ir_leaves: FlowLeaves<'a>, pub(crate) ir_node: RefCell>, - _phantom: PhantomData<(&'a mut &'a (), T, N, W)>, + _phantom: PhantomData<(&'a mut &'a (), T, N, W, C)>, } -impl<'a, T, W, N: Location> Stream<'a, T, W, N> { +impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { pub(crate) fn new( location_kind: LocationId, ir_leaves: FlowLeaves<'a>, @@ -65,7 +67,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { } } -impl<'a, T: Clone, W, N: Location> Clone for Stream<'a, T, W, N> { +impl<'a, T: Clone, W, C, N: Location> Clone for Stream<'a, T, W, C, N> { fn clone(&self) -> Self { if !matches!(self.ir_node.borrow().deref(), HfPlusNode::Tee { .. }) { let orig_ir_node = self.ir_node.replace(HfPlusNode::Placeholder); @@ -90,8 +92,11 @@ impl<'a, T: Clone, W, N: Location> Clone for Stream<'a, T, W, N> { } } -impl<'a, T, W, N: Location> Stream<'a, T, W, N> { - pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F>) -> Stream<'a, U, W, N> { +impl<'a, T, W, C, N: Location> Stream<'a, T, W, C, N> { + pub fn map U + 'a>( + self, + f: impl IntoQuotedMut<'a, F>, + ) -> Stream<'a, U, W, C, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -105,7 +110,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn flat_map, F: Fn(T) -> I + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, U, W, N> { + ) -> Stream<'a, U, W, C, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -116,29 +121,10 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { ) } - pub fn enumerate(self) -> Stream<'a, (usize, T), W, N> { - Stream::new( - self.location_kind, - self.ir_leaves, - HfPlusNode::Enumerate(Box::new(self.ir_node.into_inner())), - ) - } - - pub fn inspect(self, f: impl IntoQuotedMut<'a, F>) -> Stream<'a, T, W, N> { - Stream::new( - self.location_kind, - self.ir_leaves, - HfPlusNode::Inspect { - f: f.splice().into(), - input: Box::new(self.ir_node.into_inner()), - }, - ) - } - pub fn filter bool + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, T, W, N> { + ) -> Stream<'a, T, W, C, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -152,7 +138,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn filter_map Option + 'a>( self, f: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, U, W, N> { + ) -> Stream<'a, U, W, C, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -163,7 +149,10 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { ) } - pub fn cross_singleton(self, other: Stream<'a, O, Windowed, N>) -> Stream<'a, (T, O), W, N> + pub fn cross_singleton( + self, + other: Stream<'a, O, Bounded, C, N>, + ) -> Stream<'a, (T, O), W, C, N> where O: Clone, { @@ -182,18 +171,13 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { } /// Allow this stream through if the other stream has elements, otherwise the output is empty. - pub fn continue_if(self, signal: Stream<'a, U, Windowed, N>) -> Stream<'a, T, W, N> { + pub fn continue_if(self, signal: Stream<'a, U, Bounded, C, N>) -> Stream<'a, T, W, C, N> { self.cross_singleton(signal.map(q!(|_u| ()))) .map(q!(|(d, _signal)| d)) } - /// Allow this stream through if the other stream is empty, otherwise the output is empty. - pub fn continue_unless(self, other: Stream<'a, U, Windowed, N>) -> Stream<'a, T, W, N> { - self.continue_if(other.count().filter(q!(|c| *c == 0))) - } - // TODO(shadaj): should allow for differing windows, using strongest one - pub fn cross_product(self, other: Stream<'a, O, W, N>) -> Stream<'a, (T, O), W, N> + pub fn cross_product(self, other: Stream<'a, O, W, C, N>) -> Stream<'a, (T, O), W, C, N> where T: Clone, O: Clone, @@ -212,7 +196,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { ) } - pub fn union(self, other: Stream<'a, T, W, N>) -> Stream<'a, T, W, N> { + pub fn union(self, other: Stream<'a, T, W, C, N>) -> Stream<'a, T, W, C, N> { if self.location_kind != other.location_kind { panic!("union must be called on streams on the same node"); } @@ -227,21 +211,16 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { ) } - pub fn for_each(self, f: impl IntoQuotedMut<'a, F>) { - self.ir_leaves.borrow_mut().as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::ForEach { - input: Box::new(self.ir_node.into_inner()), - f: f.splice().into(), - }); - } - pub fn dest_sink + 'a>(self, sink: impl Quoted<'a, S>) { self.ir_leaves.borrow_mut().as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::DestSink { sink: sink.splice().into(), input: Box::new(self.ir_node.into_inner()), }); } +} - pub fn all_ticks(self) -> Stream<'a, T, Windowed, N> { +impl<'a, T, N: Location> Stream<'a, T, Bounded, Tick, N> { + pub fn all_ticks(self) -> Stream<'a, T, Unbounded, NoTick, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -249,31 +228,57 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { ) } - pub fn assume_windowed(self) -> Stream<'a, T, Windowed, N> { + pub fn persist(self) -> Stream<'a, T, Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, - self.ir_node.into_inner(), + HfPlusNode::Persist(Box::new(self.ir_node.into_inner())), ) } -} -impl<'a, T, N: Location> Stream<'a, T, Async, N> { - pub fn tick_batch(self) -> Stream<'a, T, Windowed, N> { + pub fn defer_tick(self) -> Stream<'a, T, Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, - self.ir_node.into_inner(), + HfPlusNode::DeferTick(Box::new(self.ir_node.into_inner())), + ) + } + + pub fn inspect( + self, + f: impl IntoQuotedMut<'a, F>, + ) -> Stream<'a, T, Bounded, Tick, N> { + Stream::new( + self.location_kind, + self.ir_leaves, + HfPlusNode::Inspect { + f: f.splice().into(), + input: Box::new(self.ir_node.into_inner()), + }, + ) + } + + /// Allow this stream through if the other stream is empty, otherwise the output is empty. + pub fn continue_unless( + self, + other: Stream<'a, U, Bounded, Tick, N>, + ) -> Stream<'a, T, Bounded, Tick, N> { + self.continue_if(other.count().filter(q!(|c| *c == 0))) + } + + pub fn enumerate(self) -> Stream<'a, (usize, T), Bounded, Tick, N> { + Stream::new( + self.location_kind, + self.ir_leaves, + HfPlusNode::Enumerate(Box::new(self.ir_node.into_inner())), ) } -} -impl<'a, T, N: Location> Stream<'a, T, Windowed, N> { - pub fn fold A + 'a, C: Fn(&mut A, T)>( + pub fn fold A + 'a, F: Fn(&mut A, T)>( self, init: impl IntoQuotedMut<'a, I>, - comb: impl IntoQuotedMut<'a, C>, - ) -> Stream<'a, A, Windowed, N> { + comb: impl IntoQuotedMut<'a, F>, + ) -> Stream<'a, A, Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -285,10 +290,10 @@ impl<'a, T, N: Location> Stream<'a, T, Windowed, N> { ) } - pub fn reduce( + pub fn reduce( self, - comb: impl IntoQuotedMut<'a, C>, - ) -> Stream<'a, T, Windowed, N> { + comb: impl IntoQuotedMut<'a, F>, + ) -> Stream<'a, T, Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -299,7 +304,7 @@ impl<'a, T, N: Location> Stream<'a, T, Windowed, N> { ) } - pub fn sort(self) -> Stream<'a, T, Windowed, N> + pub fn sort(self) -> Stream<'a, T, Bounded, Tick, N> where T: Ord, { @@ -310,11 +315,11 @@ impl<'a, T, N: Location> Stream<'a, T, Windowed, N> { ) } - pub fn count(self) -> Stream<'a, usize, Windowed, N> { + pub fn count(self) -> Stream<'a, usize, Bounded, Tick, N> { self.fold(q!(|| 0usize), q!(|count, _| *count += 1)) } - pub fn delta(self) -> Stream<'a, T, Windowed, N> { + pub fn delta(self) -> Stream<'a, T, Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -322,50 +327,59 @@ impl<'a, T, N: Location> Stream<'a, T, Windowed, N> { ) } - pub fn defer_tick(self) -> Stream<'a, T, Windowed, N> { + pub fn unique(self) -> Stream<'a, T, Bounded, Tick, N> + where + T: Eq + Hash, + { Stream::new( self.location_kind, self.ir_leaves, - HfPlusNode::DeferTick(Box::new(self.ir_node.into_inner())), + HfPlusNode::Unique(Box::new(self.ir_node.into_inner())), ) } +} - pub fn unique(self) -> Stream<'a, T, Windowed, N> - where - T: Eq + Hash, - { +impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { + pub fn tick_batch(self) -> Stream<'a, T, Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, - HfPlusNode::Unique(Box::new(self.ir_node.into_inner())), + HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner())), ) } - pub fn filter_not_in(self, other: Stream<'a, T, Windowed, N>) -> Stream<'a, T, Windowed, N> - where - T: Eq + Hash, - { - if self.location_kind != other.location_kind { - panic!("union must be called on streams on the same node"); - } - + pub fn inspect( + self, + f: impl IntoQuotedMut<'a, F>, + ) -> Stream<'a, T, W, NoTick, N> { Stream::new( self.location_kind, self.ir_leaves, - HfPlusNode::Difference( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + HfPlusNode::Persist(Box::new(HfPlusNode::Inspect { + f: f.splice().into(), + input: Box::new(HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner()))), + })), ) } +} +impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { + pub fn for_each(self, f: impl IntoQuotedMut<'a, F>) { + self.ir_leaves.borrow_mut().as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::ForEach { + input: Box::new(HfPlusNode::Unpersist(Box::new(self.ir_node.into_inner()))), + f: f.splice().into(), + }); + } +} + +impl<'a, T, N: Location> Stream<'a, T, Unbounded, NoTick, N> { pub fn sample_every( self, duration: impl Quoted<'a, std::time::Duration> + Copy + 'a, - ) -> Stream<'a, T, Windowed, N> { + ) -> Stream<'a, T, Unbounded, NoTick, N> { let interval = duration.splice(); - let samples = Stream::<'a, hydroflow::tokio::time::Instant, Windowed, N>::new( + let samples = Stream::<'a, hydroflow::tokio::time::Instant, Bounded, Tick, N>::new( self.location_kind, self.ir_leaves.clone(), HfPlusNode::Source { @@ -374,19 +388,42 @@ impl<'a, T, N: Location> Stream<'a, T, Windowed, N> { }, ); - self.continue_if(samples) + self.tick_batch().continue_if(samples).all_ticks() } } -impl<'a, T: Clone, W, N: Location> Stream<'a, &T, W, N> { - pub fn cloned(self) -> Stream<'a, T, W, N> { +impl<'a, T, C, N: Location> Stream<'a, T, Bounded, C, N> { + pub fn filter_not_in(self, other: Stream<'a, T, Bounded, C, N>) -> Stream<'a, T, Bounded, C, N> + where + T: Eq + Hash, + { + if self.location_kind != other.location_kind { + panic!("union must be called on streams on the same node"); + } + + Stream::new( + self.location_kind, + self.ir_leaves, + HfPlusNode::Difference( + Box::new(self.ir_node.into_inner()), + Box::new(other.ir_node.into_inner()), + ), + ) + } +} + +impl<'a, T: Clone, W, C, N: Location> Stream<'a, &T, W, C, N> { + pub fn cloned(self) -> Stream<'a, T, W, C, N> { self.map(q!(|d| d.clone())) } } -impl<'a, K, V1, W, N: Location> Stream<'a, (K, V1), W, N> { +impl<'a, K, V1, W, C, N: Location> Stream<'a, (K, V1), W, C, N> { // TODO(shadaj): figure out window semantics - pub fn join(self, n: Stream<'a, (K, V2), W2, N>) -> Stream<'a, (K, (V1, V2)), W, N> + pub fn join( + self, + n: Stream<'a, (K, V2), W2, C, N>, + ) -> Stream<'a, (K, (V1, V2)), W, C, N> where K: Eq + Hash, { @@ -404,7 +441,7 @@ impl<'a, K, V1, W, N: Location> Stream<'a, (K, V1), W, N> { ) } - pub fn anti_join(self, n: Stream<'a, K, W2, N>) -> Stream<'a, (K, V1), W, N> + pub fn anti_join(self, n: Stream<'a, K, W2, C, N>) -> Stream<'a, (K, V1), W, C, N> where K: Eq + Hash, { @@ -423,12 +460,12 @@ impl<'a, K, V1, W, N: Location> Stream<'a, (K, V1), W, N> { } } -impl<'a, K: Eq + Hash, V, N: Location> Stream<'a, (K, V), Windowed, N> { - pub fn fold_keyed A + 'a, C: Fn(&mut A, V) + 'a>( +impl<'a, K: Eq + Hash, V, N: Location> Stream<'a, (K, V), Bounded, Tick, N> { + pub fn fold_keyed A + 'a, F: Fn(&mut A, V) + 'a>( self, init: impl IntoQuotedMut<'a, I>, - comb: impl IntoQuotedMut<'a, C>, - ) -> Stream<'a, (K, A), Windowed, N> { + comb: impl IntoQuotedMut<'a, F>, + ) -> Stream<'a, (K, A), Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -443,7 +480,7 @@ impl<'a, K: Eq + Hash, V, N: Location> Stream<'a, (K, V), Windowed, N> { pub fn reduce_keyed( self, comb: impl IntoQuotedMut<'a, F>, - ) -> Stream<'a, (K, V), Windowed, N> { + ) -> Stream<'a, (K, V), Bounded, Tick, N> { Stream::new( self.location_kind, self.ir_leaves, @@ -508,11 +545,11 @@ fn deserialize_bincode(tagged: bool) -> Pipeline { } } -impl<'a, T, W, N: Location> Stream<'a, T, W, N> { +impl<'a, T, W, N: Location> Stream<'a, T, W, NoTick, N> { pub fn send_bincode( self, other: &N2, - ) -> Stream<'a, N::Out, Async, N2> + ) -> Stream<'a, N::Out, Unbounded, NoTick, N2> where N: CanSend = T>, CoreType: Serialize + DeserializeOwned, @@ -535,7 +572,10 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { ) } - pub fn send_bytes(self, other: &N2) -> Stream<'a, N::Out, Async, N2> + pub fn send_bytes( + self, + other: &N2, + ) -> Stream<'a, N::Out, Unbounded, NoTick, N2> where N: CanSend = T>, { @@ -560,7 +600,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn send_bincode_interleaved( self, other: &N2, - ) -> Stream<'a, CoreType, Async, N2> + ) -> Stream<'a, CoreType, Unbounded, NoTick, N2> where N: CanSend = T, Out = (Tag, CoreType)>, CoreType: Serialize + DeserializeOwned, @@ -571,7 +611,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn send_bytes_interleaved( self, other: &N2, - ) -> Stream<'a, Bytes, Async, N2> + ) -> Stream<'a, Bytes, Unbounded, NoTick, N2> where N: CanSend = T, Out = (Tag, Bytes)>, { @@ -581,7 +621,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn broadcast_bincode( self, other: &Cluster, - ) -> Stream<'a, N::Out, Async, Cluster> + ) -> Stream<'a, N::Out, Unbounded, NoTick, Cluster> where N: CanSend, In = (u32, T)>, T: Clone + Serialize + DeserializeOwned, @@ -601,7 +641,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn broadcast_bincode_interleaved( self, other: &Cluster, - ) -> Stream<'a, T, Async, Cluster> + ) -> Stream<'a, T, Unbounded, NoTick, Cluster> where N: CanSend, In = (u32, T), Out = (Tag, T)> + 'a, T: Clone + Serialize + DeserializeOwned, @@ -612,7 +652,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn broadcast_bytes( self, other: &Cluster, - ) -> Stream<'a, N::Out, Async, Cluster> + ) -> Stream<'a, N::Out, Unbounded, NoTick, Cluster> where N: CanSend, In = (u32, T)> + 'a, T: Clone, @@ -632,7 +672,7 @@ impl<'a, T, W, N: Location> Stream<'a, T, W, N> { pub fn broadcast_bytes_interleaved( self, other: &Cluster, - ) -> Stream<'a, Bytes, Async, Cluster> + ) -> Stream<'a, Bytes, Unbounded, NoTick, Cluster> where N: CanSend, In = (u32, T), Out = (Tag, Bytes)> + 'a, T: Clone, diff --git a/hydroflow_plus_test/src/cluster/compute_pi.rs b/hydroflow_plus_test/src/cluster/compute_pi.rs index 3affb3b93bcf..ac52c9852ee4 100644 --- a/hydroflow_plus_test/src/cluster/compute_pi.rs +++ b/hydroflow_plus_test/src/cluster/compute_pi.rs @@ -23,15 +23,18 @@ pub fn compute_pi(flow: &FlowBuilder, batch_size: usize) -> (Cluster, Pr *total += 1; }), - ); + ) + .all_ticks(); trials .send_bincode_interleaved(&process) - .all_ticks() + .tick_batch() + .persist() .reduce(q!(|(inside, total), (inside_batch, total_batch)| { *inside += inside_batch; *total += total_batch; })) + .all_ticks() .sample_every(q!(Duration::from_secs(1))) .for_each(q!(|(inside, total)| { println!( diff --git a/hydroflow_plus_test/src/cluster/map_reduce.rs b/hydroflow_plus_test/src/cluster/map_reduce.rs index 46ef7bfaec1c..5c4c2829f10f 100644 --- a/hydroflow_plus_test/src/cluster/map_reduce.rs +++ b/hydroflow_plus_test/src/cluster/map_reduce.rs @@ -14,21 +14,26 @@ pub fn map_reduce(flow: &FlowBuilder) -> (Process, Cluster) { let all_ids_vec = flow.cluster_members(&cluster); let words_partitioned = words + .tick_batch() .enumerate() - .map(q!(|(i, w)| ((i % all_ids_vec.len()) as u32, w))); + .map(q!(|(i, w)| ((i % all_ids_vec.len()) as u32, w))) + .all_ticks(); words_partitioned .send_bincode(&cluster) - .tick_batch() .map(q!(|string| (string, ()))) + .tick_batch() .fold_keyed(q!(|| 0), q!(|count, _| *count += 1)) .inspect(q!(|(string, count)| println!( "partition count: {} - {}", string, count ))) - .send_bincode_interleaved(&process) .all_ticks() + .send_bincode_interleaved(&process) + .tick_batch() + .persist() .reduce_keyed(q!(|total, count| *total += count)) + .all_ticks() .for_each(q!(|(string, count)| println!("{}: {}", string, count))); (process, cluster) diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index b29679cefd2b..eea9d371fa85 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -6,6 +6,7 @@ use std::time::{Duration, SystemTime}; use hydroflow_plus::*; use serde::{Deserialize, Serialize}; use stageleft::*; +use stream::{NoTick, Tick}; use tokio::time::Instant; pub struct Proposer {} @@ -118,7 +119,7 @@ pub fn paxos( flow.source_iter(&proposers, q!(["Proposers say hello"])) .for_each(q!(|s| println!("{}", s))); let p_id = flow.cluster_self_id(&proposers); - let (p_is_leader_complete_cycle, p_is_leader) = flow.cycle(&proposers); + let (p_is_leader_complete_cycle, p_is_leader) = flow.tick_cycle(&proposers); let (p_to_proposers_i_am_leader_complete_cycle, p_to_proposers_i_am_leader) = flow.cycle(&proposers); @@ -181,6 +182,7 @@ pub fn paxos( .continue_unless(p_next_slot) .cross_singleton(p_ballot_num) .map(q!(move |(_is_leader, ballot_num): (bool, u32)| Ballot { num: ballot_num, id: p_id})) // Only tell the clients once when leader election concludes + .all_ticks() .broadcast_bincode_interleaved(&clients); p_to_clients_leader_elected_cycle.complete(p_to_clients_new_leader_elected); // End tell clients that leader election has completed @@ -217,24 +219,24 @@ pub fn paxos( #[allow(clippy::type_complexity)] fn acceptor<'a>( - p_to_acceptors_p1a: Stream<'a, P1a, stream::Async, Cluster>, - p_to_acceptors_p2a: Stream<'a, P2a, stream::Async, Cluster>, - r_to_acceptors_checkpoint: Stream<'a, (u32, i32), stream::Async, Cluster>, + p_to_acceptors_p1a: Stream<'a, P1a, stream::Unbounded, NoTick, Cluster>, + p_to_acceptors_p2a: Stream<'a, P2a, stream::Unbounded, NoTick, Cluster>, + r_to_acceptors_checkpoint: Stream<'a, (u32, i32), stream::Unbounded, NoTick, Cluster>, proposers: &Cluster, f: usize, ) -> ( - Stream<'a, (u32, P1b), stream::Async, Cluster>, - Stream<'a, (u32, P2b), stream::Async, Cluster>, + Stream<'a, (u32, P1b), stream::Unbounded, NoTick, Cluster>, + Stream<'a, (u32, P2b), stream::Unbounded, NoTick, Cluster>, ) { // Get the latest checkpoint sequence per replica - let a_checkpoint_largest_seqs = - r_to_acceptors_checkpoint - .all_ticks() - .reduce_keyed(q!(|curr_seq: &mut i32, seq: i32| { - if seq > *curr_seq { - *curr_seq = seq; - } - })); + let a_checkpoint_largest_seqs = r_to_acceptors_checkpoint + .tick_batch() + .persist() + .reduce_keyed(q!(|curr_seq: &mut i32, seq: i32| { + if seq > *curr_seq { + *curr_seq = seq; + } + })); let a_checkpoints_quorum_reached = a_checkpoint_largest_seqs .clone() @@ -270,7 +272,7 @@ fn acceptor<'a>( ))); // .inspect(q!(|(min_seq, p2a): &(i32, P2a)| println!("Acceptor new checkpoint: {:?}", min_seq))); - let a_max_ballot = p_to_acceptors_p1a.clone().all_ticks().fold( + let a_max_ballot = p_to_acceptors_p1a.clone().tick_batch().persist().fold( q!(|| Ballot { num: 0, id: 0 }), q!(|max_ballot: &mut Ballot, p1a: P1a| { if p1a.ballot > *max_ballot { @@ -291,7 +293,7 @@ fn acceptor<'a>( )); let a_log = a_p2as_to_place_in_log .union(a_new_checkpoint) - .all_ticks() + .persist() .fold( q!(|| (-1, HashMap::::new())), q!( @@ -349,6 +351,7 @@ fn acceptor<'a>( accepted: log } ))) + .all_ticks() .send_bincode(proposers); let a_to_proposers_p2b_new = p_to_acceptors_p2a .tick_batch() @@ -362,6 +365,7 @@ fn acceptor<'a>( value: p2a.value } ))) + .all_ticks() .send_bincode(proposers); (a_to_proposers_p1b_new, a_to_proposers_p2b_new) } @@ -369,12 +373,13 @@ fn acceptor<'a>( fn p_p2b<'a>( flow: &FlowBuilder<'a>, proposers: &Cluster, - a_to_proposers_p2b: Stream<'a, (u32, P2b), stream::Async, Cluster>, + a_to_proposers_p2b: Stream<'a, (u32, P2b), stream::Unbounded, NoTick, Cluster>, replicas: &Cluster, f: usize, -) -> Stream<'a, ReplicaPayload, stream::Async, Cluster> { - let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = flow.cycle(proposers); - let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = flow.cycle(proposers); +) -> Stream<'a, ReplicaPayload, stream::Unbounded, NoTick, Cluster> { + let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = + flow.tick_cycle(proposers); + let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = flow.tick_cycle(proposers); let p_p2b = a_to_proposers_p2b .clone() .tick_batch() @@ -417,6 +422,7 @@ fn p_p2b<'a>( .clone() .anti_join(p_broadcasted_p2b_slots) // Only tell the replicas about committed values once .map(q!(|(_slot, (_count, p2b)): (i32, (usize, P2b))| ReplicaPayload { seq: p2b.slot, key: p2b.value.key, value: p2b.value.value })) + .all_ticks() .broadcast_bincode_interleaved(replicas); let p_p2b_all_commit_slots = @@ -455,19 +461,19 @@ fn p_p2b<'a>( fn p_p2a<'a>( flow: &FlowBuilder<'a>, proposers: &Cluster, - p_max_slot: Stream<'a, i32, stream::Windowed, Cluster>, - c_to_proposers: Stream<'a, ClientPayload, stream::Async, Cluster>, - p_ballot_num: Stream<'a, u32, stream::Windowed, Cluster>, - p_log_to_try_commit: Stream<'a, P2a, stream::Windowed, Cluster>, - p_log_holes: Stream<'a, P2a, stream::Windowed, Cluster>, - p_is_leader: Stream<'a, bool, stream::Windowed, Cluster>, + p_max_slot: Stream<'a, i32, stream::Bounded, Tick, Cluster>, + c_to_proposers: Stream<'a, ClientPayload, stream::Unbounded, NoTick, Cluster>, + p_ballot_num: Stream<'a, u32, stream::Bounded, Tick, Cluster>, + p_log_to_try_commit: Stream<'a, P2a, stream::Bounded, Tick, Cluster>, + p_log_holes: Stream<'a, P2a, stream::Bounded, Tick, Cluster>, + p_is_leader: Stream<'a, bool, stream::Bounded, Tick, Cluster>, acceptors: &Cluster, ) -> ( - Stream<'a, i32, stream::Windowed, Cluster>, - Stream<'a, P2a, stream::Async, Cluster>, + Stream<'a, i32, stream::Bounded, Tick, Cluster>, + Stream<'a, P2a, stream::Unbounded, NoTick, Cluster>, ) { let p_id = flow.cluster_self_id(proposers); - let (p_next_slot_complete_cycle, p_next_slot) = flow.cycle(proposers); + let (p_next_slot_complete_cycle, p_next_slot) = flow.tick_cycle(proposers); let p_next_slot_after_reconciling_p1bs = p_max_slot // .inspect(q!(|max_slot| println!("{} p_max_slot: {:?}", context.current_tick(), max_slot))) .continue_unless(p_next_slot.clone()) @@ -489,6 +495,7 @@ fn p_p2a<'a>( .continue_unless(p_next_slot.clone()) // Only resend p1b stuff once. Once it's resent, next_slot will exist. .union(p_indexed_payloads) .continue_if(p_is_leader.clone()) + .all_ticks() .broadcast_bincode_interleaved(acceptors); let p_num_payloads = c_to_proposers.clone().tick_batch().count(); @@ -527,20 +534,21 @@ fn p_p2a<'a>( fn p_p1b<'a>( flow: &FlowBuilder<'a>, proposers: &Cluster, - a_to_proposers_p1b: Stream<'a, (u32, P1b), stream::Async, Cluster>, - p_ballot_num: Stream<'a, u32, stream::Windowed, Cluster>, - p_has_largest_ballot: Stream<'a, (Ballot, u32), stream::Windowed, Cluster>, + a_to_proposers_p1b: Stream<'a, (u32, P1b), stream::Unbounded, NoTick, Cluster>, + p_ballot_num: Stream<'a, u32, stream::Bounded, Tick, Cluster>, + p_has_largest_ballot: Stream<'a, (Ballot, u32), stream::Bounded, Tick, Cluster>, f: usize, ) -> ( - Stream<'a, bool, stream::Windowed, Cluster>, - Stream<'a, P2a, stream::Windowed, Cluster>, - Stream<'a, i32, stream::Windowed, Cluster>, - Stream<'a, P2a, stream::Windowed, Cluster>, + Stream<'a, bool, stream::Bounded, Tick, Cluster>, + Stream<'a, P2a, stream::Bounded, Tick, Cluster>, + Stream<'a, i32, stream::Bounded, Tick, Cluster>, + Stream<'a, P2a, stream::Bounded, Tick, Cluster>, ) { let p_id = flow.cluster_self_id(proposers); let p_relevant_p1bs = a_to_proposers_p1b .clone() - .all_ticks() + .tick_batch() + .persist() .cross_singleton(p_ballot_num.clone()) .filter(q!(move |((_sender, p1b), ballot_num): &( (u32, P1b), @@ -644,13 +652,13 @@ fn p_p1b<'a>( fn replica<'a>( flow: &FlowBuilder<'a>, replicas: &Cluster, - p_to_replicas: Stream<'a, ReplicaPayload, stream::Async, Cluster>, + p_to_replicas: Stream<'a, ReplicaPayload, stream::Unbounded, NoTick, Cluster>, checkpoint_frequency: usize, ) -> ( - Stream<'a, i32, stream::Windowed, Cluster>, - Stream<'a, (u32, ReplicaPayload), stream::Windowed, Cluster>, + Stream<'a, i32, stream::Unbounded, NoTick, Cluster>, + Stream<'a, (u32, ReplicaPayload), stream::Unbounded, NoTick, Cluster>, ) { - let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = flow.cycle(replicas); + let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = flow.tick_cycle(replicas); // p_to_replicas.inspect(q!(|payload: ReplicaPayload| println!("Replica received payload: {:?}", payload))); let r_sorted_payloads = p_to_replicas .clone() @@ -658,8 +666,8 @@ fn replica<'a>( .union(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) = flow.cycle(replicas); - let empty_slot = flow.source_iter(replicas, q!([-1])); + let (r_highest_seq_complete_cycle, r_highest_seq) = flow.tick_cycle(replicas); + let empty_slot = flow.source_iter(replicas, q!([-1])).tick_batch(); // Either the max sequence number executed so far or -1. Need to union otherwise r_highest_seq is empty and joins with it will fail let r_highest_seq_with_default = r_highest_seq.union(empty_slot); // Find highest the sequence number of any payload that can be processed in this tick. This is the payload right before a hole. @@ -708,7 +716,7 @@ fn replica<'a>( let r_kv_store = r_processable_payloads .clone() - .all_ticks() // Optimization: all_ticks() + fold() = fold, where the state of the previous fold is saved and persisted values are deleted. + .persist() // Optimization: all_ticks() + fold() = fold, where the state of the previous fold is saved and persisted values are deleted. .fold(q!(|| (HashMap::::new(), -1)), q!(|state: &mut (HashMap::, i32), payload: ReplicaPayload| { let kv_store = &mut state.0; let last_seq = &mut state.1; @@ -727,8 +735,8 @@ fn replica<'a>( r_highest_seq_complete_cycle.complete(r_new_highest_seq.clone()); // Send checkpoints to the acceptors when we've processed enough payloads - let (r_checkpointed_seqs_complete_cycle, r_checkpointed_seqs) = flow.cycle(replicas); - let r_max_checkpointed_seq = r_checkpointed_seqs.all_ticks().fold( + let (r_checkpointed_seqs_complete_cycle, r_checkpointed_seqs) = flow.tick_cycle(replicas); + let r_max_checkpointed_seq = r_checkpointed_seqs.persist().fold( q!(|| -1), q!(|max_seq: &mut i32, seq: i32| { if seq > *max_seq { @@ -752,25 +760,29 @@ fn replica<'a>( r_checkpointed_seqs_complete_cycle.complete(r_checkpoint_seq_new.clone()); // Tell clients that the payload has been committed. All ReplicaPayloads contain the client's machine ID (to string) as value. - let r_to_clients = p_to_replicas - .tick_batch() - .map(q!(|payload: ReplicaPayload| ( - payload.value.parse::().unwrap(), - payload - ))); - (r_checkpoint_seq_new, r_to_clients) + let r_to_clients = p_to_replicas.map(q!(|payload: ReplicaPayload| ( + payload.value.parse::().unwrap(), + payload + ))); + (r_checkpoint_seq_new.all_ticks(), r_to_clients) } // Clients. All relations for clients will be prefixed with c. All ClientPayloads will contain the virtual client number as key and the client's machine ID (to string) as value. Expects p_to_clients_leader_elected containing Ballots whenever the leader is elected, and r_to_clients_payload_applied containing ReplicaPayloads whenever a payload is committed. Outputs (leader address, ClientPayload) when a new leader is elected or when the previous payload is committed. fn client<'a>( clients: &Cluster, - p_to_clients_leader_elected: Stream<'a, Ballot, stream::Async, Cluster>, - r_to_clients_payload_applied: Stream<'a, (u32, ReplicaPayload), stream::Async, Cluster>, + p_to_clients_leader_elected: Stream<'a, Ballot, stream::Unbounded, NoTick, Cluster>, + r_to_clients_payload_applied: Stream< + 'a, + (u32, ReplicaPayload), + stream::Unbounded, + NoTick, + Cluster, + >, flow: &FlowBuilder<'a>, num_clients_per_node: usize, median_latency_window_size: usize, f: usize, -) -> Stream<'a, (u32, ClientPayload), stream::Windowed, Cluster> { +) -> Stream<'a, (u32, ClientPayload), stream::Unbounded, NoTick, Cluster> { let c_id = flow.cluster_self_id(clients); p_to_clients_leader_elected .clone() @@ -780,13 +792,14 @@ fn client<'a>( ))); // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); // Only keep the latest leader - let c_max_leader_ballot = p_to_clients_leader_elected.all_ticks().reduce(q!( - |curr_max_ballot: &mut Ballot, new_ballot: Ballot| { + let c_max_leader_ballot = p_to_clients_leader_elected + .tick_batch() + .persist() + .reduce(q!(|curr_max_ballot: &mut Ballot, new_ballot: Ballot| { if new_ballot > *curr_max_ballot { *curr_max_ballot = new_ballot; } - } - )); + })); let c_new_leader_ballot = c_max_leader_ballot.clone().delta(); // Whenever the leader changes, make all clients send a message let c_new_payloads_when_leader_elected = @@ -801,7 +814,8 @@ fn client<'a>( } )))); // Whenever replicas confirm that a payload was committed, collected it and wait for a quorum - let (c_pending_quorum_payloads_complete_cycle, c_pending_quorum_payloads) = flow.cycle(clients); + let (c_pending_quorum_payloads_complete_cycle, c_pending_quorum_payloads) = + flow.tick_cycle(clients); let c_received_payloads = r_to_clients_payload_applied .tick_batch() .map(q!(|(sender, replica_payload): (u32, ReplicaPayload)| ( @@ -839,10 +853,12 @@ fn client<'a>( value: c_id.to_string() } ))); - let c_to_proposers = c_new_payloads_when_leader_elected.union(c_new_payloads_when_committed); + let c_to_proposers = c_new_payloads_when_leader_elected + .union(c_new_payloads_when_committed) + .all_ticks(); // Track statistics - let (c_timers_complete_cycle, c_timers) = flow.cycle(clients); + let (c_timers_complete_cycle, c_timers) = flow.tick_cycle(clients); let c_new_timers_when_leader_elected = c_new_leader_ballot .map(q!(|_: Ballot| SystemTime::now())) .flat_map(q!(move |now: SystemTime| (0..num_clients_per_node) @@ -880,7 +896,7 @@ fn client<'a>( curr_time.duration_since(prev_time).unwrap().as_micros() ))) .union(c_latency_reset) - .all_ticks() + .persist() .fold( // 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 HF+ with RuntimeData *median_latency_window_size @@ -932,7 +948,7 @@ fn client<'a>( let c_throughput = c_throughput_new_batch .union(c_throughput_reset) - .all_ticks() + .persist() .fold( q!(|| (0, 0)), q!( @@ -951,6 +967,7 @@ fn client<'a>( c_stats_output_timer .cross_singleton(c_latencies) .cross_singleton(c_throughput) + .all_ticks() .for_each(q!(move |( (_, (latencies, _write_index, has_any_value)), (throughput, num_ticks), @@ -977,10 +994,10 @@ fn client<'a>( // Proposer logic to calculate the largest ballot received so far. fn p_max_ballot<'a>( - a_to_proposers_p1b: Stream<'a, (u32, P1b), stream::Async, Cluster>, - a_to_proposers_p2b: Stream<'a, (u32, P2b), stream::Async, Cluster>, - p_to_proposers_i_am_leader: Stream<'a, Ballot, stream::Async, Cluster>, -) -> Stream<'a, Ballot, stream::Windowed, Cluster> { + a_to_proposers_p1b: Stream<'a, (u32, P1b), stream::Unbounded, NoTick, Cluster>, + a_to_proposers_p2b: Stream<'a, (u32, P2b), stream::Unbounded, NoTick, Cluster>, + p_to_proposers_i_am_leader: Stream<'a, Ballot, stream::Unbounded, NoTick, Cluster>, +) -> Stream<'a, Ballot, stream::Bounded, Tick, Cluster> { let p_received_p1b_ballots = a_to_proposers_p1b .clone() .map(q!(|(_, p1b): (_, P1b)| p1b.max_ballot)); @@ -990,7 +1007,8 @@ fn p_max_ballot<'a>( let p_received_max_ballot = p_received_p1b_ballots .union(p_received_p2b_ballots) .union(p_to_proposers_i_am_leader) - .all_ticks() + .tick_batch() + .persist() .fold( q!(|| Ballot { num: 0, id: 0 }), q!(|curr_max_ballot: &mut Ballot, new_ballot: Ballot| { @@ -1007,13 +1025,13 @@ fn p_max_ballot<'a>( fn p_ballot_calc<'a>( flow: &FlowBuilder<'a>, proposers: &Cluster, - p_received_max_ballot: Stream<'a, Ballot, stream::Windowed, Cluster>, + p_received_max_ballot: Stream<'a, Ballot, stream::Bounded, Tick, Cluster>, ) -> ( - Stream<'a, u32, stream::Windowed, Cluster>, - Stream<'a, (Ballot, u32), stream::Windowed, Cluster>, + Stream<'a, u32, stream::Bounded, Tick, Cluster>, + Stream<'a, (Ballot, u32), stream::Bounded, Tick, Cluster>, ) { let p_id = flow.cluster_self_id(proposers); - let (p_ballot_num_complete_cycle, p_ballot_num) = flow.cycle(proposers); + let (p_ballot_num_complete_cycle, p_ballot_num) = flow.tick_cycle(proposers); let p_has_largest_ballot = p_received_max_ballot .clone() .cross_singleton(p_ballot_num.clone()) @@ -1044,7 +1062,7 @@ fn p_ballot_calc<'a>( } })) .defer_tick(); - let p_start_ballot_num = flow.source_iter(proposers, q!([0])); + let p_start_ballot_num = flow.source_iter(proposers, q!([0])).tick_batch(); p_ballot_num_complete_cycle.complete(p_start_ballot_num.union(p_new_ballot_num)); // End stable leader election (p_ballot_num, p_has_largest_ballot) @@ -1053,37 +1071,44 @@ fn p_ballot_calc<'a>( // Proposer logic to send "I am leader" messages periodically to other proposers, or send p1a to acceptors if other leaders expired. #[allow(clippy::too_many_arguments, clippy::type_complexity)] fn p_p1a<'a>( - p_ballot_num: Stream<'a, u32, stream::Windowed, Cluster>, - p_is_leader: Stream<'a, bool, stream::Windowed, Cluster>, + p_ballot_num: Stream<'a, u32, stream::Bounded, Tick, Cluster>, + p_is_leader: Stream<'a, bool, stream::Bounded, Tick, Cluster>, proposers: &Cluster, - p_to_proposers_i_am_leader: Stream<'a, Ballot, stream::Async, Cluster>, + p_to_proposers_i_am_leader: Stream<'a, Ballot, stream::Unbounded, NoTick, Cluster>, flow: &FlowBuilder<'a>, acceptors: &Cluster, 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 */ ) -> ( - Stream<'a, Ballot, stream::Async, Cluster>, - Stream<'a, P1a, stream::Async, Cluster>, + Stream<'a, Ballot, stream::Unbounded, NoTick, Cluster>, + Stream<'a, P1a, stream::Unbounded, NoTick, Cluster>, ) { let p_id = flow.cluster_self_id(proposers); let p_to_proposers_i_am_leader_new = p_ballot_num .clone() + .all_ticks() .sample_every(q!(Duration::from_secs(i_am_leader_send_timeout))) + .tick_batch() .continue_if(p_is_leader.clone()) .map(q!(move |ballot_num: u32| Ballot { num: ballot_num, id: p_id })) + .all_ticks() .broadcast_bincode_interleaved(proposers); - let p_latest_received_i_am_leader = p_to_proposers_i_am_leader.clone().all_ticks().fold( - q!(|| None), - q!(|latest: &mut Option, _: Ballot| { - // Note: May want to check received ballot against our own? - *latest = Some(Instant::now()); - }), - ); + let p_latest_received_i_am_leader = p_to_proposers_i_am_leader + .clone() + .tick_batch() + .persist() + .fold( + q!(|| None), + q!(|latest: &mut Option, _: Ballot| { + // Note: May want to check received ballot against our own? + *latest = Some(Instant::now()); + }), + ); // Add random delay depending on node ID so not everyone sends p1a at the same time let p_leader_expired = flow.source_interval_delayed(proposers, q!(Duration::from_secs((p_id * i_am_leader_check_timeout_delay_multiplier as u32).into())), q!(Duration::from_secs(i_am_leader_check_timeout))) .tick_batch() @@ -1100,6 +1125,7 @@ fn p_p1a<'a>( })); p_leader_expired .clone() + .all_ticks() .for_each(q!(|_| println!("Proposer leader expired"))); let p_to_acceptors_p1a = p_ballot_num @@ -1111,6 +1137,24 @@ fn p_p1a<'a>( id: p_id } })) + .all_ticks() .broadcast_bincode_interleaved(acceptors); (p_to_proposers_i_am_leader_new, p_to_acceptors_p1a) } + +#[cfg(test)] +mod tests { + use hydroflow_plus_deploy::DeployRuntime; + use stageleft::RuntimeData; + + #[test] + fn paxos_ir() { + let builder = hydroflow_plus::FlowBuilder::new(); + let _ = super::paxos(&builder, 1, 1, 1, 1, 1, 1, 1); + let built = builder.with_default_optimize(); + + insta::assert_debug_snapshot!(built.ir()); + + let _ = built.compile::(&RuntimeData::new("FAKE")); + } +} diff --git a/hydroflow_plus_test/src/cluster/simple_cluster.rs b/hydroflow_plus_test/src/cluster/simple_cluster.rs index 1ce57304e517..e50965f52908 100644 --- a/hydroflow_plus_test/src/cluster/simple_cluster.rs +++ b/hydroflow_plus_test/src/cluster/simple_cluster.rs @@ -15,10 +15,12 @@ pub fn simple_cluster(flow: &FlowBuilder) -> (Process<()>, Cluster<()>) { ids.cross_product(numbers) .map(q!(|(id, n)| (id, (id, n)))) .send_bincode(&cluster) + .tick_batch() .inspect(q!(move |n| println!( "cluster received: {:?} (self cluster id: {})", n, cluster_self_id ))) + .all_ticks() .send_bincode(&process) .for_each(q!(|(id, d)| println!("node received: ({}, {:?})", id, d))); diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap new file mode 100644 index 000000000000..4e569fa25362 --- /dev/null +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap @@ -0,0 +1,5416 @@ +--- +source: hydroflow_plus_test/src/cluster/paxos.rs +expression: built.ir() +--- +[ + ForEach { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ballot : Ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + input: DeferTick( + AntiJoin( + Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + ), + ), + }, + CycleSink { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + input: DeferTick( + ReduceKeyed { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_time : & mut SystemTime , new_time : SystemTime | { if new_time > * curr_time { * curr_time = new_time ; } } }, + input: Union( + Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let num_clients_per_node = 1usize ; move | now : SystemTime | (0 .. num_clients_per_node) . map (move | virtual_id | (virtual_id , now)) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | _ : Ballot | SystemTime :: now () }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | key : u32 | (key as usize , SystemTime :: now ()) }, + input: Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + ), + }, + ForEach { + f: { use crate :: __staged :: cluster :: paxos :: * ; let median_latency_window_size = 1usize ; move | ((_ , (latencies , _write_index , has_any_value)) , (throughput , num_ticks) ,) : ((() , (Rc < RefCell < Vec < u128 > > > , usize , bool)) , (u32 , u32)) | { let mut latencies_mut = latencies . borrow_mut () ; let median_latency = if has_any_value { let out = latencies_mut . select_nth_unstable (median_latency_window_size / 2) ; * out . 1 } else { 0 } ; println ! ("Median latency: {}ms" , median_latency as f64 / 1000.0) ; println ! ("Throughput: {} requests/s" , throughput) ; println ! ("Num ticks per second: {}" , num_ticks) ; } }, + input: CrossSingleton( + CrossSingleton( + Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 2, + ), + }, + }, + }, + Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; let median_latency_window_size = 1usize ; move | | (Rc :: new (RefCell :: new (Vec :: < u128 > :: with_capacity (median_latency_window_size))) , 0usize , false) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; let median_latency_window_size = 1usize ; move | (latencies , write_index , has_any_value) : & mut (Rc < RefCell < Vec < u128 > > > , usize , bool) , latency : Option < u128 > | { let mut latencies_mut = latencies . borrow_mut () ; if let Some (latency) = latency { if let Some (prev_latency) = latencies_mut . get_mut (* write_index) { * prev_latency = latency ; } else { latencies_mut . push (latency) ; } * has_any_value = true ; * write_index += 1 ; if * write_index == median_latency_window_size { * write_index = 0 ; } } else { latencies_mut . clear () ; * write_index = 0 ; * has_any_value = false ; } } }, + input: Persist( + Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_virtual_id , (prev_time , curr_time)) : (usize , (SystemTime , SystemTime)) | Some (curr_time . duration_since (prev_time) . unwrap () . as_micros ()) }, + input: Join( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + Tee { + inner: RefCell { + value: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | key : u32 | (key as usize , SystemTime :: now ()) }, + input: Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | _ : () | None }, + input: Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + ), + ), + ), + }, + ), + Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , 0) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | (total , num_ticks) : & mut (u32 , u32) , (batch_size , reset) : (usize , bool) | { if reset { * total = 0 ; * num_ticks = 0 ; } else { * total += batch_size as u32 ; * num_ticks += 1 ; } } }, + input: Persist( + Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | batch_size : usize | (batch_size , false) }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | _ : () | (0 , true) }, + input: Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + ), + ), + ), + }, + ), + }, + ForEach { + f: { use crate :: __staged :: cluster :: paxos :: * ; | s | println ! ("{}" , s) }, + input: Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; ["Proposers say hello"] }, + ), + location_kind: Cluster( + 0, + ), + }, + }, + ForEach { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) : (u32 , P1b) | println ! ("Proposer received P1b: {:?}" , p1b) }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + input: Union( + Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; [0] }, + ), + location_kind: Cluster( + 0, + ), + }, + DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) : (Ballot , u32) | { if received_max_ballot > (Ballot { num : ballot_num , id : p_id , }) { received_max_ballot . num + 1 } else { ballot_num } } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | Ballot { num : 0 , id : 0 } }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Union( + Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) : (_ , P1b) | p1b . max_ballot }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) : (_ , P2b) | p2b . max_ballot }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + ), + }, + }, + }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + ), + ), + }, + ForEach { + f: { use crate :: __staged :: cluster :: paxos :: * ; | _ | println ! ("Proposer leader expired") }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; move | (_ , latest_received_i_am_leader) : & (_ , Option < Instant >) | { if let Some (latest_received_i_am_leader) = latest_received_i_am_leader { (Instant :: now () . duration_since (* latest_received_i_am_leader)) > Duration :: from_secs (i_am_leader_check_timeout) } else { true } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + CrossSingleton( + Source { + source: Stream( + { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 0, + ), + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | None }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | latest : & mut Option < Instant > , _ : Ballot | { * latest = Some (Instant :: now ()) ; } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num : u32 | Ballot { num : ballot_num , id : p_id } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_send_timeout = 1u64 ; Duration :: from_secs (i_am_leader_send_timeout) }, + ), + location_kind: Cluster( + 0, + ), + }, + }, + ), + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received : usize | if num_received > f { Some (true) } else { None } }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Unique( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) : ((u32 , P1b) , u32) | { sender } }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) : & (Ballot , u32) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | Ballot { num : 0 , id : 0 } }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Union( + Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) : (_ , P1b) | p1b . max_ballot }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) : (_ , P2b) | p2b . max_ballot }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + ), + }, + }, + }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + CycleSink { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + input: DeferTick( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Union( + Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : i32 | max_slot + 1 }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : & mut i32 , (slot , (_count , _entry)) : (i32 , (u32 , LogValue)) | { if slot > * max_slot { * max_slot = slot ; } } }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , next_slot) : (usize , i32) | next_slot + num_payloads as i32 }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | num_payloads : & usize | * num_payloads > 0 }, + input: Tee { + inner: RefCell { + value: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | num_payloads : & usize | * num_payloads > 0 }, + input: Tee { + inner: RefCell { + value: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | _ : bool | 0 }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Union( + Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : i32 | max_slot + 1 }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : & mut i32 , (slot , (_count , _entry)) : (i32 , (u32 , LogValue)) | { if slot > * max_slot { * max_slot = slot ; } } }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , next_slot) : (usize , i32) | next_slot + num_payloads as i32 }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | num_payloads : & usize | * num_payloads > 0 }, + input: Tee { + inner: RefCell { + value: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | num_payloads : & usize | * num_payloads > 0 }, + input: Tee { + inner: RefCell { + value: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + ), + ), + }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 2, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_2 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (_is_leader , ballot_num) : (bool , u32) | Ballot { num : ballot_num , id : p_id } }, + input: CrossSingleton( + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + input: DeferTick( + Difference( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (slot , (_count , _p2b)) : (i32 , (usize , P2b)) | slot }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (slot , (count , _p2b)) : (i32 , (usize , P2b)) | if count == 2 * f + 1 { Some (slot) } else { None } }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + ), + }, + CycleSink { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + input: DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (sender , p2b)) : (i32 , (u32 , P2b)) | (sender , p2b) }, + input: AntiJoin( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | (p2b . slot , (sender , p2b)) }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (slot , (count , _p2b)) : (i32 , (usize , P2b)) | if count == 2 * f + 1 { Some (slot) } else { None } }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + ), + }, + ForEach { + f: { use crate :: __staged :: cluster :: paxos :: * ; | s | println ! ("{}" , s) }, + input: Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; ["Acceptors say hello"] }, + ), + location_kind: Cluster( + 1, + ), + }, + }, + ForEach { + f: { use crate :: __staged :: cluster :: paxos :: * ; | p1a : P1a | println ! ("Acceptor received P1a: {:?}" , p1a) }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num : u32 | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; move | (_ , latest_received_i_am_leader) : & (_ , Option < Instant >) | { if let Some (latest_received_i_am_leader) = latest_received_i_am_leader { (Instant :: now () . duration_since (* latest_received_i_am_leader)) > Duration :: from_secs (i_am_leader_check_timeout) } else { true } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + CrossSingleton( + Source { + source: Stream( + { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 0, + ), + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | None }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | latest : & mut Option < Instant > , _ : Ballot | { * latest = Some (Instant :: now ()) ; } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + input: Network { + from_location: Cluster( + 1, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , log)) : ((P1a , Ballot) , (i32 , HashMap :: < i32 , LogValue >)) | (p1a . ballot . id , P1b { ballot : p1a . ballot , max_ballot , accepted : log }) }, + input: CrossSingleton( + CrossSingleton( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num : u32 | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; move | (_ , latest_received_i_am_leader) : & (_ , Option < Instant >) | { if let Some (latest_received_i_am_leader) = latest_received_i_am_leader { (Instant :: now () . duration_since (* latest_received_i_am_leader)) > Duration :: from_secs (i_am_leader_check_timeout) } else { true } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + CrossSingleton( + Source { + source: Stream( + { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 0, + ), + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | None }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | latest : & mut Option < Instant > , _ : Ballot | { * latest = Some (Instant :: now ()) ; } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | Ballot { num : 0 , id : 0 } }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_ballot : & mut Ballot , p1a : P1a | { if p1a . ballot > * max_ballot { * max_ballot = p1a . ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num : u32 | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; move | (_ , latest_received_i_am_leader) : & (_ , Option < Instant >) | { if let Some (latest_received_i_am_leader) = latest_received_i_am_leader { (Instant :: now () . duration_since (* latest_received_i_am_leader)) > Duration :: from_secs (i_am_leader_check_timeout) } else { true } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + CrossSingleton( + Source { + source: Stream( + { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 0, + ), + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | None }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | latest : & mut Option < Instant > , _ : Ballot | { * latest = Some (Instant :: now ()) ; } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + ), + Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: < i32 , LogValue > :: new ()) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) : & mut (i32 , HashMap :: < i32 , LogValue >) , (new_checkpoint , p2a) : (i32 , P2a) | { if new_checkpoint != - 1 { for slot in * prev_checkpoint .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = new_checkpoint ; } else { if p2a . slot > * prev_checkpoint { match log . get (& p2a . slot) { None => { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } Some (prev_p2a) => { if p2a . ballot > prev_p2a . ballot { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } ; } } } }, + input: Persist( + Union( + FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) : (P2a , Ballot) | if p2a . ballot >= max_ballot { Some ((- 1 , p2a)) } else { None } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Union( + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Union( + FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((slot , (count , entry)) , ballot_num) : ((i32 , (u32 , LogValue)) , u32) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , id : p_id , } , slot , value : entry . value , }) } else { None } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (slot , ballot_num) : (i32 , u32) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : ClientPayload { key : 0 , value : "0" . to_string () } } }, + input: CrossSingleton( + Difference( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : i32 | 0 .. max_slot }, + input: Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : & mut i32 , (slot , (_count , _entry)) : (i32 , (u32 , LogValue)) | { if slot > * max_slot { * max_slot = slot ; } } }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) : (i32 , (u32 , LogValue)) | slot }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (((index , payload) , next_slot) , ballot_num) : (((usize , ClientPayload) , i32) , u32) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }, + input: CrossSingleton( + CrossSingleton( + Enumerate( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | Ballot { num : 0 , id : 0 } }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_ballot : & mut Ballot , p1a : P1a | { if p1a . ballot > * max_ballot { * max_ballot = p1a . ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num : u32 | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; move | (_ , latest_received_i_am_leader) : & (_ , Option < Instant >) | { if let Some (latest_received_i_am_leader) = latest_received_i_am_leader { (Instant :: now () . duration_since (* latest_received_i_am_leader)) > Duration :: from_secs (i_am_leader_check_timeout) } else { true } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + CrossSingleton( + Source { + source: Stream( + { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 0, + ), + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | None }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | latest : & mut Option < Instant > , _ : Ballot | { * latest = Some (Instant :: now ()) ; } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + ), + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | min_seq : i32 | (min_seq , P2a { ballot : Ballot { num : 0 , id : 0 } , slot : - 1 , value : ClientPayload { key : 0 , value : "" . to_string () , } }) }, + input: Delta( + Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | min_seq : & mut i32 , (_sender , seq) : (u32 , i32) | { if * min_seq == - 1 || seq < * min_seq { * min_seq = seq ; } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: ReduceKeyed { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_seq : & mut i32 , seq : i32 | { if seq > * curr_seq { * curr_seq = seq ; } } }, + input: Persist( + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + }, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received : usize | if num_received == f + 1 { Some (true) } else { None } }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: ReduceKeyed { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_seq : & mut i32 , seq : i32 | { if seq > * curr_seq { * curr_seq = seq ; } } }, + input: Persist( + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + }, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + ), + }, + ), + ), + }, + ), + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + input: Network { + from_location: Cluster( + 1, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) : (P2a , Ballot) | (p2a . ballot . id , P2b { ballot : p2a . ballot , max_ballot , slot : p2a . slot , value : p2a . value }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Union( + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Union( + FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((slot , (count , entry)) , ballot_num) : ((i32 , (u32 , LogValue)) , u32) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , id : p_id , } , slot , value : entry . value , }) } else { None } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (slot , ballot_num) : (i32 , u32) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : ClientPayload { key : 0 , value : "0" . to_string () } } }, + input: CrossSingleton( + Difference( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : i32 | 0 .. max_slot }, + input: Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_slot : & mut i32 , (slot , (_count , _entry)) : (i32 , (u32 , LogValue)) | { if slot > * max_slot { * max_slot = slot ; } } }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) : (i32 , (u32 , LogValue)) | slot }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_entry : & mut (u32 , LogValue) , new_entry : LogValue | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }, + input: FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) : ((u32 , P1b) , u32) | p1b . accepted . into_iter () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) : & ((u32 , P1b) , u32) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }, + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (((index , payload) , next_slot) , ballot_num) : (((usize , ClientPayload) , i32) , u32) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }, + input: CrossSingleton( + CrossSingleton( + Enumerate( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 2, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Union( + FlatMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot : Ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }, + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) : (u32 , Ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (key , count) : (u32 , usize) | { if count == f + 1 { Some (key) } else { None } } }, + input: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | 0 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | curr_count : & mut usize , _sender : u32 | { * curr_count += 1 ; } }, + input: Tee { + inner: RefCell { + value: Union( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) : (u32 , ReplicaPayload) | (replica_payload . key , sender) }, + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Reduce { + f: { use crate :: __staged :: cluster :: paxos :: * ; | curr_max_ballot : & mut Ballot , new_ballot : Ballot | { if new_ballot > * curr_max_ballot { * curr_max_ballot = new_ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | Ballot { num : 0 , id : 0 } }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_ballot : & mut Ballot , p1a : P1a | { if p1a . ballot > * max_ballot { * max_ballot = p1a . ballot ; } } }, + input: Persist( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num : u32 | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; move | (_ , latest_received_i_am_leader) : & (_ , Option < Instant >) | { if let Some (latest_received_i_am_leader) = latest_received_i_am_leader { (Instant :: now () . duration_since (* latest_received_i_am_leader)) > Duration :: from_secs (i_am_leader_check_timeout) } else { true } } }, + input: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }, + input: CrossSingleton( + CrossSingleton( + Source { + source: Stream( + { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 0, + ), + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | None }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | latest : & mut Option < Instant > , _ : Ballot | { * latest = Some (Instant :: now ()) ; } }, + input: Persist( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + ), + }, + }, + }, + ), + Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | _u | () }, + input: Filter { + f: { use hydroflow_plus :: __staged :: stream :: * ; | c | * c == 0 }, + input: Fold { + init: { use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }, + acc: { use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }, + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + input: DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) : (ReplicaPayload , i32) | { sorted_payload } }, + input: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) : & (ReplicaPayload , i32) | sorted_payload . seq > * highest_seq }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | filled_slot : & mut i32 , (sorted_payload , highest_seq) : (ReplicaPayload , i32) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 3, + ), + }, + Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; [- 1] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + ), + }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 3, + ), + input: Tee { + inner: RefCell { + value: DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_kv_store , highest_seq) : (HashMap :: < u32 , String > , i32) | highest_seq }, + input: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (HashMap :: < u32 , String > :: new () , - 1) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | state : & mut (HashMap :: < u32 , String > , i32) , payload : ReplicaPayload | { let kv_store = & mut state . 0 ; let last_seq = & mut state . 1 ; kv_store . insert (payload . key , payload . value) ; debug_assert ! (payload . seq == * last_seq + 1 , "Hole in log between seq {} and {}" , * last_seq , payload . seq) ; * last_seq = payload . seq ; } }, + input: Persist( + Tee { + inner: RefCell { + value: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) : (ReplicaPayload , i32) | { sorted_payload } }, + input: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) : & (ReplicaPayload , i32) | sorted_payload . seq <= * highest_seq }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | filled_slot : & mut i32 , (sorted_payload , highest_seq) : (ReplicaPayload , i32) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 3, + ), + }, + Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; [- 1] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + ), + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 3, + ), + input: Tee { + inner: RefCell { + value: DeferTick( + FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let checkpoint_frequency = 1usize ; move | (max_checkpointed_seq , new_highest_seq) : (i32 , i32) | if new_highest_seq - max_checkpointed_seq >= checkpoint_frequency as i32 { Some (new_highest_seq) } else { None } }, + input: CrossSingleton( + Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_seq : & mut i32 , seq : i32 | { if seq > * max_seq { * max_seq = seq ; } } }, + input: Persist( + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 3, + ), + }, + ), + }, + Tee { + inner: RefCell { + value: DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_kv_store , highest_seq) : (HashMap :: < u32 , String > , i32) | highest_seq }, + input: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (HashMap :: < u32 , String > :: new () , - 1) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | state : & mut (HashMap :: < u32 , String > , i32) , payload : ReplicaPayload | { let kv_store = & mut state . 0 ; let last_seq = & mut state . 1 ; kv_store . insert (payload . key , payload . value) ; debug_assert ! (payload . seq == * last_seq + 1 , "Hole in log between seq {} and {}" , * last_seq , payload . seq) ; * last_seq = payload . seq ; } }, + input: Persist( + Tee { + inner: RefCell { + value: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) : (ReplicaPayload , i32) | { sorted_payload } }, + input: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) : & (ReplicaPayload , i32) | sorted_payload . seq <= * highest_seq }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | filled_slot : & mut i32 , (sorted_payload , highest_seq) : (ReplicaPayload , i32) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 3, + ), + }, + Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; [- 1] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + ), + }, + }, + ), + }, + ), + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + input: Network { + from_location: Cluster( + 3, + ), + to_location: Cluster( + 2, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | payload : ReplicaPayload | (payload . value . parse :: < u32 > () . unwrap () , payload) }, + input: Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + input: Network { + from_location: Cluster( + 3, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Tee { + inner: RefCell { + value: DeferTick( + FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; let checkpoint_frequency = 1usize ; move | (max_checkpointed_seq , new_highest_seq) : (i32 , i32) | if new_highest_seq - max_checkpointed_seq >= checkpoint_frequency as i32 { Some (new_highest_seq) } else { None } }, + input: CrossSingleton( + Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | max_seq : & mut i32 , seq : i32 | { if seq > * max_seq { * max_seq = seq ; } } }, + input: Persist( + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 3, + ), + }, + ), + }, + Tee { + inner: RefCell { + value: DeferTick( + Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_kv_store , highest_seq) : (HashMap :: < u32 , String > , i32) | highest_seq }, + input: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (HashMap :: < u32 , String > :: new () , - 1) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | state : & mut (HashMap :: < u32 , String > , i32) , payload : ReplicaPayload | { let kv_store = & mut state . 0 ; let last_seq = & mut state . 1 ; kv_store . insert (payload . key , payload . value) ; debug_assert ! (payload . seq == * last_seq + 1 , "Hole in log between seq {} and {}" , * last_seq , payload . seq) ; * last_seq = payload . seq ; } }, + input: Persist( + Tee { + inner: RefCell { + value: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) : (ReplicaPayload , i32) | { sorted_payload } }, + input: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) : & (ReplicaPayload , i32) | sorted_payload . seq <= * highest_seq }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Fold { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | filled_slot : & mut i32 , (sorted_payload , highest_seq) : (ReplicaPayload , i32) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }, + input: CrossSingleton( + Tee { + inner: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: { use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }, + input: Network { + from_location: Cluster( + 0, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: { use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }, + input: Map { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) : (i32 , (usize , P2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }, + input: AntiJoin( + Tee { + inner: RefCell { + value: Filter { + f: { use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) : & (i32 , (usize , P2b)) | * count > f }, + input: Tee { + inner: RefCell { + value: FoldKeyed { + init: { use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }, + acc: { use crate :: __staged :: cluster :: paxos :: * ; | accum : & mut (usize , P2b) , (_sender , p2b) : (u32 , P2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }, + input: FilterMap { + f: { use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) : (u32 , P2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }, + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_7, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 3, + ), + }, + Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; [- 1] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + ), + }, + }, + ), + }, + ), + }, + }, + }, + }, + }, +] diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap index 0600ed0215ca..66b4ff1f871d 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap @@ -65,26 +65,32 @@ expression: built.ir() ), input: Map { f: { use crate :: __staged :: cluster :: simple_cluster :: * ; | (id , n) | (id , (id , n)) }, - input: CrossProduct( - Map { - f: { use crate :: __staged :: cluster :: simple_cluster :: * ; | & id | id }, - input: Source { - source: Iter( - __hydroflow_plus_cluster_ids_1, - ), - location_kind: Process( - 0, - ), - }, - }, - Source { - source: Iter( - { use crate :: __staged :: cluster :: simple_cluster :: * ; 0 .. 5 }, + input: Delta( + CrossProduct( + Persist( + Map { + f: { use crate :: __staged :: cluster :: simple_cluster :: * ; | & id | id }, + input: Source { + source: Iter( + __hydroflow_plus_cluster_ids_1, + ), + location_kind: Process( + 0, + ), + }, + }, ), - location_kind: Process( - 0, + Persist( + Source { + source: Iter( + { use crate :: __staged :: cluster :: simple_cluster :: * ; 0 .. 5 }, + ), + location_kind: Process( + 0, + ), + }, ), - }, + ), ), }, }, diff --git a/hydroflow_plus_test_local/src/local/chat_app.rs b/hydroflow_plus_test_local/src/local/chat_app.rs index 4b568f85d369..2e2e281bd7f2 100644 --- a/hydroflow_plus_test_local/src/local/chat_app.rs +++ b/hydroflow_plus_test_local/src/local/chat_app.rs @@ -14,10 +14,13 @@ pub fn chat_app<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let users = flow.source_stream(&process, users_stream).all_ticks(); + let users = flow + .source_stream(&process, users_stream) + .tick_batch() + .persist(); let messages = flow.source_stream(&process, messages); let messages = if replay_messages { - messages.all_ticks() + messages.tick_batch().persist() } else { messages.tick_batch() }; @@ -30,7 +33,7 @@ pub fn chat_app<'a>( joined = joined.delta(); } - joined.for_each(q!(|t| { + joined.all_ticks().for_each(q!(|t| { output.send(t).unwrap(); })); diff --git a/hydroflow_plus_test_local/src/local/compute_pi.rs b/hydroflow_plus_test_local/src/local/compute_pi.rs index c09273c72add..f700f30e9fda 100644 --- a/hydroflow_plus_test_local/src/local/compute_pi.rs +++ b/hydroflow_plus_test_local/src/local/compute_pi.rs @@ -23,11 +23,12 @@ pub fn compute_pi(flow: &FlowBuilder, batch_size: RuntimeData) -> Process ); trials - .all_ticks() + .persist() .reduce(q!(|(inside, total), (inside_batch, total_batch)| { *inside += inside_batch; *total += total_batch; })) + .all_ticks() .sample_every(q!(Duration::from_secs(1))) .for_each(q!(|(inside, total)| { println!( diff --git a/hydroflow_plus_test_local/src/local/count_elems.rs b/hydroflow_plus_test_local/src/local/count_elems.rs index 64cfa440be5d..098d1b92f128 100644 --- a/hydroflow_plus_test_local/src/local/count_elems.rs +++ b/hydroflow_plus_test_local/src/local/count_elems.rs @@ -16,7 +16,8 @@ pub fn count_elems<'a, T: 'a>( let count = source .map(q!(|_| 1)) .tick_batch() - .fold(q!(|| 0), q!(|a, b| *a += b)); + .fold(q!(|| 0), q!(|a, b| *a += b)) + .all_ticks(); count.for_each(q!(|v| { output.send(v).unwrap(); diff --git a/hydroflow_plus_test_local/src/local/graph_reachability.rs b/hydroflow_plus_test_local/src/local/graph_reachability.rs index 1febe37fd9ad..fcea5f07f39b 100644 --- a/hydroflow_plus_test_local/src/local/graph_reachability.rs +++ b/hydroflow_plus_test_local/src/local/graph_reachability.rs @@ -13,20 +13,20 @@ pub fn graph_reachability<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let roots = flow.source_stream(&process, roots); + let roots = flow.source_stream(&process, roots).tick_batch(); let edges = flow.source_stream(&process, edges); - let (set_reached_cycle, reached_cycle) = flow.cycle(&process); + let (set_reached_cycle, reached_cycle) = flow.tick_cycle(&process); let reached = roots.union(reached_cycle); let reachable = reached .clone() .map(q!(|r| (r, ()))) - .join(edges) + .join(edges.tick_batch().persist()) .map(q!(|(_from, (_, to))| to)); set_reached_cycle.complete(reachable); - reached.tick_batch().unique().for_each(q!(|v| { + reached.unique().all_ticks().for_each(q!(|v| { reached_out.send(v).unwrap(); })); diff --git a/hydroflow_plus_test_local/src/local/negation.rs b/hydroflow_plus_test_local/src/local/negation.rs index dedc3d036e28..8e4180d5db5d 100644 --- a/hydroflow_plus_test_local/src/local/negation.rs +++ b/hydroflow_plus_test_local/src/local/negation.rs @@ -12,17 +12,17 @@ pub fn test_difference<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let mut source = flow.source_iter(&process, q!(0..5)); + let mut source = flow.source_iter(&process, q!(0..5)).tick_batch(); if persist1 { - source = source.all_ticks(); + source = source.persist(); } - let mut source2 = flow.source_iter(&process, q!(3..6)); + let mut source2 = flow.source_iter(&process, q!(3..6)).tick_batch(); if persist2 { - source2 = source2.all_ticks(); + source2 = source2.persist(); } - source.filter_not_in(source2).for_each(q!(|v| { + source.filter_not_in(source2).all_ticks().for_each(q!(|v| { output.send(v).unwrap(); })); @@ -39,20 +39,26 @@ pub fn test_anti_join<'a>( ) -> impl Quoted<'a, Hydroflow<'a>> { let process = flow.process::<()>(); - let mut source = flow.source_iter(&process, q!(0..5)).map(q!(|v| (v, v))); + let mut source = flow + .source_iter(&process, q!(0..5)) + .map(q!(|v| (v, v))) + .tick_batch(); if persist1 { - source = source.all_ticks(); + source = source.persist(); } - let mut source2 = flow.source_iter(&process, q!(3..6)); + let mut source2 = flow.source_iter(&process, q!(3..6)).tick_batch(); if persist2 { - source2 = source2.all_ticks(); + source2 = source2.persist(); } // TODO(shadaj): inference fails without a for_each type annotation here - source.anti_join(source2).for_each(q!(|v: (u32, u32)| { - output.send(v.0).unwrap(); - })); + source + .anti_join(source2) + .all_ticks() + .for_each(q!(|v: (u32, u32)| { + output.send(v.0).unwrap(); + })); flow.with_default_optimize() .compile_no_network::() diff --git a/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_dot.snap b/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_dot.snap index 2e581ba979f6..2dc293d37588 100644 --- a/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_dot.snap +++ b/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_dot.snap @@ -10,7 +10,7 @@ digraph { n3v1 [label="(n3v1) tee()", shape=house, fillcolor="#ffff88"] n4v1 [label="(n4v1) map({\l use crate::__staged::local::graph_reachability::*;\l |r| (r, ())\l})\l", shape=house, fillcolor="#ffff88"] n5v1 [label="(n5v1) source_stream(edges)", shape=invhouse, fillcolor="#88aaff"] - n6v1 [label="(n6v1) join_multiset::<'tick, 'tick>()", shape=invhouse, fillcolor="#88aaff"] + n6v1 [label="(n6v1) join_multiset::<'tick, 'static>()", shape=invhouse, fillcolor="#88aaff"] n7v1 [label="(n7v1) map({\l use crate::__staged::local::graph_reachability::*;\l |(_from, (_, to))| to\l})\l", shape=invhouse, fillcolor="#88aaff"] n8v1 [label="(n8v1) unique::<'tick>()", shape=house, fillcolor="#ffff88"] n9v1 [label="(n9v1) for_each({\l use crate::__staged::local::graph_reachability::*;\l let reached_out = reached_out;\l |v| {\l reached_out.send(v).unwrap();\l }\l})\l", shape=house, fillcolor="#ffff88"] diff --git a/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_mermaid.snap b/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_mermaid.snap index 69013b449fe5..69a991e2cf42 100644 --- a/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_mermaid.snap +++ b/hydroflow_plus_test_local/src/local/snapshots/hydroflow_plus_test_local__local__graph_reachability__tests__reachability@graphvis_mermaid.snap @@ -13,7 +13,7 @@ linkStyle default stroke:#aaa 3v1[/"(3v1) tee()"\]:::pushClass 4v1[/"
(4v1)
map({
use crate::__staged::local::graph_reachability::*;
|r| (r, ())
})
"\]:::pushClass 5v1[\"(5v1) source_stream(edges)"/]:::pullClass -6v1[\"(6v1) join_multiset::<'tick, 'tick>()"/]:::pullClass +6v1[\"(6v1) join_multiset::<'tick, 'static>()"/]:::pullClass 7v1[\"
(7v1)
map({
use crate::__staged::local::graph_reachability::*;
|(_from, (_, to))| to
})
"/]:::pullClass 8v1[/"(8v1) unique::<'tick>()"\]:::pushClass 9v1[/"
(9v1)
for_each({
use crate::__staged::local::graph_reachability::*;
let reached_out = reached_out;
|v| {
reached_out.send(v).unwrap();
}
})
"\]:::pushClass diff --git a/hydroflow_plus_test_local/src/local/teed_join.rs b/hydroflow_plus_test_local/src/local/teed_join.rs index d0fe28434ba8..42ffb698a165 100644 --- a/hydroflow_plus_test_local/src/local/teed_join.rs +++ b/hydroflow_plus_test_local/src/local/teed_join.rs @@ -19,18 +19,18 @@ pub fn teed_join<'a, S: Stream + Unpin + 'a>( let node_zero = flow.process::(); let node_one = flow.process::(); - let source = flow.source_stream(&node_zero, input_stream); + let source = flow.source_stream(&node_zero, input_stream).tick_batch(); let map1 = source.clone().map(q!(|v| (v + 1, ()))); let map2 = source.map(q!(|v| (v - 1, ()))); let joined = map1.join(map2).map(q!(|t| t.0)); - joined.clone().for_each(q!(|v| { + joined.clone().all_ticks().for_each(q!(|v| { output.send(v).unwrap(); })); if send_twice { - joined.for_each(q!(|v| { + joined.all_ticks().for_each(q!(|v| { output.send(v).unwrap(); })); }