From 3d4d628e10861f71f8a9969117df81462b6360de Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Thu, 21 Nov 2024 12:43:18 -0800 Subject: [PATCH] refactor(paxos): generalize quorum logic --- hydroflow_plus/src/optional.rs | 7 + hydroflow_plus/src/stream.rs | 68 +- hydroflow_plus_test/src/cluster/paxos.rs | 558 +++--- .../src/cluster/paxos_bench.rs | 95 +- hydroflow_plus_test/src/cluster/paxos_kv.rs | 2 +- ...cluster__paxos_bench__tests__paxos_ir.snap | 1508 +++++++++-------- .../src/local/graph_reachability.rs | 4 +- stageleft/src/type_name.rs | 26 +- 8 files changed, 1251 insertions(+), 1017 deletions(-) diff --git a/hydroflow_plus/src/optional.rs b/hydroflow_plus/src/optional.rs index 9fa8609d5297..bfde8c2ef5e1 100644 --- a/hydroflow_plus/src/optional.rs +++ b/hydroflow_plus/src/optional.rs @@ -213,6 +213,13 @@ impl<'a, T, L: Location<'a>, B> Optional { ) } + pub fn flatten(self) -> Stream + where + T: IntoIterator, + { + self.flat_map(q!(|v| v)) + } + pub fn filter bool + 'a>( self, f: impl IntoQuotedMut<'a, F, L>, diff --git a/hydroflow_plus/src/stream.rs b/hydroflow_plus/src/stream.rs index 266c64500e16..e3ec2c7a9a36 100644 --- a/hydroflow_plus/src/stream.rs +++ b/hydroflow_plus/src/stream.rs @@ -73,6 +73,16 @@ pub struct Stream { _phantom: PhantomData<(T, L, B, Order)>, } +impl<'a, T, L: Location<'a>, B> From> for Stream { + fn from(stream: Stream) -> Stream { + Stream { + location: stream.location, + ir_node: stream.ir_node, + _phantom: PhantomData, + } + } +} + impl<'a, T, L: Location<'a>, B, Order> Stream { fn location_kind(&self) -> LocationId { self.location.id() @@ -417,6 +427,33 @@ where })) } + pub fn max_by_key K + 'a>( + self, + key: impl IntoQuotedMut<'a, F, L> + Copy, + ) -> Optional { + let f = key.splice_fn1_borrow_ctx(&self.location); + + let wrapped: syn::Expr = parse_quote!({ + let key_fn = #f; + move |curr, new| { + if key_fn(&new) > key_fn(&*curr) { + *curr = new; + } + } + }); + + let mut core = HfPlusNode::Reduce { + f: wrapped.into(), + input: Box::new(self.ir_node.into_inner()), + }; + + if L::is_top_level() { + core = HfPlusNode::Persist(Box::new(core)); + } + + Optional::new(self.location, core) + } + pub fn min(self) -> Optional where T: Ord, @@ -504,6 +541,35 @@ impl<'a, T, L: Location<'a>, B> Stream { } } +impl<'a, T, L: Location<'a>> Stream { + pub fn chain( + self, + other: Stream, + ) -> Stream { + check_matching_location(&self.location, &other.location); + + Stream::new( + self.location, + HfPlusNode::Chain( + Box::new(self.ir_node.into_inner()), + Box::new(other.ir_node.into_inner()), + ), + ) + } +} + +impl<'a, T, L: Location<'a> + NoTick> Stream { + pub fn union( + self, + other: Stream, + ) -> Stream { + let tick = self.location.tick(); + self.tick_batch(&tick) + .union(other.tick_batch(&tick)) + .all_ticks() + } +} + impl<'a, T, L: Location<'a>, Order> Stream { pub fn sort(self) -> Stream where @@ -515,7 +581,7 @@ impl<'a, T, L: Location<'a>, Order> Stream { ) } - pub fn chain(self, other: Stream) -> Stream + pub fn union(self, other: Stream) -> Stream where Order: MinOrder, { diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 060775f8af83..a65152059370 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -1,11 +1,13 @@ use std::collections::HashMap; use std::fmt::Debug; +use std::hash::Hash; use std::time::Duration; use hydroflow_plus::*; +use location::NoTick; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; -use stream::{NoOrder, TotalOrder}; +use stream::NoOrder; use tokio::time::Instant; pub struct Proposer {} @@ -14,7 +16,7 @@ pub struct Acceptor {} pub trait PaxosPayload: Serialize + DeserializeOwned + PartialEq + Eq + Clone + Debug {} impl PaxosPayload for T {} -#[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug, Hash)] +#[derive(Serialize, Deserialize, PartialEq, Eq, Copy, Clone, Debug, Hash)] pub struct Ballot { pub num: u32, pub proposer_id: ClusterId, @@ -34,24 +36,12 @@ impl PartialOrd for Ballot { } } -#[derive(Serialize, Deserialize, Clone, Debug)] -struct P1a { - ballot: Ballot, -} - #[derive(Serialize, Deserialize, Clone, Debug)] struct LogValue

{ ballot: Ballot, value: Option

, // might be a hole } -#[derive(Serialize, Deserialize, Clone, Debug)] -struct P1b { - ballot: Ballot, - max_ballot: Ballot, - accepted: L, -} - #[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug)] struct P2a

{ ballot: Ballot, @@ -59,14 +49,6 @@ struct P2a

{ value: Option

, // might be a re-committed hole } -#[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug)] -struct P2b

{ - ballot: Ballot, - max_ballot: Ballot, - slot: usize, - value: Option

, // might be a hole -} - #[expect( clippy::too_many_arguments, clippy::type_complexity, @@ -101,11 +83,10 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( let proposer_tick = proposers.tick(); let acceptor_tick = acceptors.tick(); - let (a_to_proposers_p2b_complete_cycle, a_to_proposers_p2b_forward_reference) = - proposers.forward_ref::, _, _, NoOrder>>(); + let (sequencing_max_ballot_complete_cycle, sequencing_max_ballot_forward_reference) = + proposers.forward_ref::>(); let (a_log_complete_cycle, a_log_forward_reference) = - acceptor_tick - .forward_ref::, HashMap>), _, _>>(); + acceptor_tick.forward_ref::>(); let (p_ballot, p_is_leader, p_relevant_p1bs, a_max_ballot) = leader_election( proposers, @@ -116,46 +97,36 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( i_am_leader_send_timeout, i_am_leader_check_timeout, i_am_leader_check_timeout_delay_multiplier, - a_to_proposers_p2b_forward_reference.map(q!(|p2b| p2b.max_ballot)), - a_log_forward_reference.map(q!(|(_ckpnt, log)| log.clone())), + sequencing_max_ballot_forward_reference, + a_log_forward_reference, ); let just_became_leader = p_is_leader .clone() .continue_unless(p_is_leader.clone().defer_tick()); - // Tell clients that leader election has completed and they can begin sending messages - let p_to_clients_new_leader_elected = just_became_leader - .clone() - .then(p_ballot.clone()) // Only tell the clients once when leader election concludes - .all_ticks(); - - let (p_log_to_try_commit, p_max_slot, p_log_holes) = - recommit_after_leader_election(p_relevant_p1bs, p_ballot.clone(), f); - - let p_log_to_recommit = p_log_to_try_commit - .chain(p_log_holes) - .continue_if(just_became_leader); // Only resend p1b stuff once the moment we become leader. - - let (p_to_replicas, a_log, a_to_proposers_p2b) = sequence_payload( + let (p_to_replicas, a_log, sequencing_max_ballots) = sequence_payload( proposers, acceptors, &proposer_tick, &acceptor_tick, c_to_proposers, r_to_acceptors_checkpoint, - p_ballot, + p_ballot.clone(), p_is_leader, - p_max_slot, - p_log_to_recommit, + p_relevant_p1bs, f, a_max_ballot, ); a_log_complete_cycle.complete(a_log); - a_to_proposers_p2b_complete_cycle.complete(a_to_proposers_p2b); + sequencing_max_ballot_complete_cycle.complete(sequencing_max_ballots); - (p_to_clients_new_leader_elected, p_to_replicas) + ( + // Only tell the clients once when leader election concludes + just_became_leader.then(p_ballot).all_ticks(), + p_to_replicas, + ) } #[expect( @@ -176,26 +147,29 @@ fn leader_election<'a, L: Clone + Debug + Serialize + DeserializeOwned>( a_log: Singleton>, Bounded>, ) -> ( Singleton>, Bounded>, - Optional>, Bounded>, - Stream, Tick>, Bounded, NoOrder>, + Optional<(), Tick>, Bounded>, + Stream>, Bounded, NoOrder>, Singleton>, Bounded>, ) { - let (a_to_proposers_p1b_complete_cycle, a_to_proposers_p1b_forward_ref) = - proposers.forward_ref::, _, _, NoOrder>>(); + let (p1b_fail_complete, p1b_fail) = + proposers.forward_ref::>(); let (p_to_proposers_i_am_leader_complete_cycle, p_to_proposers_i_am_leader_forward_ref) = proposers.forward_ref::>(); let (p_is_leader_complete_cycle, p_is_leader_forward_ref) = - proposer_tick.forward_ref::>(); + proposer_tick.forward_ref::>(); // a_to_proposers_p2b.clone().for_each(q!(|(_, p2b): (u32, P2b)| println!("Proposer received P2b: {:?}", p2b))); // p_to_proposers_i_am_leader.clone().for_each(q!(|ballot: Ballot| println!("Proposer received I am leader: {:?}", ballot))); // c_to_proposers.clone().for_each(q!(|payload: ClientPayload| println!("Client sent proposer payload: {:?}", payload))); - let p_received_max_ballot = p_max_ballot( - proposers, - a_to_proposers_p1b_forward_ref.map(q!(|p1a| p1a.max_ballot)), - p_received_p2b_ballots, - p_to_proposers_i_am_leader_forward_ref, - ); + let p_received_max_ballot = p1b_fail + .union(p_received_p2b_ballots) + .union(p_to_proposers_i_am_leader_forward_ref) + .max() + .unwrap_or(proposers.singleton(q!(Ballot { + num: 0, + proposer_id: ClusterId::from_raw(0) + }))); + let (p_ballot, p_has_largest_ballot) = p_ballot_calc( proposer_tick, p_received_max_ballot.latest_tick(proposer_tick), @@ -213,13 +187,16 @@ fn leader_election<'a, L: Clone + Debug + Serialize + DeserializeOwned>( p_to_proposers_i_am_leader_complete_cycle.complete(p_to_proposers_i_am_leader); - let p_to_acceptors_p1a = p_p1a(p_ballot.clone(), p_trigger_election, acceptors); + let p_to_acceptors_p1a = p_trigger_election + .then(p_ballot.clone()) + .all_ticks() + .inspect(q!(|_| println!("Proposer leader expired, sending P1a"))) + .broadcast_bincode_interleaved(acceptors); let (a_max_ballot, a_to_proposers_p1b) = acceptor_p1(acceptor_tick, p_to_acceptors_p1a, a_log, proposers); - a_to_proposers_p1b_complete_cycle.complete(a_to_proposers_p1b.clone()); - let (p_is_leader, p_relevant_p1bs) = p_p1b( + let (p_is_leader, p_accepted_values, fail_ballots) = p_p1b( proposer_tick, a_to_proposers_p1b.inspect(q!(|p1b| println!("Proposer received P1b: {:?}", p1b))), p_ballot.clone(), @@ -227,28 +204,9 @@ fn leader_election<'a, L: Clone + Debug + Serialize + DeserializeOwned>( f, ); p_is_leader_complete_cycle.complete(p_is_leader.clone()); + p1b_fail_complete.complete(fail_ballots.all_ticks()); - (p_ballot, p_is_leader, p_relevant_p1bs, a_max_ballot) -} - -// Proposer logic to calculate the largest ballot received so far. -fn p_max_ballot<'a>( - proposers: &Cluster<'a, Proposer>, - p_received_p1b_ballots: Stream, Unbounded, NoOrder>, - p_received_p2b_ballots: Stream, Unbounded, NoOrder>, - p_to_proposers_i_am_leader: Stream, Unbounded, NoOrder>, -) -> Singleton, Unbounded> { - let ballot_batcher = proposers.tick(); - p_received_p1b_ballots - .tick_batch(&ballot_batcher) - .chain(p_received_p2b_ballots.tick_batch(&ballot_batcher)) - .chain(p_to_proposers_i_am_leader.tick_batch(&ballot_batcher)) - .all_ticks() - .max() - .unwrap_or(proposers.singleton(q!(Ballot { - num: 0, - proposer_id: ClusterId::from_raw(0) - }))) + (p_ballot, p_is_leader, p_accepted_values, a_max_ballot) } // Proposer logic to calculate the next ballot number. Expects p_received_max_ballot, the largest ballot received so far. Outputs streams: ballot_num, and has_largest_ballot, which only contains a value if we have the largest ballot. @@ -280,7 +238,7 @@ fn p_ballot_calc<'a>( })); p_ballot_num_complete_cycle.complete_next_tick(p_new_ballot_num); - let p_ballot = p_ballot_num.clone().map(q!(move |num| Ballot { + let p_ballot = p_ballot_num.map(q!(move |num| Ballot { num, proposer_id: CLUSTER_SELF_ID })); @@ -300,7 +258,7 @@ fn p_ballot_calc<'a>( fn p_leader_expired<'a>( proposer_tick: &Tick>, p_to_proposers_i_am_leader: Stream, Unbounded, NoOrder>, - p_is_leader: Optional>, Bounded>, + p_is_leader: Optional<(), Tick>, Bounded>, i_am_leader_check_timeout: u64, // How often to check if heartbeat expired ) -> Optional, Tick>, Bounded> { let p_latest_received_i_am_leader = p_to_proposers_i_am_leader.clone().fold_commutative( @@ -328,7 +286,7 @@ fn p_leader_expired<'a>( fn p_leader_heartbeat<'a>( proposers: &Cluster<'a, Proposer>, proposer_tick: &Tick>, - p_is_leader: Optional>, Bounded>, + p_is_leader: Optional<(), Tick>, Bounded>, p_ballot: Singleton>, Bounded>, i_am_leader_send_timeout: u64, // How often to heartbeat i_am_leader_check_timeout: u64, // How often to check if heartbeat expired @@ -367,36 +325,21 @@ fn p_leader_heartbeat<'a>( (p_to_proposers_i_am_leader, p_trigger_election) } -// Proposer logic to send "I am leader" messages periodically to other proposers, or send p1a to acceptors if other leaders expired. -fn p_p1a<'a>( - p_ballot: Singleton>, Bounded>, - p_trigger_election: Optional, Tick>, Bounded>, - acceptors: &Cluster<'a, Acceptor>, -) -> Stream, Unbounded, NoOrder> { - p_trigger_election - .then(p_ballot) - .map(q!(|ballot| P1a { ballot })) - .all_ticks() - .inspect(q!(|_| println!("Proposer leader expired, sending P1a"))) - .broadcast_bincode_interleaved(acceptors) -} - #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn acceptor_p1<'a, L: Serialize + DeserializeOwned + Clone>( acceptor_tick: &Tick>, - p_to_acceptors_p1a: Stream, Unbounded, NoOrder>, + p_to_acceptors_p1a: Stream, Unbounded, NoOrder>, a_log: Singleton>, Bounded>, proposers: &Cluster<'a, Proposer>, ) -> ( Singleton>, Bounded>, - Stream, Cluster<'a, Proposer>, Unbounded, NoOrder>, + Stream<(Ballot, Result), Cluster<'a, Proposer>, Unbounded, NoOrder>, ) { let p_to_acceptors_p1a = p_to_acceptors_p1a.tick_batch(acceptor_tick); let a_max_ballot = p_to_acceptors_p1a .clone() .inspect(q!(|p1a| println!("Acceptor received P1a: {:?}", p1a))) .persist() - .map(q!(|p1a| p1a.ballot)) .max() .unwrap_or(acceptor_tick.singleton(q!(Ballot { num: 0, @@ -408,13 +351,16 @@ fn acceptor_p1<'a, L: Serialize + DeserializeOwned + Clone>( p_to_acceptors_p1a .cross_singleton(a_max_ballot) .cross_singleton(a_log) - .map(q!(|((p1a, max_ballot), log)| ( - p1a.ballot.proposer_id, - P1b { - ballot: p1a.ballot, - max_ballot, - accepted: log - } + .map(q!(|((ballot, max_ballot), log)| ( + ballot.proposer_id, + ( + ballot, + if ballot == max_ballot { + Ok(log) + } else { + Err(max_ballot) + } + ) ))) .all_ticks() .send_bincode_interleaved(proposers), @@ -425,39 +371,56 @@ fn acceptor_p1<'a, L: Serialize + DeserializeOwned + Clone>( #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_p1b<'a, P: Clone + Serialize + DeserializeOwned>( proposer_tick: &Tick>, - a_to_proposers_p1b: Stream, Cluster<'a, Proposer>, Unbounded, NoOrder>, + a_to_proposers_p1b: Stream< + (Ballot, Result), + Cluster<'a, Proposer>, + Unbounded, + NoOrder, + >, p_ballot: Singleton>, Bounded>, p_has_largest_ballot: Optional<(), Tick>, Bounded>, f: usize, ) -> ( - Optional>, Bounded>, - Stream, Tick>, Bounded, NoOrder>, + Optional<(), Tick>, Bounded>, + Stream>, Bounded, NoOrder>, + Stream>, Bounded, NoOrder>, ) { - let p_relevant_p1bs = a_to_proposers_p1b - .tick_prefix(proposer_tick) - // NOTE: because `p_ballot` grows monotonically across ticks, we could garbage gollect - // but we don't do that here since leader election is a rare event - .cross_singleton(p_ballot.clone()) - .filter(q!(|(p1b, ballot)| p1b.ballot == *ballot)) - .map(q!(|t| t.0)); - let p_received_quorum_of_p1bs = - p_relevant_p1bs - .clone() - .count() - .filter_map(q!(move |num_received| if num_received > f { - Some(true) + let (quorums, fails) = collect_quorum( + proposer_tick, + a_to_proposers_p1b.tick_batch(proposer_tick), + f + 1, + 2 * f + 1, + ); + + let p_received_quorum_of_p1bs = quorums + .persist() + .max_by_key(q!(|t| t.0)) + .zip(p_ballot.clone()) + .filter_map(q!( + move |((quorum_ballot, quorum_accepted), my_ballot)| if quorum_ballot == my_ballot { + Some(quorum_accepted) } else { None - })); - let p_is_leader = p_received_quorum_of_p1bs.continue_if(p_has_largest_ballot.clone()); + } + )); - (p_is_leader, p_relevant_p1bs) + let p_is_leader = p_received_quorum_of_p1bs + .clone() + .map(q!(|_| ())) + .continue_if(p_has_largest_ballot.clone()); + + ( + p_is_leader, + // we used an unordered accumulator, so flattened has no order + p_received_quorum_of_p1bs.flatten().assume_ordering(), + fails.map(q!(|(_, ballot)| ballot)), + ) } #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn recommit_after_leader_election<'a, P: PaxosPayload>( - p_relevant_p1bs: Stream< - P1b>>, + accepted_logs: Stream< + HashMap>, Tick>, Bounded, NoOrder, @@ -467,10 +430,9 @@ fn recommit_after_leader_election<'a, P: PaxosPayload>( ) -> ( Stream, Tick>, Bounded, NoOrder>, Optional>, Bounded>, - Stream, Tick>, Bounded>, ) { - let p_p1b_highest_entries_and_count = p_relevant_p1bs - .flat_map(q!(|p1b| p1b.accepted.into_iter())) // Convert HashMap log back to stream + let p_p1b_highest_entries_and_count = accepted_logs + .flatten() // Convert HashMap log back to stream .fold_keyed_commutative::<(usize, Option>), _, _>(q!(|| (0, None)), q!(|curr_entry, new_entry| { if let Some(curr_entry_payload) = &mut curr_entry.1 { let same_values = new_entry.value == curr_entry_payload.value; @@ -524,7 +486,8 @@ fn recommit_after_leader_election<'a, P: PaxosPayload>( slot, value: None })); - (p_log_to_try_commit, p_max_slot, p_log_holes) + + (p_log_to_try_commit.union(p_log_holes), p_max_slot) } #[expect( @@ -546,42 +509,73 @@ fn sequence_payload<'a, P: PaxosPayload, R>( >, p_ballot: Singleton>, Bounded>, - p_is_leader: Optional>, Bounded>, - p_max_slot: Optional>, Bounded>, + p_is_leader: Optional<(), Tick>, Bounded>, - p_log_to_recommit: Stream, Tick>, Bounded, NoOrder>, + p_relevant_p1bs: Stream< + HashMap>, + Tick>, + Bounded, + NoOrder, + >, f: usize, a_max_ballot: Singleton>, Bounded>, ) -> ( Stream<(usize, Option

), Cluster<'a, Proposer>, Unbounded, NoOrder>, - Singleton<(Option, HashMap>), Tick>, Bounded>, - Stream, Cluster<'a, Proposer>, Unbounded, NoOrder>, + Singleton>, Tick>, Bounded>, + Stream, Unbounded, NoOrder>, ) { - let p_to_acceptors_p2a = p_p2a( + let (p_log_to_recommit, p_max_slot) = + recommit_after_leader_election(p_relevant_p1bs, p_ballot.clone(), f); + + let indexed_payloads = index_payloads( proposer_tick, p_max_slot, c_to_proposers, - p_ballot.clone(), - p_log_to_recommit, p_is_leader.clone(), - acceptors, ); - // Acceptors. - // p_to_acceptors_p2a.clone().for_each(q!(|p2a: P2a| println!("Acceptor received P2a: {:?}", p2a))); + let payloads_to_send = indexed_payloads + .cross_singleton(p_ballot.clone()) + .map(q!(|((slot, payload), ballot)| ( + (slot, ballot), + Some(payload) + ))) + .union(p_log_to_recommit.map(q!(|p2a| ((p2a.slot, p2a.ballot), p2a.value)))) + .continue_if(p_is_leader); + let (a_log, a_to_proposers_p2b) = acceptor_p2( acceptor_tick, a_max_ballot.clone(), - p_to_acceptors_p2a, + payloads_to_send + .clone() + .all_ticks() + .map(q!(|((slot, ballot), value)| P2a { + ballot, + slot, + value + })) + .broadcast_bincode_interleaved(acceptors), r_to_acceptors_checkpoint, proposers, f, ); - let p_to_replicas = p_p2b(proposer_tick, a_to_proposers_p2b.clone(), f); + // TOOD: only persist if we are the leader + let (quorums, fails) = collect_quorum( + proposer_tick, + a_to_proposers_p2b.clone().tick_batch(proposer_tick), + f + 1, + 2 * f + 1, + ); - (p_to_replicas, a_log, a_to_proposers_p2b) + let p_to_replicas = join_responses(proposer_tick, quorums, payloads_to_send).all_ticks(); + + ( + p_to_replicas.map(q!(|((slot, _ballot), (value, _))| (slot, value))), + a_log.map(q!(|(_ckpnt, log)| log)), + fails.map(q!(|(_, ballot)| ballot)).all_ticks(), + ) } #[derive(Clone)] @@ -591,56 +585,37 @@ enum CheckpointOrP2a

{ } // Proposer logic to send p2as, outputting the next slot and the p2as to send to acceptors. -fn p_p2a<'a, P: PaxosPayload>( +fn index_payloads<'a, P: PaxosPayload>( proposer_tick: &Tick>, p_max_slot: Optional>, Bounded>, c_to_proposers: Stream, Unbounded>, - p_ballot: Singleton>, Bounded>, - p_log_to_recommit: Stream, Tick>, Bounded, NoOrder>, - p_is_leader: Optional>, Bounded>, - acceptors: &Cluster<'a, Acceptor>, -) -> Stream, Cluster<'a, Acceptor>, Unbounded, NoOrder> { - let (p_next_slot_complete_cycle, p_next_slot) = proposer_tick.cycle::>(); - let p_next_slot_after_reconciling_p1bs = p_max_slot - .map(q!(|max_slot| max_slot + 1)) - .unwrap_or(proposer_tick.singleton(q!(0))) - // .inspect(q!(|max_slot| println!("{} p_max_slot: {:?}", context.current_tick(), max_slot))) - .continue_unless(p_next_slot.clone()); - - // Send p2as + p_is_leader: Optional<(), Tick>, Bounded>, +) -> Stream<(usize, P), Tick>, Bounded> { + let (p_next_slot_complete_cycle, p_next_slot) = + proposer_tick.cycle_with_initial::>(proposer_tick.singleton(q!(0))); + let p_next_slot_after_reconciling_p1bs = p_max_slot.map(q!(|max_slot| max_slot + 1)); + + let base_slot = p_next_slot_after_reconciling_p1bs.unwrap_or(p_next_slot); + let p_indexed_payloads = c_to_proposers .tick_batch(proposer_tick) + .continue_if(p_is_leader) .enumerate() - .cross_singleton(p_next_slot.clone()) - // .inspect(q!(|next| println!("{} p_indexed_payloads next slot: {}", context.current_tick(), next)))) - .cross_singleton(p_ballot.clone()) - // .inspect(q!(|ballot_num| println!("{} p_indexed_payloads ballot_num: {}", context.current_tick(), ballot_num)))) - .map(q!(|(((index, payload), next_slot), ballot)| P2a { - ballot, - slot: next_slot + index, - value: Some(payload) - })); - // .inspect(q!(|p2a: &P2a| println!("{} p_indexed_payloads P2a: {:?}", context.current_tick(), p2a))); - let p_to_acceptors_p2a = p_log_to_recommit - .chain(p_indexed_payloads.clone()) - .continue_if(p_is_leader.clone()) - .all_ticks() - .broadcast_bincode_interleaved(acceptors); - - let p_num_payloads = p_indexed_payloads.count(); - let p_next_slot_after_sending_payloads = p_num_payloads - .clone() - .zip(p_next_slot.clone()) - .map(q!(|(num_payloads, next_slot)| next_slot + num_payloads)); - - let p_new_next_slot = p_next_slot_after_reconciling_p1bs - // .inspect(q!(|slot| println!("{} p_new_next_slot_after_reconciling_p1bs: {:?}", context.current_tick(), slot))) - .union(p_next_slot_after_sending_payloads) - // .inspect(q!(|slot| println!("{} p_next_slot_after_sending_payloads: {:?}", context.current_tick(), slot)))) - .continue_if(p_is_leader.clone()); + .cross_singleton(base_slot.clone()) + .map(q!(|((index, payload), base_slot)| ( + base_slot + index, + payload + ))); + + let p_num_payloads = p_indexed_payloads.clone().count(); + let p_next_slot_after_sending_payloads = + p_num_payloads + .clone() + .zip(base_slot) + .map(q!(|(num_payloads, base_slot)| base_slot + num_payloads)); - p_next_slot_complete_cycle.complete_next_tick(p_new_next_slot); - p_to_acceptors_p2a + p_next_slot_complete_cycle.complete_next_tick(p_next_slot_after_sending_payloads); + p_indexed_payloads } #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] @@ -658,7 +633,7 @@ fn acceptor_p2<'a, P: PaxosPayload, R>( f: usize, ) -> ( Singleton<(Option, HashMap>), Tick>, Bounded>, - Stream, Cluster<'a, Proposer>, Unbounded, NoOrder>, + Stream<((usize, Ballot), Result<(), Ballot>), Cluster<'a, Proposer>, Unbounded, NoOrder>, ) { let p_to_acceptors_p2a_batch = p_to_acceptors_p2a.tick_batch(acceptor_tick); @@ -695,21 +670,25 @@ fn acceptor_p2<'a, P: PaxosPayload, R>( } else { None } - )) - .assume_ordering::(); + )); let a_log = a_p2as_to_place_in_log - .chain(a_new_checkpoint.into_stream()) + .union(a_new_checkpoint.into_stream()) .persist() - .fold( + .fold_commutative( q!(|| (None, HashMap::new())), q!(|(prev_checkpoint, log), checkpoint_or_p2a| { match checkpoint_or_p2a { CheckpointOrP2a::Checkpoint(new_checkpoint) => { - // This is a checkpoint message. Delete all entries up to the checkpoint - for slot in (prev_checkpoint.unwrap_or(0))..new_checkpoint { - log.remove(&slot); + if prev_checkpoint + .map(|prev| new_checkpoint > prev) + .unwrap_or(true) + { + for slot in (prev_checkpoint.unwrap_or(0))..new_checkpoint { + log.remove(&slot); + } + + *prev_checkpoint = Some(new_checkpoint); } - *prev_checkpoint = Some(new_checkpoint); } CheckpointOrP2a::P2a(p2a) => { // This is a regular p2a message. Insert it into the log if it is not checkpointed and has a higher ballot than what was there before @@ -736,82 +715,127 @@ fn acceptor_p2<'a, P: PaxosPayload, R>( .cross_singleton(a_max_ballot) .map(q!(|(p2a, max_ballot)| ( p2a.ballot.proposer_id, - P2b { - ballot: p2a.ballot, - max_ballot, - slot: p2a.slot, - value: p2a.value - } + ( + (p2a.slot, p2a.ballot), + if p2a.ballot == max_ballot { + Ok(()) + } else { + Err(max_ballot) + } + ) ))) .all_ticks() .send_bincode_interleaved(proposers); (a_log, a_to_proposers_p2b) } -fn p_p2b<'a, P: PaxosPayload>( - proposer_tick: &Tick>, - a_to_proposers_p2b: Stream, Cluster<'a, Proposer>, Unbounded, NoOrder>, - f: usize, -) -> Stream<(usize, Option

), Cluster<'a, Proposer>, Unbounded, NoOrder> { - let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = proposer_tick.cycle(); - let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = - proposer_tick.cycle::>(); - let p_p2b = a_to_proposers_p2b - .tick_batch(proposer_tick) - .chain(p_persisted_p2bs); - let p_count_matching_p2bs = p_p2b - .clone() - .filter_map(q!(|p2b| if p2b.ballot == p2b.max_ballot { - // Only consider p2bs where max ballot = ballot, which means that no one preempted us - Some(((p2b.slot, p2b.ballot), p2b.value)) - } else { - None - })) - .fold_keyed_commutative( - q!(|| (0, None)), - q!(|accum, value| { - accum.0 += 1; - accum.1 = Some(value); - // this is commutative because p2bs with the same slot and ballot - // will have identical values since they originated from the payload - }), - ) - .map(q!(|(k, (count, v))| (k, (count, v.unwrap())))); - let p_p2b_quorum_reached = - p_count_matching_p2bs +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] +pub fn collect_quorum<'a, L: Location<'a> + NoTick, K: Clone + Eq + Hash, V: Clone, E: Clone>( + tick: &Tick, + responses: Stream<(K, Result), Tick, Bounded, NoOrder>, + min: usize, + max: usize, +) -> ( + Stream<(K, Vec), Tick, Bounded, NoOrder>, + Stream<(K, E), Tick, Bounded, NoOrder>, +) { + let (not_all_complete_cycle, not_all) = tick.cycle::>(); + + let current_responses = responses.clone().union(not_all); + + let count_per_key = current_responses.clone().fold_keyed_commutative( + q!(move || ((0, 0), vec![])), + q!(move |accum, value| { + match value { + Ok(v) => { + accum.0 .0 += 1; + accum.1.push(v); + } + Err(_) => { + accum.0 .1 += 1; + } + } + }), + ); + + let reached_min_count = + count_per_key .clone() - .filter_map(q!(move |((slot, _ballot), (count, value))| if count > f { - Some((slot, value)) + .filter_map(q!( + move |(key, ((success, _error), values))| if success >= min { + Some((key, values)) + } else { + None + } + )); + + let just_reached_quorum = if max == min { + not_all_complete_cycle.complete_next_tick( + current_responses.anti_join(reached_min_count.clone().map(q!(|(key, _values)| key))), + ); + + reached_min_count + } else { + let (min_but_not_max_complete_cycle, min_but_not_max) = tick.cycle(); + let just_reached_quorum = reached_min_count.clone().anti_join(min_but_not_max); // Only tell the replicas about committed values once + + let received_from_all = + count_per_key.filter_map(q!(move |(key, ((success, error), _values))| if (success + + error) + >= max + { + Some(key) } else { None })); - let p_to_replicas = p_p2b_quorum_reached - .clone() - .anti_join(p_broadcasted_p2b_slots) // Only tell the replicas about committed values once - .all_ticks(); - let p_p2b_all_commit_slots = - p_count_matching_p2bs.clone().filter_map(q!( - move |((slot, _ballot), (count, _p2b))| if count == 2 * f + 1 { - Some(slot) - } else { - None - } - )); - // p_p2b_all_commit_slots.inspect(q!(|slot: i32| println!("Proposer slot all received: {:?}", slot))); - let p_broadcasted_p2b_slots_new = p_p2b_quorum_reached - .clone() - .map(q!(|(slot, _value)| slot)) - .filter_not_in(p_p2b_all_commit_slots.clone()); - // p_broadcasted_p2b_slots_new.inspect(q!(|slot: i32| println!("Proposer slot broadcasted: {:?}", slot))); - p_broadcasted_p2b_slots_complete_cycle.complete_next_tick(p_broadcasted_p2b_slots_new); - let p_persisted_p2bs_new = p_p2b - .clone() - .map(q!(|p2b| (p2b.slot, p2b))) - .anti_join(p_p2b_all_commit_slots.clone()) - .map(q!(|(_slot, p2b)| p2b)); - // TOOD: only persist if we are the leader - // p_persisted_p2bs_new.inspect(q!(|(sender, p2b): (u32, P2b)| println!("Proposer persisting p2b: {:?}", p2b))); - p_persisted_p2bs_complete_cycle.complete_next_tick(p_persisted_p2bs_new); - p_to_replicas + min_but_not_max_complete_cycle.complete_next_tick( + reached_min_count + .map(q!(|(key, _values)| key)) + .filter_not_in(received_from_all.clone()), + ); + + not_all_complete_cycle + .complete_next_tick(current_responses.anti_join(received_from_all.clone())); + + just_reached_quorum + }; + + ( + just_reached_quorum, + responses.filter_map(q!(move |(key, res)| match res { + Ok(_) => None, + Err(e) => Some((key, e)), + })), + ) +} + +type JoinResponses = Stream<(K, (M, V)), Tick, Bounded, NoOrder>; + +/// Given an incoming stream of request-response responses, joins with metadata generated +/// at request time that is stored in-memory. +/// +/// Only one response element should be produced with a given key, same for the metadata stream. +fn join_responses<'a, K: Clone + Eq + Hash, M: Clone, V: Clone, L: Location<'a> + NoTick>( + tick: &Tick, + responses: Stream<(K, V), Tick, Bounded, NoOrder>, + metadata: Stream<(K, M), Tick, Bounded, NoOrder>, +) -> JoinResponses { + let (remaining_to_join_complete_cycle, remaining_to_join) = + tick.cycle::>(); + + let remaining_and_new: Stream<(K, M), Tick, Bounded, _> = remaining_to_join.union(metadata); + + // TODO(shadaj): we should have a "split-join" operator + // that returns both join and anti-join without cloning + let joined_this_tick = + remaining_and_new + .clone() + .join(responses.clone()) + .map(q!(|(key, (meta, resp))| (key, (meta, resp)))); + + remaining_to_join_complete_cycle + .complete_next_tick(remaining_and_new.anti_join(responses.map(q!(|(key, _)| key)))); + + joined_this_tick } diff --git a/hydroflow_plus_test/src/cluster/paxos_bench.rs b/hydroflow_plus_test/src/cluster/paxos_bench.rs index fb763ff60ce3..bbfc17d28da3 100644 --- a/hydroflow_plus_test/src/cluster/paxos_bench.rs +++ b/hydroflow_plus_test/src/cluster/paxos_bench.rs @@ -3,9 +3,9 @@ use std::rc::Rc; use std::time::{Duration, SystemTime}; use hydroflow_plus::*; -use stream::NoOrder; +use stream::{NoOrder, TotalOrder}; -use super::paxos::{Acceptor, Ballot, Proposer}; +use super::paxos::{collect_quorum, Acceptor, Ballot, Proposer}; use super::paxos_kv::{paxos_kv, KvPayload, Replica}; pub struct Client {} @@ -60,10 +60,13 @@ pub fn paxos_bench<'a>( .tick_batch(&client_tick) .cross_singleton(cur_leader_id) .all_ticks() - .map(q!(move |(key, leader_id)| (leader_id, KvPayload { + .map(q!(move |((key, value), leader_id)| (leader_id, KvPayload { key, - // we use our ID as the value and use that so the replica only notifies us - value: CLUSTER_SELF_ID + // we use our ID as part of the value and use that so the replica only notifies us + value: ( + CLUSTER_SELF_ID, + value + ) }))) .send_bincode_interleaved(&proposers) // clients "own" certain keys, so interleaving elements from clients will not affect @@ -79,36 +82,22 @@ pub fn paxos_bench<'a>( new_leader_elected_complete .complete(new_leader_elected.broadcast_bincode_interleaved(&clients)); - // we only mark a transaction as committed when `f + 1` replicas have committed it - let (c_pending_quorum_payloads_complete_cycle, c_pending_quorum_payloads) = - client_tick.cycle::>(); let c_received_payloads = processed_payloads - .map(q!(|payload| (payload.value, payload))) - .send_bincode_interleaved(&clients) - .tick_batch(&client_tick) - .map(q!(|replica_payload| (replica_payload.key, ()))) - .chain(c_pending_quorum_payloads); - let c_received_quorum_payloads = c_received_payloads - .clone() - .fold_keyed_commutative( - q!(|| 0), - q!(|curr_count, _sender| { - *curr_count += 1; // Assumes the same replica will only send commit once - }), - ) - .filter_map(q!(move |(key, count)| { - if count == f + 1 { - Some(key) - } else { - None - } - })); - let c_new_pending_quorum_payloads = - c_received_payloads.anti_join(c_received_quorum_payloads.clone()); - c_pending_quorum_payloads_complete_cycle - .complete_next_tick(c_new_pending_quorum_payloads); + .map(q!(|payload| ( + payload.value.0, + ((payload.key, payload.value.1), Ok(())) + ))) + .send_bincode_interleaved(&clients); - c_received_quorum_payloads.all_ticks() + // we only mark a transaction as committed when all replicas have applied it + let (c_quorum_payloads, _) = collect_quorum::<_, _, _, ()>( + &client_tick, + c_received_payloads.tick_batch(&client_tick), + f + 1, + f + 1, + ); + + c_quorum_payloads.map(q!(|(key, _)| key)).all_ticks() }, num_clients_per_node, median_latency_window_size, @@ -121,8 +110,8 @@ fn bench_client<'a>( clients: &Cluster<'a, Client>, trigger_restart: Stream<(), Cluster<'a, Client>, Unbounded>, transaction_cycle: impl FnOnce( - Stream, Unbounded>, - ) -> Stream, Unbounded, NoOrder>, + Stream<(u32, u32), Cluster<'a, Client>, Unbounded>, + ) -> Stream<(u32, u32), Cluster<'a, Client>, Unbounded, NoOrder>, num_clients_per_node: usize, median_latency_window_size: usize, ) { @@ -132,24 +121,28 @@ fn bench_client<'a>( // Whenever the leader changes, make all clients send a message let restart_this_tick = trigger_restart.tick_batch(&client_tick).last(); - let c_new_payloads_when_restart = restart_this_tick - .clone() - .flat_map(q!(move |_| (0..num_clients_per_node).map(move |i| i as u32))); + let c_new_payloads_when_restart = restart_this_tick.clone().flat_map(q!(move |_| (0 + ..num_clients_per_node) + .map(move |i| ( + (CLUSTER_SELF_ID.raw_id * (num_clients_per_node as u32)) + i as u32, + 0 + )))); let (c_to_proposers_complete_cycle, c_to_proposers) = - clients.forward_ref::>(); - let c_received_quorum_payloads = transaction_cycle( - c_to_proposers.assume_ordering(), /* we don't send a new write for the same key until the previous one is committed, - * so writes to the same key are ordered */ - ) - .tick_batch(&client_tick); + clients.forward_ref::>(); + let c_received_quorum_payloads = transaction_cycle(c_to_proposers).tick_batch(&client_tick); // Whenever all replicas confirm that a payload was committed, send another payload - let c_new_payloads_when_committed = c_received_quorum_payloads.clone(); + let c_new_payloads_when_committed = c_received_quorum_payloads + .clone() + .map(q!(|payload| (payload.0, payload.1 + 1))); c_to_proposers_complete_cycle.complete( + // we don't send a new write for the same key until the previous one is committed, + // so writes to the same key are ordered c_new_payloads_when_restart - .chain(c_new_payloads_when_committed) - .all_ticks(), + .union(c_new_payloads_when_committed) + .all_ticks() + .assume_ordering(), ); // Track statistics @@ -162,11 +155,11 @@ fn bench_client<'a>( )); let c_updated_timers = c_received_quorum_payloads .clone() - .map(q!(|key| (key as usize, SystemTime::now()))); + .map(q!(|(key, _prev_count)| (key as usize, SystemTime::now()))); let c_new_timers = c_timers .clone() // Update c_timers in tick+1 so we can record differences during this tick (to track latency) - .chain(c_new_timers_when_leader_elected) - .chain(c_updated_timers.clone()) + .union(c_new_timers_when_leader_elected) + .union(c_updated_timers.clone()) .reduce_keyed_commutative(q!(|curr_time, new_time| { if new_time > *curr_time { *curr_time = new_time; @@ -186,7 +179,7 @@ fn bench_client<'a>( .map(q!(|(_virtual_id, (prev_time, curr_time))| Some( curr_time.duration_since(prev_time).unwrap().as_micros() ))) - .chain(c_latency_reset.into_stream()) + .union(c_latency_reset.into_stream()) .all_ticks() .flatten() .fold_commutative( diff --git a/hydroflow_plus_test/src/cluster/paxos_kv.rs b/hydroflow_plus_test/src/cluster/paxos_kv.rs index f2f02b7b4224..83d1ff0bfd88 100644 --- a/hydroflow_plus_test/src/cluster/paxos_kv.rs +++ b/hydroflow_plus_test/src/cluster/paxos_kv.rs @@ -104,7 +104,7 @@ pub fn replica<'a, K: KvKey, V: KvValue>( // p_to_replicas.inspect(q!(|payload: ReplicaPayload| println!("Replica received payload: {:?}", payload))); let r_sorted_payloads = p_to_replicas .tick_batch(&replica_tick) - .chain(r_buffered_payloads) // Combine with all payloads that we've received and not processed yet + .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) = diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap index 23c5ab44165d..9e56ee11ef7f 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap @@ -46,27 +46,21 @@ expression: built.ir() input: Persist( Chain( Chain( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | p1a . max_ballot }), - input: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), + CycleSource { + ident: Ident { + sym: cycle_1, }, + location_kind: Cluster( + 0, + ), }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | p2b . max_ballot }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), + CycleSource { + ident: Ident { + sym: cycle_0, }, + location_kind: Cluster( + 0, + ), }, ), CycleSource { @@ -178,7 +172,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { inner: : CycleSource { ident: Ident { @@ -215,198 +209,257 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_1, + sym: cycle_6, }, - location_kind: Cluster( - 0, + location_kind: Tick( + 2, + Cluster( + 0, + ), ), - input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > >) , hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 1, - ), - from_key: None, - to_location: Cluster( - 0, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , log) | (p1a . ballot . proposer_id , P1b { ballot : p1a . ballot , max_ballot , accepted : log }) }), - input: CrossSingleton( - CrossSingleton( - Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - from_key: None, - to_location: Cluster( - 1, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1a) | { (id . raw_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 () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | println ! ("Proposer leader expired, sending P1a") }), + input: DeferTick( + Difference( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (key , _values) | key }), + input: Tee { + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , ((usize , usize) , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >)) , core :: option :: Option < (hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let min__free = 2usize ; move | (key , ((success , _error) , values)) | if success >= min__free { Some ((key , values)) } else { None } }), + input: Tee { + inner: : FoldKeyed { + init: stageleft :: runtime_support :: fn0_type_hint :: < ((usize , usize) , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; move | | ((0 , 0) , vec ! []) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < ((usize , usize) , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >) , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; move | accum , value | { match value { Ok (v) => { accum . 0 . 0 += 1 ; accum . 1 . push (v) ; } Err (_) => { accum . 0 . 1 += 1 ; } } } }), + input: Tee { + inner: : Chain( + Tee { + inner: : Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; | ballot | P1a { ballot } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , ()) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: , - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < tokio :: time :: Instant > , ()) , core :: option :: Option < tokio :: time :: Instant > > ({ use hydroflow_plus :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout__free = 1u64 ; move | latest_received_i_am_leader | { 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__free) } else { true } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < tokio :: time :: Instant > , ()) , core :: option :: Option < tokio :: time :: Instant > > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < tokio :: time :: Instant > > ({ use crate :: __staged :: cluster :: paxos :: * ; | | None }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | latest , _ | { * latest = Some (Instant :: now ()) ; } }), - input: Persist( - Tee { - inner: , - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: , + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot >)) , (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot >) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 1, + ), + from_key: None, + to_location: Cluster( + 0, + ), + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot >)) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , 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 () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot >) > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot) , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((ballot , max_ballot) , log) | (ballot . proposer_id , (ballot , if ballot == max_ballot { Ok (log) } else { Err (max_ballot) })) }), + input: CrossSingleton( + CrossSingleton( + Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 0, + ), + from_key: None, + to_location: Cluster( + 1, + ), + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: Ballot) | { (id . raw_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 () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | println ! ("Proposer leader expired, sending P1a") }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , ()) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: , + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < tokio :: time :: Instant > , ()) , core :: option :: Option < tokio :: time :: Instant > > ({ use hydroflow_plus :: __staged :: optional :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout__free = 1u64 ; move | latest_received_i_am_leader | { 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__free) } else { true } } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < tokio :: time :: Instant > , ()) , core :: option :: Option < tokio :: time :: Instant > > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < tokio :: time :: Instant > > ({ use crate :: __staged :: cluster :: paxos :: * ; | | None }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | latest , _ | { * latest = Some (Instant :: now ()) ; } }), + input: Persist( + Tee { + inner: , + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), + input: Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: , + }, + }, + }, + }, + ), + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | _u | () }), + input: Source { + source: Stream( + { use hydroflow_plus :: __staged :: location :: * ; let delay__free = { use crate :: __staged :: cluster :: paxos :: * ; let CLUSTER_SELF_ID__free = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; let i_am_leader_check_timeout_delay_multiplier__free = 1usize ; Duration :: from_secs ((CLUSTER_SELF_ID__free . raw_id * i_am_leader_check_timeout_delay_multiplier__free as u32) . into ()) } ; let interval__free = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout__free = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout__free) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay__free , interval__free)) }, + ), + location_kind: Cluster( + 0, + ), + }, + }, + ), + }, }, - }, + ), }, }, - ), + }, }, }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | _u | () }), - input: Source { - source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay__free = { use crate :: __staged :: cluster :: paxos :: * ; let CLUSTER_SELF_ID__free = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; let i_am_leader_check_timeout_delay_multiplier__free = 1usize ; Duration :: from_secs ((CLUSTER_SELF_ID__free . raw_id * i_am_leader_check_timeout_delay_multiplier__free as u32) . into ()) } ; let interval__free = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout__free = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout__free) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay__free , interval__free)) }, - ), - location_kind: Cluster( - 0, + }, + Tee { + inner: : Chain( + Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), + input: Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), + input: Tee { + inner: , + }, + }, ), }, - }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e__free] }, + ), + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + ), + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Tick( + 3, + Cluster( + 1, + ), ), }, - }, - ), + ), + }, }, }, }, }, - }, - }, - }, - Tee { - inner: : Chain( - Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | p1a . ballot }), - input: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), - input: Tee { - inner: , - }, - }, + CycleSource { + ident: Ident { + sym: cycle_5, }, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e__free] }, - ), - location_kind: Cluster( - 1, + location_kind: Tick( + 2, + Cluster( + 0, + ), ), }, ), - ), - }, - ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ckpnt , log) | log . clone () }), - input: CycleSource { - ident: Ident { - sym: cycle_0, }, - location_kind: Tick( - 3, - Cluster( - 1, - ), - ), }, }, - ), + }, }, }, - }, + Tee { + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , ((usize , usize) , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >)) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: Ballot > > ({ use crate :: __staged :: cluster :: paxos :: * ; let max__free = 3usize ; move | (key , ((success , error) , _values)) | if (success + error) >= max__free { Some (key) } else { None } }), + input: Tee { + inner: , + }, + }, + }, + ), + ), + }, + CycleSink { + ident: Ident { + sym: cycle_5, }, + location_kind: Tick( + 2, + Cluster( + 0, + ), + ), + input: DeferTick( + AntiJoin( + Tee { + inner: , + }, + Tee { + inner: , + }, + ), + ), }, CycleSink { ident: Ident { @@ -419,41 +472,47 @@ expression: built.ir() ), ), input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: optional :: * ; | (d , _signal) | d }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | (d , _signal) | d }), input: CrossSingleton( - FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | num_received | if num_received > f__free { Some (true) } else { None } }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p1b , ballot) | p1b . ballot == * ballot }), - input: CrossSingleton( - Persist( - Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), - input: Tee { - inner: , + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), + input: Tee { + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >) , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | ((quorum_ballot , quorum_accepted) , my_ballot) | if quorum_ballot == my_ballot { Some (quorum_accepted) } else { None } }), + input: CrossSingleton( + Reduce { + f: { let key_fn = stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }) ; move | curr , new | { if key_fn (& new) > key_fn (& * curr) { * curr = new ; } } }, + input: Persist( + AntiJoin( + Tee { + inner: , + }, + CycleSource { + ident: Ident { + sym: cycle_6, }, + location_kind: Tick( + 2, + Cluster( + 0, + ), + ), }, ), - Tee { - inner: , - }, ), }, - }, + Tee { + inner: , + }, + ), }, }, }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | _u | () }), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (received_max_ballot , cur_ballot) | * received_max_ballot <= * cur_ballot }), @@ -475,7 +534,24 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_5, + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , ballot) | ballot }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , core :: result :: Result < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot) > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), + input: Tee { + inner: , + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_7, }, location_kind: Tick( 2, @@ -485,186 +561,166 @@ expression: built.ir() ), input: DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: optional :: * ; | (d , _signal) | d }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , base_slot) | base_slot + num_payloads }), input: CrossSingleton( - Chain( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Chain( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), - input: Tee { - inner: : Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), - input: Tee { - inner: : FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , None) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { if let Some (curr_entry_payload) = & mut curr_entry . 1 { let same_values = new_entry . value == curr_entry_payload . value ; let higher_ballot = new_entry . ballot > curr_entry_payload . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry_payload . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry_payload . value = new_entry . value ; } } } else { * curr_entry = (1 , Some (new_entry)) ; } } }), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > , std :: collections :: hash_map :: IntoIter < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | p1b . accepted . into_iter () }), - input: Tee { - inner: , - }, + Tee { + inner: : Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) , usize) , (usize , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((index , payload) , base_slot) | (base_slot + index , payload) }), + input: CrossSingleton( + Enumerate { + is_static: false, + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , ()) , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 2, + ), + from_key: None, + to_location: Cluster( + 0, + ), + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer >) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((key , value) , leader_id) | (leader_id , KvPayload { key , value : (CLUSTER_SELF_ID__free , value) }) }), + input: CrossSingleton( + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot : Ballot | ballot . proposer_id }), + input: Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), + input: Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + ), + }, + }, + }, + ), }, }, }, - }, - }, - }, - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e__free] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: : CycleSource { - ident: Ident { - sym: cycle_5, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), + input: Tee { + inner: , + }, }, - location_kind: Tick( - 2, - Cluster( - 0, - ), - ), - }, + ), }, }, - }, - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , next_slot) | next_slot + num_payloads }), - input: CrossSingleton( - Tee { - inner: : Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) , usize) , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (((index , payload) , next_slot) , ballot) | P2a { ballot , slot : next_slot + index , value : Some (payload) } }), - input: CrossSingleton( - CrossSingleton( - Enumerate { - is_static: false, + Tee { + inner: : Chain( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), + input: Tee { + inner: : Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 2, - ), - from_key: None, - to_location: Cluster( - 0, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer >) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (key , leader_id) | (leader_id , KvPayload { key , value : CLUSTER_SELF_ID__free }) }), - input: CrossSingleton( - CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot : Ballot | ballot . proposer_id }), - input: Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - ), - }, + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + input: Tee { + inner: : FoldKeyed { + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , None) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { if let Some (curr_entry_payload) = & mut curr_entry . 1 { let same_values = new_entry . value == curr_entry_payload . value ; let higher_ballot = new_entry . ballot > curr_entry_payload . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry_payload . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry_payload . value = new_entry . value ; } } } else { * curr_entry = (1 , Some (new_entry)) ; } } }), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | d | d }), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > , std :: vec :: Vec < std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > > ({ use hydroflow_plus :: __staged :: optional :: * ; | v | v }), + input: Tee { + inner: , }, }, - ), + }, }, }, }, }, - Tee { - inner: , + }, + }, + Chain( + CycleSource { + ident: Ident { + sym: cycle_7, }, - ), - Tee { - inner: , + location_kind: Tick( + 2, + Cluster( + 0, + ), + ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e__free] }, + ), + location_kind: Cluster( + 0, + ), + }, + ), ), - }, + ), }, - }, - }, - Tee { - inner: , + ), }, - ), - }, - ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | _u | () }), - input: Tee { - inner: , + }, }, }, + Tee { + inner: , + }, ), }, ), }, CycleSink { ident: Ident { - sym: cycle_6, + sym: cycle_9, }, location_kind: Tick( 2, @@ -675,104 +731,114 @@ expression: built.ir() input: DeferTick( Difference( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _value) | slot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , std :: vec :: Vec < () >) , (usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (key , _values) | key }), input: Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >)) , core :: option :: Option < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | ((slot , _ballot) , (count , value)) | if count > f__free { Some ((slot , value)) } else { None } }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , ((usize , usize) , std :: vec :: Vec < () >)) , core :: option :: Option < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , std :: vec :: Vec < () >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let min__free = 2usize ; move | (key , ((success , _error) , values)) | if success >= min__free { Some ((key , values)) } else { None } }), input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , core :: option :: Option < core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >)) , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (k , (count , v)) | (k , (count , v . unwrap ())) }), - input: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , core :: option :: Option < core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , None) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , core :: option :: Option < core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , value | { accum . 0 += 1 ; accum . 1 = Some (value) ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , core :: option :: Option < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | if p2b . ballot == p2b . max_ballot { Some (((p2b . slot , p2b . ballot) , p2b . value)) } else { None } }), - input: Tee { - inner: : Chain( - Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 1, - ), - from_key: None, - to_location: Cluster( - 0, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > (& data) . unwrap () . into ()) }", - ], - }, - ), + inner: : FoldKeyed { + init: stageleft :: runtime_support :: fn0_type_hint :: < ((usize , usize) , std :: vec :: Vec < () >) > ({ use crate :: __staged :: cluster :: paxos :: * ; move | | ((0 , 0) , vec ! []) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < ((usize , usize) , std :: vec :: Vec < () >) , core :: result :: Result < () , hydroflow_plus_test :: cluster :: paxos :: Ballot > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; move | accum , value | { match value { Ok (v) => { accum . 0 . 0 += 1 ; accum . 1 . push (v) ; } Err (_) => { accum . 0 . 1 += 1 ; } } } }), + input: Tee { + inner: : Chain( + Tee { + inner: : Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydroflow_plus_test :: cluster :: paxos :: Ballot >)) , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydroflow_plus_test :: cluster :: paxos :: Ballot >) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 1, + ), + from_key: None, + to_location: Cluster( + 0, + ), + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydroflow_plus_test :: cluster :: paxos :: Ballot >)) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , 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 () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > (& b) . unwrap ()) }", - ], - }, - ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydroflow_plus_test :: cluster :: paxos :: Ballot >) > (& b) . unwrap ()) }", + ], + }, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , P2b { ballot : p2a . ballot , max_ballot , slot : p2a . slot , value : p2a . value }) }), - input: CrossSingleton( - Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - from_key: None, - to_location: Cluster( - 1, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > (& b) . unwrap ()) }", - ], - }, - ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydroflow_plus_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , ((p2a . slot , p2a . ballot) , if p2a . ballot == max_ballot { Ok (()) } else { Err (max_ballot) })) }), + input: CrossSingleton( + Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 0, + ), + from_key: None, + to_location: Cluster( + 1, + ), + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > (& data) . unwrap () . into ()) }", + ], + }, ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Chain( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Chain( + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , ballot) , value) | P2a { ballot , slot , value } }), + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) , ()) , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Chain( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) , hydroflow_plus_test :: cluster :: paxos :: Ballot) , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , payload) , ballot) | ((slot , ballot) , Some (payload)) }), + input: CrossSingleton( + Tee { + inner: , + }, + Tee { + inner: , + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2a | ((p2a . slot , p2a . ballot) , p2a . value) }), + input: Chain( FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >)) , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | ((slot , (count , entry)) , ballot) | { let entry = entry . unwrap () ; if count <= f__free { Some (P2a { ballot , slot , value : entry . value , }) } else { None } } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >)) , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | ((slot , (count , entry)) , ballot) | { let entry = entry . unwrap () ; if count <= f__free { Some (P2a { ballot , slot , value : entry . value , }) } else { None } } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { inner: , @@ -780,19 +846,19 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , ballot) | P2a { ballot , slot , value : None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , ballot) | P2a { ballot , slot , value : None } }), input: CrossSingleton( Difference( FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , std :: ops :: Range < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | 0 .. max_slot }), input: Tee { - inner: , + inner: , }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: , + inner: , }, }, ), @@ -802,74 +868,43 @@ expression: built.ir() ), }, ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), - input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: , - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: optional :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: DeferTick( - Tee { - inner: , - }, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - Tee { - inner: , + }, + ), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), + input: Tee { + inner: , + }, }, ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), - input: Tee { - inner: , - }, - }, - ), + }, }, }, }, }, }, - Tee { - inner: , - }, - ), - }, + }, + Tee { + inner: , + }, + ), }, }, }, - CycleSource { - ident: Ident { - sym: cycle_7, - }, - location_kind: Tick( - 2, - Cluster( - 0, - ), - ), - }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_8, + }, + location_kind: Tick( + 2, + Cluster( + 0, + ), ), }, - }, + ), }, }, }, @@ -877,10 +912,10 @@ expression: built.ir() }, }, Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >)) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | ((slot , _ballot) , (count , _p2b)) | if count == 2 * f__free + 1 { Some (slot) } else { None } }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , ((usize , usize) , std :: vec :: Vec < () >)) , core :: option :: Option < (usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let max__free = 3usize ; move | (key , ((success , error) , _values)) | if (success + error) >= max__free { Some (key) } else { None } }), input: Tee { - inner: , + inner: , }, }, }, @@ -889,7 +924,7 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_7, + sym: cycle_8, }, location_kind: Tick( 2, @@ -898,128 +933,179 @@ expression: built.ir() ), ), input: DeferTick( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , p2b) | p2b }), - input: AntiJoin( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | (p2b . slot , p2b) }), - input: Tee { - inner: , - }, - }, - Tee { - inner: , - }, - ), - }, + AntiJoin( + Tee { + inner: , + }, + Tee { + inner: , + }, + ), ), }, CycleSink { ident: Ident { - sym: cycle_0, + sym: cycle_10, }, location_kind: Tick( - 3, + 2, Cluster( - 1, + 0, ), ), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (None , HashMap :: new ()) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > >) , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { for slot in (prev_checkpoint . unwrap_or (0)) .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = Some (new_checkpoint) ; } CheckpointOrP2a :: P2a (p2a) => { if prev_checkpoint . map (| prev | p2a . slot > prev) . unwrap_or (true) && log . get (& p2a . slot) . map (| prev_p2a : & LogValue < _ > | p2a . ballot > prev_p2a . ballot) . unwrap_or (true) { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } } }), - input: Persist( - Chain( - FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), - input: CrossSingleton( + input: DeferTick( + AntiJoin( + Tee { + inner: : Chain( + CycleSource { + ident: Ident { + sym: cycle_10, + }, + location_kind: Tick( + 2, + Cluster( + 0, + ), + ), + }, + Tee { + inner: , + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , std :: vec :: Vec < () >) , (usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (key , _) | key }), + input: Tee { + inner: : AntiJoin( Tee { inner: , }, - Tee { - inner: , + CycleSource { + ident: Ident { + sym: cycle_9, + }, + location_kind: Tick( + 2, + Cluster( + 0, + ), + ), }, ), }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), - input: Delta( - Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + }, + ), + ), + }, + CycleSink { + ident: Ident { + sym: cycle_0, + }, + location_kind: Tick( + 3, + Cluster( + 1, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >) , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ckpnt , log) | log }), + input: Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (None , HashMap :: new ()) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { if prev_checkpoint . map (| prev | new_checkpoint > prev) . unwrap_or (true) { for slot in (prev_checkpoint . unwrap_or (0)) .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = Some (new_checkpoint) ; } } CheckpointOrP2a :: P2a (p2a) => { if prev_checkpoint . map (| prev | p2a . slot > prev) . unwrap_or (true) && log . get (& p2a . slot) . map (| prev_p2a : & LogValue < _ > | p2a . ballot > prev_p2a . ballot) . unwrap_or (true) { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } } }), + input: Persist( + Chain( + FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), + input: CrossSingleton( + Tee { + inner: , + }, + Tee { + inner: , + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), + input: Delta( + Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) , ()) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: : ReduceKeyed { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), - input: Persist( - Network { - from_location: Cluster( - 3, - ), - from_key: None, - to_location: Cluster( - 1, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , usize) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < usize > (& data) . unwrap () . into ()) }", - ], - }, + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) , ()) , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: : ReduceKeyed { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), + input: Persist( + Network { + from_location: Cluster( + 3, ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < usize > (& b) . unwrap ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 1, ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 3, + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , usize) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < usize > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < usize > (& b) . unwrap ()) }", + ], + }, ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, }, }, - }, - ), + ), + }, }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | num_received | if num_received == f__free + 1 { Some (true) } else { None } }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: , + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | num_received | if num_received == f__free + 1 { Some (true) } else { None } }), + input: Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , usize) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: , + }, }, }, }, - }, - ), + ), + }, }, }, - }, - ), - }, + ), + }, + ), ), - ), + }, }, }, CycleSink { @@ -1029,8 +1115,14 @@ expression: built.ir() location_kind: Cluster( 0, ), - input: Tee { - inner: , + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , ballot) | ballot }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydroflow_plus_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot) > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), + input: Tee { + inner: , + }, + }, }, }, CycleSink { @@ -1038,22 +1130,22 @@ expression: built.ir() sym: cycle_1, }, location_kind: Tick( - 6, + 7, Cluster( 3, ), ), input: DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , usize) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), input: CrossSingleton( Tee { - inner: : Sort( + inner: : Sort( Chain( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -1068,7 +1160,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > (& data) . unwrap () . into ()) }", ], }, ), @@ -1079,31 +1171,29 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > >) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (slot , kv) | SequencedKv { seq : slot , kv } }), - input: AntiJoin( - Tee { - inner: , - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Tick( - 2, - Cluster( - 0, - ), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (slot , kv) | SequencedKv { seq : slot , kv } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , std :: vec :: Vec < () >)) , (usize , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , _ballot) , (value , _)) | (slot , value) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , std :: vec :: Vec < () >)) , ((usize , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > , std :: vec :: Vec < () >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (key , (meta , resp)) | (key , (meta , resp)) }), + input: Join( + Tee { + inner: , + }, + Tee { + inner: , + }, ), }, - ), + }, }, }, }, @@ -1113,7 +1203,7 @@ expression: built.ir() sym: cycle_1, }, location_kind: Tick( - 6, + 7, Cluster( 3, ), @@ -1123,14 +1213,14 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < usize > , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | v | v }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | | None }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , core :: option :: Option < usize >) , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | filled_slot , (sorted_payload , highest_seq) | { let expected_next_slot = std :: cmp :: max (filled_slot . map (| v | v + 1) . unwrap_or (0) , highest_seq . map (| v | v + 1) . unwrap_or (0) ,) ; if sorted_payload . seq == expected_next_slot { * filled_slot = Some (sorted_payload . seq) ; } } }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | filled_slot , (sorted_payload , highest_seq) | { let expected_next_slot = std :: cmp :: max (filled_slot . map (| v | v + 1) . unwrap_or (0) , highest_seq . map (| v | v + 1) . unwrap_or (0) ,) ; if sorted_payload . seq == expected_next_slot { * filled_slot = Some (sorted_payload . seq) ; } } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Chain( Map { @@ -1140,7 +1230,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Tick( - 6, + 7, Cluster( 3, ), @@ -1172,30 +1262,30 @@ expression: built.ir() sym: cycle_2, }, location_kind: Tick( - 6, + 7, Cluster( 3, ), ), input: DeferTick( Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , core :: option :: Option < usize >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (_kv_store , highest_seq) | highest_seq }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , core :: option :: Option < usize >) > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | | (HashMap :: new () , None) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , core :: option :: Option < usize >) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (kv_store , last_seq) , payload | { if let Some (kv) = payload . kv { kv_store . insert (kv . key , kv . value) ; } debug_assert ! (payload . seq == (last_seq . map (| s | s + 1) . unwrap_or (0)) , "Hole in log between seq {:?} and {}" , * last_seq , payload . seq) ; * last_seq = Some (payload . seq) ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | | (HashMap :: new () , None) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (kv_store , last_seq) , payload | { if let Some (kv) = payload . kv { kv_store . insert (kv . key , kv . value) ; } debug_assert ! (payload . seq == (last_seq . map (| s | s + 1) . unwrap_or (0)) , "Hole in log between seq {:?} and {}" , * last_seq , payload . seq) ; * last_seq = Some (payload . seq) ; } }), input: Persist( Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , usize) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1212,14 +1302,14 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 6, + 7, Cluster( 3, ), ), input: DeferTick( Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , usize) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; let checkpoint_frequency__free = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if max_checkpointed_seq . map (| m | new_highest_seq - m >= checkpoint_frequency__free) . unwrap_or (true) { Some (new_highest_seq) } else { None } }), input: CrossSingleton( Chain( @@ -1233,7 +1323,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 6, + 7, Cluster( 3, ), @@ -1254,7 +1344,7 @@ expression: built.ir() ), ), Tee { - inner: , + inner: , }, ), }, @@ -1269,7 +1359,7 @@ expression: built.ir() 3, ), input: Tee { - inner: , + inner: , }, }, CycleSink { @@ -1320,9 +1410,29 @@ expression: built.ir() inner: , }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: , + f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: , + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: optional :: * ; | _u | () }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: optional :: * ; | c | * c == 0 }), + input: Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: DeferTick( + Tee { + inner: , + }, + ), + }, + }, + }, + ), }, }, ), @@ -1344,11 +1454,10 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Chain( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , (u32 , ()) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | replica_payload | (replica_payload . key , ()) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + inner: : Chain( + Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > , ((u32 , u32) , core :: result :: Result < () , () >)) , ((u32 , u32) , core :: result :: Result < () , () >) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 3, @@ -1363,7 +1472,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , ((u32 , u32) , core :: result :: Result < () , () >)) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < ((u32 , u32) , core :: result :: Result < () , () >) > (& data) . unwrap () . into ()) }", ], }, ), @@ -1374,17 +1483,17 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < ((u32 , u32) , core :: result :: Result < () , () >) > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value , payload) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , ((u32 , u32) , core :: result :: Result < () , () >)) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . 0 , ((payload . key , payload . value . 1) , Ok (()))) }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | payload | payload . kv }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | payload | payload . kv }), input: Tee { - inner: , + inner: , }, }, }, @@ -1404,14 +1513,19 @@ expression: built.ir() }, ), }, - Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f__free = 1usize ; move | (key , count) | { if count == f__free + 1 { Some (key) } else { None } } }), - input: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , std :: vec :: Vec < () >) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (key , _values) | key }), + input: Tee { + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , ((usize , usize) , std :: vec :: Vec < () >)) , core :: option :: Option < ((u32 , u32) , std :: vec :: Vec < () >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let min__free = 2usize ; move | (key , ((success , _error) , values)) | if success >= min__free { Some ((key , values)) } else { None } }), input: Tee { - inner: , + inner: : FoldKeyed { + init: stageleft :: runtime_support :: fn0_type_hint :: < ((usize , usize) , std :: vec :: Vec < () >) > ({ use crate :: __staged :: cluster :: paxos :: * ; move | | ((0 , 0) , vec ! []) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < ((usize , usize) , std :: vec :: Vec < () >) , core :: result :: Result < () , () > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; move | accum , value | { match value { Ok (v) => { accum . 0 . 0 += 1 ; accum . 1 . push (v) ; } Err (_) => { accum . 0 . 1 += 1 ; } } } }), + input: Tee { + inner: , + }, + }, }, }, }, @@ -1428,24 +1542,30 @@ expression: built.ir() ), input: Chain( FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let num_clients_per_node__free = 1usize ; move | _ | (0 .. num_clients_per_node__free) . map (move | i | i as u32) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; let num_clients_per_node__free = 1usize ; move | _ | (0 .. num_clients_per_node__free) . map (move | i | ((CLUSTER_SELF_ID__free . raw_id * (num_clients_per_node__free as u32)) + i as u32 , 0)) }), input: Tee { - inner: : Reduce { + inner: : Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < () , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | * curr = new }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus :: location :: cluster :: cluster_id :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | () }), input: Delta( Tee { - inner: , + inner: , }, ), }, }, }, }, - Tee { - inner: : Tee { - inner: , + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . 0 , payload . 1 + 1) }), + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , std :: vec :: Vec < () >) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (key , _) | key }), + input: Tee { + inner: , + }, + }, }, }, ), @@ -1466,7 +1586,7 @@ expression: built.ir() input: Chain( Chain( Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_3, }, @@ -1483,16 +1603,16 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: time :: SystemTime > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | SystemTime :: now () }), input: Tee { - inner: , + inner: , }, }, }, ), Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , (usize , std :: time :: SystemTime) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | key | (key as usize , SystemTime :: now ()) }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (usize , std :: time :: SystemTime) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (key , _prev_count) | (key as usize , SystemTime :: now ()) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1519,10 +1639,10 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (std :: time :: SystemTime , std :: time :: SystemTime)) , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (_virtual_id , (prev_time , curr_time)) | Some (curr_time . duration_since (prev_time) . unwrap () . as_micros ()) }), input: Join( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1530,7 +1650,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), input: Tee { - inner: : Source { + inner: : Source { source: Stream( { use hydroflow_plus :: __staged :: location :: * ; let interval__free = { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval (interval__free)) }, ), @@ -1558,9 +1678,9 @@ expression: built.ir() input: CrossSingleton( Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (u32 , u32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, Map { @@ -1571,7 +1691,7 @@ expression: built.ir() init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1583,7 +1703,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), input: Tee { - inner: , + inner: , }, }, ), @@ -1594,7 +1714,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ), diff --git a/hydroflow_plus_test_local/src/local/graph_reachability.rs b/hydroflow_plus_test_local/src/local/graph_reachability.rs index 8dfea4fb1f8f..0d50a88f51ae 100644 --- a/hydroflow_plus_test_local/src/local/graph_reachability.rs +++ b/hydroflow_plus_test_local/src/local/graph_reachability.rs @@ -19,13 +19,13 @@ pub fn graph_reachability<'a>( let reachability_tick = process.tick(); let (set_reached_cycle, reached_cycle) = reachability_tick.cycle::>(); - let reached = roots.tick_batch(&reachability_tick).chain(reached_cycle); + let reached = roots.tick_batch(&reachability_tick).union(reached_cycle); let reachable = reached .clone() .map(q!(|r| (r, ()))) .join(edges.tick_batch(&reachability_tick).persist()) .map(q!(|(_from, (_, to))| to)); - set_reached_cycle.complete_next_tick(reached.clone().chain(reachable)); + set_reached_cycle.complete_next_tick(reached.clone().union(reachable)); reached.all_ticks().unique().for_each(q!(|v| { reached_out.send(v).unwrap(); diff --git a/stageleft/src/type_name.rs b/stageleft/src/type_name.rs index 1077e54dde30..486fb866f0a5 100644 --- a/stageleft/src/type_name.rs +++ b/stageleft/src/type_name.rs @@ -93,6 +93,24 @@ impl VisitMut for RewriteAlloc { .chain(i.segments.iter().skip(4).cloned()), ), }; + } else if i.segments.iter().take(3).collect::>() + == vec![ + &syn::PathSegment::from(syn::Ident::new("std", Span::call_site())), + &syn::PathSegment::from(syn::Ident::new("vec", Span::call_site())), + &syn::PathSegment::from(syn::Ident::new("into_iter", Span::call_site())), + ] + { + *i = syn::Path { + leading_colon: i.leading_colon, + segments: syn::punctuated::Punctuated::from_iter( + vec![ + syn::PathSegment::from(syn::Ident::new("std", Span::call_site())), + syn::PathSegment::from(syn::Ident::new("vec", Span::call_site())), + ] + .into_iter() + .chain(i.segments.iter().skip(3).cloned()), + ), + }; } else if i.segments.iter().take(3).collect::>() == vec![ &syn::PathSegment::from(syn::Ident::new("tokio", Span::call_site())), @@ -134,10 +152,16 @@ impl VisitMut for RewriteAlloc { syn::parse2(get_final_crate_name(final_name)).unwrap(); i.segments.insert(1, parse_quote!(__staged)); + } else { + syn::visit_mut::visit_path_mut(self, i); + return; } + } else { + syn::visit_mut::visit_path_mut(self, i); + return; } - syn::visit_mut::visit_path_mut(self, i); + self.visit_path_mut(i); } }