From 93f8d19071ba313cdf62fe3d957cf3d1c2b9d89e Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 8 Oct 2024 14:30:25 -0700 Subject: [PATCH 1/5] refactor(paxos): start splitting out leader election into a separate module --- hydroflow_plus/src/singleton.rs | 4 + hydroflow_plus_test/src/cluster/paxos.rs | 240 ++-- ...cluster__paxos_bench__tests__paxos_ir.snap | 1035 +++++++++-------- 3 files changed, 633 insertions(+), 646 deletions(-) diff --git a/hydroflow_plus/src/singleton.rs b/hydroflow_plus/src/singleton.rs index 41140422696..54374e6c3f1 100644 --- a/hydroflow_plus/src/singleton.rs +++ b/hydroflow_plus/src/singleton.rs @@ -580,6 +580,10 @@ impl<'a, T, N: Location<'a>> Optional { .map(q!(|(d, _signal)| d)) } + pub fn then(self, value: Singleton) -> Optional { + value.continue_if(self) + } + pub fn continue_unless( self, other: Optional, diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 488be19c01e..0667dae4e9d 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -21,12 +21,12 @@ pub trait PaxosPayload: pub struct Ballot { // Note: Important that num comes before id, since Ord is defined lexicographically pub num: u32, - pub id: ClusterId, + pub proposer_id: ClusterId, } impl LeaderElected for Ballot { fn leader_id(&self) -> ClusterId { - self.id + self.proposer_id } } @@ -57,8 +57,8 @@ struct P2a

{ #[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug)] struct P2b

{ + victory: bool, ballot: Ballot, - max_ballot: Ballot, slot: i32, value: P, } @@ -119,7 +119,7 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( &proposers, - a_to_proposers_p1b.inspect(q!(|(_, p1b)| println!("Proposer received P1b: {:?}", p1b))), + a_to_proposers_p1b.inspect(q!(|p1b| println!("Proposer received P1b: {:?}", p1b))), p_ballot_num.clone(), p_has_largest_ballot, f, @@ -152,7 +152,7 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( let p_to_clients_new_leader_elected = p_is_leader.clone() .continue_unless(p_next_slot) .cross_singleton(p_ballot_num) - .map(q!(move |(_is_leader, ballot_num)| Ballot { num: ballot_num, id: p_id})) // Only tell the clients once when leader election concludes + .map(q!(move |(_is_leader, ballot_num)| Ballot { num: ballot_num, proposer_id: p_id})) // Only tell the clients once when leader election concludes .all_ticks(); // End tell clients that leader election has completed let p_to_replicas = p_p2b(&proposers, a_to_proposers_p2b, f); @@ -197,8 +197,8 @@ fn acceptor<'a, P: PaxosPayload, R>( acceptors: &Cluster<'a, Acceptor>, f: usize, ) -> ( - Stream<(ClusterId, P1b

), Unbounded, NoTick, Cluster<'a, Proposer>>, - Stream<(ClusterId, P2b

), Unbounded, NoTick, Cluster<'a, Proposer>>, + Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, ) { // Get the latest checkpoint sequence per replica let a_checkpoint_largest_seqs = @@ -229,7 +229,7 @@ fn acceptor<'a, P: PaxosPayload, R>( // Create tuple with checkpoint number and dummy p2a ballot: Ballot { num: 0, - id: ClusterId::from_raw(0) + proposer_id: ClusterId::from_raw(0) }, slot: -1, value: Default::default() @@ -243,7 +243,7 @@ fn acceptor<'a, P: PaxosPayload, R>( .max() .unwrap_or(acceptors.singleton(q!(Ballot { num: 0, - id: ClusterId::from_raw(0) + proposer_id: ClusterId::from_raw(0) }))); let a_p2as_to_place_in_log = p_to_acceptors_p2a .clone() @@ -303,7 +303,7 @@ fn acceptor<'a, P: PaxosPayload, R>( .cross_singleton(a_max_ballot.clone().latest_tick()) .cross_singleton(a_log) .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( - p1a.ballot.id, + p1a.ballot.proposer_id, P1b { ballot: p1a.ballot, max_ballot, @@ -311,33 +311,27 @@ fn acceptor<'a, P: PaxosPayload, R>( } ))) .all_ticks() - .send_bincode(proposers); + .send_bincode_interleaved(proposers); let a_to_proposers_p2b_new = p_to_acceptors_p2a .tick_batch() .cross_singleton(a_max_ballot.latest_tick()) .map(q!(|(p2a, max_ballot)| ( - p2a.ballot.id, + p2a.ballot.proposer_id, P2b { + victory: p2a.ballot == max_ballot, ballot: p2a.ballot, - max_ballot, slot: p2a.slot, value: p2a.value } ))) .all_ticks() - .send_bincode(proposers); + .send_bincode_interleaved(proposers); (a_to_proposers_p1b_new, a_to_proposers_p2b_new) } -#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_p2b<'a, P: PaxosPayload>( proposers: &Cluster<'a, Proposer>, - a_to_proposers_p2b: Stream< - (ClusterId, P2b

), - Unbounded, - NoTick, - Cluster<'a, Proposer>, - >, + a_to_proposers_p2b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, f: usize, ) -> Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>> { let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = proposers.tick_cycle(); @@ -348,62 +342,55 @@ fn p_p2b<'a, P: PaxosPayload>( .union(p_persisted_p2bs); let p_count_matching_p2bs = p_p2b .clone() - .filter_map(q!(|(sender, p2b)| if p2b.ballot == p2b.max_ballot { + .filter_map(q!(|p2b| if p2b.victory { // Only consider p2bs where max ballot = ballot, which means that no one preempted us - Some((p2b.slot, (sender, p2b))) + Some(((p2b.slot, p2b.ballot), p2b.value)) } else { None })) .fold_keyed( - q!(|| ( - 0, - P2b { - ballot: Ballot { - num: 0, - id: ClusterId::from_raw(0) - }, - max_ballot: Ballot { - num: 0, - id: ClusterId::from_raw(0) - }, - slot: 0, - value: Default::default() - } - )), - q!(|accum, (_sender, p2b)| { + q!(|| (0, Default::default())), + q!(|accum, value| { + // TODO(shadaj): why is sender unused? should we de-dup? accum.0 += 1; - accum.1 = p2b; + accum.1 = value; }), ); - let p_p2b_quorum_reached = p_count_matching_p2bs - .clone() - .filter(q!(move |(_slot, (count, _p2b))| *count > f)); + let p_p2b_quorum_reached = + p_count_matching_p2bs + .clone() + .filter_map(q!(move |((slot, _ballot), (count, value))| if count > f { + Some((slot, value)) + } 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 - .map(q!(|(_slot, (_count, p2b))| (p2b.slot, p2b.value))) + .map(q!(|(slot, value)| (slot, value))) .all_ticks(); let p_p2b_all_commit_slots = - p_count_matching_p2bs - .clone() - .filter_map(q!(move |(slot, (count, _p2b))| if count == 2 * f + 1 { + 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, (_count, _p2b))| slot)) + .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!(|(sender, p2b)| (p2b.slot, (sender, p2b)))) + .map(q!(|p2b| (p2b.slot, p2b))) .anti_join(p_p2b_all_commit_slots.clone()) - .map(q!(|(_slot, (sender, p2b))| (sender, p2b))); + .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 @@ -446,7 +433,7 @@ fn p_p2a<'a, P: PaxosPayload>( // .inspect(q!(|next| println!("{} p_indexed_payloads next slot: {}", context.current_tick(), next)))) .cross_singleton(p_ballot_num.clone()) // .inspect(q!(|ballot_num| println!("{} p_indexed_payloads ballot_num: {}", context.current_tick(), ballot_num)))) - .map(q!(move |(((index, payload), next_slot), ballot_num)| P2a { ballot: Ballot { num: ballot_num, id: p_id }, slot: next_slot + index as i32, value: payload })); + .map(q!(move |(((index, payload), next_slot), ballot_num)| P2a { ballot: Ballot { num: ballot_num, proposer_id: p_id }, slot: next_slot + index as i32, value: payload })); // .inspect(q!(|p2a: &P2a| println!("{} p_indexed_payloads P2a: {:?}", context.current_tick(), p2a))); let p_to_acceptors_p2a = p_log_to_try_commit .union(p_log_holes) @@ -489,12 +476,7 @@ fn p_p2a<'a, P: PaxosPayload>( #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_p1b<'a, P: PaxosPayload>( proposers: &Cluster<'a, Proposer>, - a_to_proposers_p1b: Stream< - (ClusterId, P1b

), - Unbounded, - NoTick, - Cluster<'a, Proposer>, - >, + a_to_proposers_p1b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, p_ballot_num: Singleton>, p_has_largest_ballot: Optional<(Ballot, u32), Bounded, Tick, Cluster<'a, Proposer>>, f: usize, @@ -506,30 +488,26 @@ fn p_p1b<'a, P: PaxosPayload>( ) { let p_id = proposers.self_id(); let p_relevant_p1bs = a_to_proposers_p1b - .clone() .tick_prefix() + // NOTE: because `p_ballot_num` 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_num.clone()) - .filter(q!(move |((_sender, p1b), ballot_num)| p1b.ballot - == Ballot { - num: *ballot_num, - id: p_id + .filter(q!(move |(p1b, ballot_num)| p1b.ballot.num == *ballot_num && p1b.ballot.proposer_id == p_id)) + .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) + } else { + None })); - let p_received_quorum_of_p1bs = p_relevant_p1bs - .clone() - .map(q!(|((sender, _p1b), _ballot_num)| { sender })) - .unique() - .count() - .filter_map(q!(move |num_received| if num_received > f { - Some(true) - } else { - None - })); let p_is_leader = p_received_quorum_of_p1bs.continue_if(p_has_largest_ballot.clone()); let p_p1b_highest_entries_and_count = p_relevant_p1bs - .clone() - .flat_map(q!(|((_, p1b), _)| p1b.accepted.into_iter())) // Convert HashMap log back to stream - .fold_keyed(q!(|| (0, LogValue { ballot: Ballot { num: 0, id: ClusterId::from_raw(0) }, value: Default::default() })), q!(|curr_entry, new_entry| { + .flat_map(q!(|p1b| p1b.accepted.into_iter())) // Convert HashMap log back to stream + .fold_keyed(q!(|| (0, LogValue { ballot: Ballot { num: 0, proposer_id: ClusterId::from_raw(0) }, value: Default::default() })), q!(|curr_entry, new_entry| { let same_values = new_entry.value == curr_entry.1.value; let higher_ballot = new_entry.ballot > curr_entry.1.ballot; // Increment count if the values are the same @@ -554,7 +532,7 @@ fn p_p1b<'a, P: PaxosPayload>( Some(P2a { ballot: Ballot { num: ballot_num, - id: p_id, + proposer_id: p_id, }, slot, value: entry.value, @@ -578,7 +556,7 @@ fn p_p1b<'a, P: PaxosPayload>( .map(q!(move |(slot, ballot_num)| P2a { ballot: Ballot { num: ballot_num, - id: p_id + proposer_id: p_id }, slot, value: Default::default() @@ -587,36 +565,21 @@ fn p_p1b<'a, P: PaxosPayload>( } // Proposer logic to calculate the largest ballot received so far. -#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_max_ballot<'a, P: PaxosPayload>( proposers: &Cluster<'a, Proposer>, - a_to_proposers_p1b: Stream< - (ClusterId, P1b

), - Unbounded, - NoTick, - Cluster<'a, Proposer>, - >, - a_to_proposers_p2b: Stream< - (ClusterId, P2b

), - Unbounded, - NoTick, - Cluster<'a, Proposer>, - >, + a_to_proposers_p1b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + a_to_proposers_p2b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, p_to_proposers_i_am_leader: Stream>, ) -> Singleton> { - let p_received_p1b_ballots = a_to_proposers_p1b - .clone() - .map(q!(|(_, p1b)| p1b.max_ballot)); - let p_received_p2b_ballots = a_to_proposers_p2b - .clone() - .map(q!(|(_, p2b)| p2b.max_ballot)); + let p_received_p1b_ballots = a_to_proposers_p1b.clone().map(q!(|p1b| p1b.max_ballot)); + let p_received_p2b_ballots = a_to_proposers_p2b.clone().map(q!(|p2b| p2b.ballot)); p_received_p1b_ballots .union(p_received_p2b_ballots) .union(p_to_proposers_i_am_leader) .max() .unwrap_or(proposers.singleton(q!(Ballot { num: 0, - id: ClusterId::from_raw(0) + proposer_id: ClusterId::from_raw(0) }))) } @@ -640,7 +603,7 @@ fn p_ballot_calc<'a>( if received_max_ballot > (Ballot { num: ballot_num, - id: p_id, + proposer_id: p_id, }) { received_max_ballot.num + 1 @@ -657,7 +620,7 @@ fn p_ballot_calc<'a>( move |(received_max_ballot, ballot_num)| *received_max_ballot <= Ballot { num: *ballot_num, - id: p_id + proposer_id: p_id } )); @@ -685,21 +648,55 @@ fn p_p1a<'a>( Stream>, ) { let p_id = proposers.self_id(); - let p_to_proposers_i_am_leader_new = p_ballot_num + let p_to_proposers_i_am_leader_new = p_is_leader .clone() + .then(p_ballot_num.clone()) + .latest() + .sample_every(q!(Duration::from_secs(i_am_leader_send_timeout))) + .map(q!(move |ballot_num| Ballot { + num: ballot_num, + proposer_id: p_id + })) + .broadcast_bincode_interleaved(proposers); + + let p_leader_expired = p_leader_expired( + p_to_proposers_i_am_leader, + p_is_leader, + i_am_leader_check_timeout, + ); + + let p_id = proposers.self_id(); + + // Add random delay depending on node ID so not everyone sends p1a at the same time + let p_to_acceptors_p1a = p_leader_expired + .then(p_ballot_num) .continue_if( proposers - .source_interval(q!(Duration::from_secs(i_am_leader_send_timeout))) + .source_interval_delayed( + q!(Duration::from_secs( + (p_id.raw_id * i_am_leader_check_timeout_delay_multiplier as u32).into() + )), + q!(Duration::from_secs(i_am_leader_check_timeout)), + ) .latest_tick(), ) - .continue_if(p_is_leader.clone()) - .map(q!(move |ballot_num| Ballot { - num: ballot_num, - id: p_id + .map(q!(move |ballot_num| P1a { + ballot: Ballot { + num: ballot_num, + proposer_id: p_id + } })) .all_ticks() - .broadcast_bincode_interleaved(proposers); + .inspect(q!(|_| println!("Proposer leader expired, sending P1a"))) + .broadcast_bincode_interleaved(acceptors); + (p_to_proposers_i_am_leader_new, p_to_acceptors_p1a) +} +fn p_leader_expired<'a>( + p_to_proposers_i_am_leader: Stream>, + p_is_leader: Optional>, + i_am_leader_check_timeout: u64, // How often to check if heartbeat expired +) -> Optional, Bounded, Tick, Cluster<'a, Proposer>> { let p_latest_received_i_am_leader = p_to_proposers_i_am_leader.clone().fold( q!(|| None), q!(|latest, _| { @@ -707,34 +704,17 @@ fn p_p1a<'a>( *latest = Some(Instant::now()); }), ); - // Add random delay depending on node ID so not everyone sends p1a at the same time - let p_leader_expired = proposers.source_interval_delayed(q!(Duration::from_secs((p_id.raw_id * i_am_leader_check_timeout_delay_multiplier as u32).into())), q!(Duration::from_secs(i_am_leader_check_timeout))) - .cross_singleton(p_latest_received_i_am_leader.clone()) + + p_latest_received_i_am_leader + .clone() .latest_tick() - // .inspect(q!(|v| println!("Proposer checking if leader expired"))) - // .continue_if(p_is_leader.clone().count().filter(q!(|c| *c == 0)).inspect(q!(|c| println!("Proposer is_leader count: {}", c)))) .continue_unless(p_is_leader) - .filter(q!(move |(_, latest_received_i_am_leader)| { + .filter(q!(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) + (Instant::now().duration_since(*latest_received_i_am_leader)) + > Duration::from_secs(i_am_leader_check_timeout) } else { true } - })); - p_leader_expired - .clone() - .all_ticks() - .for_each(q!(|_| println!("Proposer leader expired"))); - - let p_to_acceptors_p1a = p_ballot_num - .continue_if(p_leader_expired) - .map(q!(move |ballot_num| P1a { - ballot: Ballot { - num: ballot_num, - id: p_id - } })) - .all_ticks() - .broadcast_bincode_interleaved(acceptors); - (p_to_proposers_i_am_leader_new, p_to_acceptors_p1a) } 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 f20193cd83b..af57d72b873 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 @@ -22,7 +22,7 @@ expression: built.ir() 2, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , id : p_id , }) { received_max_ballot . num + 1 } else { ballot_num } } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , proposer_id : p_id , }) { received_max_ballot . num + 1 } else { ballot_num } } }), input: CrossSingleton( Tee { inner: : Union( @@ -32,7 +32,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | p1b . max_ballot }), input: Tee { inner: : CycleSource { ident: Ident { @@ -45,7 +45,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | p2b . ballot }), input: Tee { inner: : CycleSource { ident: Ident { @@ -74,7 +74,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -108,105 +108,6 @@ expression: built.ir() ), }, }, - ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < (tokio :: time :: Instant , core :: option :: Option < tokio :: time :: Instant >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | println ! ("Proposer leader expired") }), - input: Tee { - inner: : Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (tokio :: time :: Instant , core :: option :: Option < tokio :: time :: Instant >) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 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) } else { true } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((tokio :: time :: Instant , core :: option :: Option < tokio :: time :: Instant >) , ()) , (tokio :: time :: Instant , core :: option :: Option < tokio :: time :: Instant >) > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - CrossSingleton( - Source { - source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, - ), - location_kind: Cluster( - 2, - ), - }, - Tee { - inner: : 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: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (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 = 1usize ; move | num_received | if num_received > f { 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: : Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: : Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), - input: Tee { - inner: , - }, - }, - }, - ), - Tee { - inner: , - }, - ), - }, - }, - }, - ), - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: : Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { - inner: , - }, - Tee { - inner: , - }, - ), - }, - }, - }, - ), - }, - }, - }, - }, - }, - ), - }, - }, - }, - }, CycleSink { ident: Ident { sym: cycle_0, @@ -249,7 +150,7 @@ expression: built.ir() input: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > > (__hydroflow_plus_cluster_ids_2) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | Ballot { num : ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | Ballot { num : ballot_num , proposer_id : p_id } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -260,22 +161,70 @@ expression: built.ir() inner: , }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Source { - source: Interval( - { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_send_timeout = 1u64 ; Duration :: from_secs (i_am_leader_send_timeout) }, - ), - location_kind: Cluster( - 2, - ), + f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (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 = 1usize ; move | num_received | if num_received > f { 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (p1b , ballot_num) | p1b . ballot . num == * ballot_num && p1b . ballot . proposer_id == p_id }), + input: CrossSingleton( + Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), + input: Tee { + inner: , + }, + }, + ), + Tee { + inner: , + }, + ), + }, + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: : Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , proposer_id : p_id } }), + input: CrossSingleton( + Tee { + inner: , + }, + Tee { + 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: Source { + source: Interval( + stageleft :: runtime_support :: type_hint :: < core :: time :: Duration > ({ use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_send_timeout = 1u64 ; Duration :: from_secs (i_am_leader_send_timeout) }), + ), + location_kind: Cluster( + 2, + ), }, }, ), @@ -295,7 +244,7 @@ expression: built.ir() input: DeferTick( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Union( @@ -307,18 +256,18 @@ expression: built.ir() input: CrossSingleton( Union( Tee { - inner: : Reduce { + inner: : Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: : FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), + inner: : FoldKeyed { + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | p1b . accepted . into_iter () }), input: Tee { - inner: , + inner: , }, }, }, @@ -345,7 +294,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_4, }, @@ -364,15 +313,15 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , next_slot) | next_slot + num_payloads as i32 }), input: CrossSingleton( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: : Fold { + 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_bench :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -420,10 +369,10 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: : Filter { + inner: : Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | num_payloads | * num_payloads > 0 }), input: Tee { - inner: , + inner: , }, }, }, @@ -432,7 +381,7 @@ expression: built.ir() }, }, Tee { - inner: , + inner: , }, ), }, @@ -441,7 +390,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -451,7 +400,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 , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -462,7 +411,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ), @@ -474,7 +423,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -484,7 +433,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -505,18 +454,18 @@ expression: built.ir() input: DeferTick( Difference( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , (_count , _p2b)) | slot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _value) | slot }), input: Tee { - inner: : Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , value)) | if count > f { Some ((slot , value)) } else { None } }), input: Tee { - inner: : FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + inner: : FoldKeyed { + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , Default :: default ()) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , value | { accum . 0 += 1 ; accum . 1 = value ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , core :: option :: Option < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | if p2b . victory { Some (((p2b . slot , p2b . ballot) , p2b . value)) } else { None } }), input: Tee { - inner: : Union( + inner: : Union( Tee { inner: , }, @@ -537,10 +486,10 @@ expression: built.ir() }, }, Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (slot , (count , _p2b)) | if count == 2 * f + 1 { Some (slot) } else { None } }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , _p2b)) | if count == 2 * f + 1 { Some (slot) } else { None } }), input: Tee { - inner: , + inner: , }, }, }, @@ -556,16 +505,16 @@ expression: built.ir() ), input: DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (sender , p2b)) | (sender , p2b) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , p2b) | p2b }), input: AntiJoin( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | (p2b . slot , (sender , p2b)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | (p2b . slot , p2b) }), input: Tee { - inner: , + inner: , }, }, Tee { - inner: , + inner: , }, ), }, @@ -589,355 +538,406 @@ expression: built.ir() location_kind: Cluster( 2, ), - input: Network { - from_location: Cluster( - 3, - ), - from_key: None, - to_location: Cluster( - 2, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", - ], - }, + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: 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 :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 2, ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , log)) | (p1a . ballot . id , P1b { ballot : p1a . ballot , max_ballot , accepted : log }) }), - input: CrossSingleton( - CrossSingleton( - Tee { - inner: : 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: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: 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( - 2, - ), - from_key: None, - to_location: Cluster( - 3, - ), - 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 ()) }", - ], - }, + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , log)) | (p1a . ballot . proposer_id , P1b { ballot : p1a . ballot , max_ballot , accepted : log }) }), + input: CrossSingleton( + CrossSingleton( + Tee { + inner: : 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: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: 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( + 2, ), - ), - 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 ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 3, ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: , - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (tokio :: time :: Instant , core :: option :: Option < tokio :: time :: Instant >) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: , - }, + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , proposer_id : p_id } } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ 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: 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 = 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) } 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( + Tee { + inner: : 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: , + }, + }, + }, + }, + ), + }, + }, + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Source { + source: Stream( + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 2, + ), + }, + }, + ), }, - ), + }, }, }, }, }, }, }, - }, - Tee { - inner: : Union( - 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: Tee { - inner: , + Tee { + inner: : Union( + 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: Tee { + inner: , + }, }, - }, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, - ), - location_kind: Cluster( - 3, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), ), - ), - }, - ), - Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: new ()) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , (new_checkpoint , p2a) | { if new_checkpoint != - 1 { for slot in * prev_checkpoint .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = new_checkpoint ; } else { if p2a . slot > * prev_checkpoint { match log . get (& p2a . slot) { None => { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } Some (prev_p2a) => { if p2a . ballot > prev_p2a . ballot { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } ; } } } }), - input: Persist( - Union( - FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some ((- 1 , p2a)) } else { None } }), - input: CrossSingleton( - Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 2, - ), - from_key: None, - to_location: Cluster( - 3, - ), - 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", - ], - }, + }, + ), + Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: new ()) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , (new_checkpoint , p2a) | { if new_checkpoint != - 1 { for slot in * prev_checkpoint .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = new_checkpoint ; } else { if p2a . slot > * prev_checkpoint { match log . get (& p2a . slot) { None => { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } Some (prev_p2a) => { if p2a . ballot > prev_p2a . ballot { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } ; } } } }), + input: Persist( + Union( + FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some ((- 1 , p2a)) } else { None } }), + input: CrossSingleton( + Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 2, ), - ), - 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_bench :: ClientPayload > > (& b) . unwrap ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 3, ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Union( - FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , id : p_id , } , slot , value : entry . value , }) } else { None } }), - input: CrossSingleton( - Tee { - inner: , - }, - Tee { - inner: , - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : Default :: default () } }), - input: CrossSingleton( - Difference( - FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: ops :: Range < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | 0 .. max_slot }), - input: Tee { - inner: , - }, + 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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_bench :: ClientPayload > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Union( + FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id , } , slot , value : entry . value , }) } else { None } }), + input: CrossSingleton( + Tee { + inner: , }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), - input: Tee { - inner: , + Tee { + inner: , + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id } , slot , value : Default :: default () } }), + input: CrossSingleton( + Difference( + FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: ops :: Range < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | 0 .. max_slot }), + input: Tee { + inner: , + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + input: Tee { + inner: , + }, }, + ), + Tee { + inner: , }, ), - Tee { - inner: , + }, + ), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: stream :: * ; | 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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: , + }, }, - ), + }, }, ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: stream :: * ; | 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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id } , slot : next_slot + index as i32 , value : payload } }), + input: CrossSingleton( + CrossSingleton( + Enumerate( + Tee { inner: , }, - }, - }, - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }), - input: CrossSingleton( - CrossSingleton( - Enumerate( + ), Tee { - inner: , + inner: , }, ), Tee { - inner: , + inner: , }, ), - Tee { - inner: , - }, - ), + }, + ), + 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 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), - input: Tee { - inner: , - }, - }, - ), + }, }, }, }, }, - }, - Tee { - inner: , - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | (min_seq , P2a { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : - 1 , value : Default :: default () }) }), - input: Delta( - Union( - Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + Tee { + inner: , + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | (min_seq , P2a { ballot : Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } , slot : - 1 , value : Default :: default () }) }), + input: Delta( + Union( + Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , ()) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: : ReduceKeyed { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), - input: Persist( - Network { - from_location: Cluster( - 1, - ), - from_key: None, - to_location: Cluster( - 3, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , i32) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", - ], - }, + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , ()) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: : ReduceKeyed { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), + input: Persist( + Network { + from_location: Cluster( + 1, ), - ), - 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 :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 3, ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 1, + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , i32) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + ], + }, ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + }, }, }, - }, - ), + ), + }, }, - }, - 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 = 1usize ; move | num_received | if num_received == f + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , () > ({ 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 = 1usize ; move | num_received | if num_received == f + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: , + }, }, }, }, - }, - ), + ), + }, }, }, - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), - }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), ), ), - ), - }, + }, + ), ), - ), - }, - ), + }, + ), + }, }, }, }, @@ -948,46 +948,49 @@ expression: built.ir() location_kind: Cluster( 2, ), - input: Network { - from_location: Cluster( - 3, - ), - from_key: None, - to_location: Cluster( - 2, - ), - 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", - ], - }, + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: 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 :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 2, ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . id , P2b { ballot : p2a . ballot , max_ballot , slot : p2a . slot , value : p2a . value }) }), - input: CrossSingleton( - Tee { - inner: , - }, - Tee { - inner: , - }, + 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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_bench :: ClientPayload > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , P2b { victory : p2a . ballot == max_ballot , ballot : p2a . ballot , slot : p2a . slot , value : p2a . value }) }), + input: CrossSingleton( + Tee { + inner: , + }, + Tee { + inner: , + }, + ), + }, }, }, }, @@ -1005,10 +1008,10 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), input: CrossSingleton( Tee { - inner: : Sort( + inner: : Sort( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -1045,10 +1048,10 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | (p2b . slot , p2b . value) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , value) | (slot , value) }), input: AntiJoin( Tee { - inner: , + inner: , }, CycleSource { ident: Ident { @@ -1077,12 +1080,12 @@ expression: built.ir() ), }, Tee { - inner: : Fold { + inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | - 1 }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | filled_slot , (sorted_payload , highest_seq) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Union( CycleSource { @@ -1119,23 +1122,23 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | (HashMap :: < u32 , String > :: new () , - 1) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | state , payload | { let kv_store = & mut state . 0 ; let last_seq = & mut state . 1 ; kv_store . insert (payload . key , payload . value) ; debug_assert ! (payload . seq == * last_seq + 1 , "Hole in log between seq {} and {}" , * last_seq , payload . seq) ; * last_seq = payload . seq ; } }), input: Persist( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1156,7 +1159,7 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , i32) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let checkpoint_frequency = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if new_highest_seq - max_checkpointed_seq >= checkpoint_frequency as i32 { Some (new_highest_seq) } else { None } }), input: CrossSingleton( Union( @@ -1185,7 +1188,7 @@ expression: built.ir() ), ), Tee { - inner: , + inner: , }, ), }, @@ -1200,7 +1203,7 @@ expression: built.ir() 1, ), input: Tee { - inner: , + inner: , }, }, CycleSink { @@ -1213,7 +1216,7 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Union( + inner: : Union( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { @@ -1249,7 +1252,7 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1265,13 +1268,13 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 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 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { - inner: , + inner: , }, }, }, @@ -1292,7 +1295,7 @@ expression: built.ir() input: Union( Union( Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_2, }, @@ -1306,9 +1309,9 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: time :: SystemTime > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | SystemTime :: now () }), input: Tee { - inner: : Delta( + inner: : Delta( Tee { - inner: : Reduce { + inner: : 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 { @@ -1348,13 +1351,13 @@ expression: built.ir() input: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (__hydroflow_plus_cluster_ids_0) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , proposer_id : p_id } }), input: CrossSingleton( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -1364,7 +1367,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1389,10 +1392,10 @@ expression: built.ir() }, ), Tee { - inner: : Map { + 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 ()) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1405,7 +1408,7 @@ expression: built.ir() input: CrossSingleton( CrossSingleton( Tee { - inner: : Source { + inner: : Source { source: Interval( { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, ), @@ -1423,10 +1426,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: , }, ), }, @@ -1434,7 +1437,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), input: Tee { - inner: , + inner: , }, }, ), @@ -1456,7 +1459,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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, Map { @@ -1467,7 +1470,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 , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1479,7 +1482,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), input: Tee { - inner: , + inner: , }, }, ), @@ -1499,17 +1502,17 @@ expression: built.ir() FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . leader_id () , ClientPayload { key : i as u32 , value : c_id . raw_id . to_string () })) }), input: Tee { - inner: , + inner: , }, }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; move | (key , leader_ballot) | (leader_ballot . leader_id () , ClientPayload { key , value : c_id . raw_id . to_string () }) }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, From ae13c612fc8f93f5c812f440b63531dec5af758c Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 8 Oct 2024 14:30:25 -0700 Subject: [PATCH 2/5] refactor(paxos): complete split into leader election and sequencing phases --- hydroflow_plus/src/cycle.rs | 21 +- hydroflow_plus/src/lib.rs | 2 +- hydroflow_plus/src/location.rs | 49 +- hydroflow_plus/src/singleton.rs | 36 +- hydroflow_plus/src/stream.rs | 15 +- hydroflow_plus_test/src/cluster/paxos.rs | 349 +++-- .../src/cluster/paxos_bench.rs | 4 +- ...cluster__paxos_bench__tests__paxos_ir.snap | 1170 +++++++++-------- .../src/local/graph_reachability.rs | 2 +- stageleft_tool/src/lib.rs | 4 + 10 files changed, 930 insertions(+), 722 deletions(-) diff --git a/hydroflow_plus/src/cycle.rs b/hydroflow_plus/src/cycle.rs index d8a240e5654..4f2f010ecb2 100644 --- a/hydroflow_plus/src/cycle.rs +++ b/hydroflow_plus/src/cycle.rs @@ -2,7 +2,11 @@ use std::marker::PhantomData; use crate::builder::FlowState; use crate::location::{Location, LocationId}; -use crate::{NoTick, Tick}; +use crate::Tick; + +pub trait DeferTick { + fn defer_tick(self) -> Self; +} pub trait CycleComplete<'a, T> { fn complete(self, ident: syn::Ident); @@ -25,25 +29,30 @@ pub trait CycleCollectionWithInitial<'a, T>: CycleComplete<'a, T> { ) -> Self; } -/// Represents a fixpoint cycle in the graph that will be fulfilled +/// Represents a forward reference in the graph that will be fulfilled /// by a stream that is not yet known. /// /// See [`crate::FlowBuilder`] for an explainer on the type parameters. -pub struct HfCycle<'a, T, S: CycleComplete<'a, T>> { +pub struct HfForwardRef<'a, T, S: CycleComplete<'a, T>> { pub(crate) ident: syn::Ident, pub(crate) _phantom: PhantomData<(&'a mut &'a (), T, S)>, } -impl<'a, S: CycleComplete<'a, NoTick>> HfCycle<'a, NoTick, S> { +impl<'a, T, S: CycleComplete<'a, T>> HfForwardRef<'a, T, S> { pub fn complete(self, stream: S) { let ident = self.ident; S::complete(stream, ident) } } -impl<'a, S: CycleComplete<'a, Tick>> HfCycle<'a, Tick, S> { +pub struct HfCycle<'a, S: CycleComplete<'a, Tick> + DeferTick> { + pub(crate) ident: syn::Ident, + pub(crate) _phantom: PhantomData<(&'a mut &'a (), S)>, +} + +impl<'a, S: CycleComplete<'a, Tick> + DeferTick> HfCycle<'a, S> { pub fn complete_next_tick(self, stream: S) { let ident = self.ident; - S::complete(stream, ident) + S::complete(stream.defer_tick(), ident) } } diff --git a/hydroflow_plus/src/lib.rs b/hydroflow_plus/src/lib.rs index 3f3c5fbb109..607d3970c12 100644 --- a/hydroflow_plus/src/lib.rs +++ b/hydroflow_plus/src/lib.rs @@ -31,7 +31,7 @@ pub mod deploy; pub use deploy::{ClusterSpec, Deploy, ProcessSpec}; pub mod cycle; -pub use cycle::HfCycle; +pub use cycle::HfForwardRef; pub mod builder; pub use builder::FlowBuilder; diff --git a/hydroflow_plus/src/location.rs b/hydroflow_plus/src/location.rs index da9a5978dd8..95aae37ce28 100644 --- a/hydroflow_plus/src/location.rs +++ b/hydroflow_plus/src/location.rs @@ -12,9 +12,9 @@ use serde::{Deserialize, Serialize}; use stageleft::{q, quote_type, Quoted}; use super::builder::{ClusterIds, ClusterSelfId, FlowState}; -use crate::cycle::{CycleCollection, CycleCollectionWithInitial}; +use crate::cycle::{CycleCollection, CycleCollectionWithInitial, DeferTick, HfCycle}; use crate::ir::{HfPlusNode, HfPlusSource}; -use crate::{Bounded, HfCycle, NoTick, Optional, Singleton, Stream, Tick, Unbounded}; +use crate::{Bounded, HfForwardRef, NoTick, Optional, Singleton, Stream, Tick, Unbounded}; #[derive(PartialEq, Eq, Clone, Copy, Debug)] pub enum LocationId { @@ -170,9 +170,9 @@ pub trait Location<'a> { .latest() } - fn tick_cycle>( + fn forward_ref>( &self, - ) -> (HfCycle<'a, Tick, S>, S) { + ) -> (HfForwardRef<'a, NoTick, S>, S) { let next_id = { let on_id = match self.id() { LocationId::Process(id) => id, @@ -191,7 +191,36 @@ pub trait Location<'a> { let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); ( - HfCycle { + HfForwardRef { + ident: ident.clone(), + _phantom: PhantomData, + }, + S::create_source(ident, self.flow_state().clone(), self.id()), + ) + } + + fn tick_forward_ref>( + &self, + ) -> (HfForwardRef<'a, Tick, S>, S) { + let next_id = { + let on_id = match self.id() { + LocationId::Process(id) => id, + LocationId::Cluster(id) => id, + LocationId::ExternalProcess(_) => panic!(), + }; + + let mut flow_state = self.flow_state().borrow_mut(); + let next_id_entry = flow_state.cycle_counts.entry(on_id).or_default(); + + let id = *next_id_entry; + *next_id_entry += 1; + id + }; + + let ident = syn::Ident::new(&format!("cycle_{}", next_id), Span::call_site()); + + ( + HfForwardRef { ident: ident.clone(), _phantom: PhantomData, }, @@ -199,9 +228,9 @@ pub trait Location<'a> { ) } - fn cycle>( + fn tick_cycle + DeferTick>( &self, - ) -> (HfCycle<'a, NoTick, S>, S) { + ) -> (HfCycle<'a, S>, S) { let next_id = { let on_id = match self.id() { LocationId::Process(id) => id, @@ -228,10 +257,12 @@ pub trait Location<'a> { ) } - fn tick_cycle_with_initial>( + fn tick_cycle_with_initial< + S: CycleCollectionWithInitial<'a, Tick, Location = Self> + DeferTick, + >( &self, initial: S, - ) -> (HfCycle<'a, Tick, S>, S) { + ) -> (HfCycle<'a, S>, S) { let next_id = { let on_id = match self.id() { LocationId::Process(id) => id, diff --git a/hydroflow_plus/src/singleton.rs b/hydroflow_plus/src/singleton.rs index 54374e6c3f1..9a3beaa036e 100644 --- a/hydroflow_plus/src/singleton.rs +++ b/hydroflow_plus/src/singleton.rs @@ -6,7 +6,7 @@ use std::rc::Rc; use stageleft::{q, IntoQuotedMut, Quoted}; use crate::builder::FlowState; -use crate::cycle::{CycleCollection, CycleCollectionWithInitial, CycleComplete}; +use crate::cycle::{CycleCollection, CycleCollectionWithInitial, CycleComplete, DeferTick}; use crate::ir::{HfPlusLeaf, HfPlusNode, HfPlusSource, TeeNode}; use crate::location::{Location, LocationId}; use crate::stream::{Bounded, NoTick, Tick, Unbounded}; @@ -128,6 +128,12 @@ impl<'a, T, C, N: Location<'a>> From> } } +impl<'a, T, N: Location<'a>> DeferTick for Singleton { + fn defer_tick(self) -> Self { + Singleton::defer_tick(self) + } +} + impl<'a, T, N: Location<'a>> CycleComplete<'a, Tick> for Singleton { fn complete(self, ident: syn::Ident) { self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { @@ -138,6 +144,21 @@ impl<'a, T, N: Location<'a>> CycleComplete<'a, Tick> for Singleton> CycleCollection<'a, Tick> for Singleton { + type Location = N; + + fn create_source(ident: syn::Ident, flow_state: FlowState, l: LocationId) -> Self { + Singleton::new( + l, + flow_state, + HfPlusNode::CycleSource { + ident, + location_kind: l, + }, + ) + } +} + impl<'a, T, N: Location<'a>> CycleCollectionWithInitial<'a, Tick> for Singleton { @@ -403,13 +424,18 @@ impl<'a, T, W, C, N: Location<'a>> Optional { } } +impl<'a, T, N: Location<'a>> DeferTick for Optional { + fn defer_tick(self) -> Self { + Optional::defer_tick(self) + } +} + impl<'a, T, N: Location<'a>> CycleComplete<'a, Tick> for Optional { fn complete(self, ident: syn::Ident) { - let me = self.defer_tick(); - me.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { ident, - location_kind: me.location_kind, - input: Box::new(me.ir_node.into_inner()), + location_kind: self.location_kind, + input: Box::new(self.ir_node.into_inner()), }); } } diff --git a/hydroflow_plus/src/stream.rs b/hydroflow_plus/src/stream.rs index 89b1d8476b2..f374d5761bb 100644 --- a/hydroflow_plus/src/stream.rs +++ b/hydroflow_plus/src/stream.rs @@ -14,7 +14,7 @@ use syn::parse_quote; use super::staging_util::get_this_crate; use crate::builder::{self, FlowState}; -use crate::cycle::{CycleCollection, CycleComplete}; +use crate::cycle::{CycleCollection, CycleComplete, DeferTick}; use crate::ir::{DebugInstantiate, HfPlusLeaf, HfPlusNode, HfPlusSource, TeeNode}; use crate::location::{ CanSend, ExternalBincodeStream, ExternalBytesPort, ExternalProcess, Location, LocationId, @@ -54,13 +54,18 @@ pub struct Stream { _phantom: PhantomData<(T, N, W, C)>, } +impl<'a, T, N: Location<'a>> DeferTick for Stream { + fn defer_tick(self) -> Self { + Stream::defer_tick(self) + } +} + impl<'a, T, N: Location<'a>> CycleComplete<'a, Tick> for Stream { fn complete(self, ident: syn::Ident) { - let me = self.defer_tick(); - me.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { + self.flow_state.borrow_mut().leaves.as_mut().expect("Attempted to add a leaf to a flow that has already been finalized. No leaves can be added after the flow has been compiled.").push(HfPlusLeaf::CycleSink { ident, - location_kind: me.location_kind, - input: Box::new(me.ir_node.into_inner()), + location_kind: self.location_kind, + input: Box::new(self.ir_node.into_inner()), }); } } diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 0667dae4e9d..7ebdc8160af 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -96,56 +96,93 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( proposers .source_iter(q!(["Proposers say hello"])) .for_each(q!(|s| println!("{}", s))); - let p_id = proposers.self_id(); - let (p_to_proposers_i_am_leader_complete_cycle, p_to_proposers_i_am_leader) = - proposers.cycle::>(); - let (a_to_proposers_p1b_complete_cycle, a_to_proposers_p1b) = - proposers.cycle::>(); let (a_to_proposers_p2b_complete_cycle, a_to_proposers_p2b) = - proposers.cycle::>(); - // 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))); + proposers.forward_ref::>(); + let (a_log_complete_cycle, a_log_forward_reference) = + acceptors.tick_forward_ref::>(); + + let (p_ballot_num, p_is_leader, p_max_slot, p_log_to_try_commit, p_log_holes, a_max_ballot) = + leader_election( + &proposers, + &acceptors, + f, + i_am_leader_send_timeout, + i_am_leader_check_timeout, + i_am_leader_check_timeout_delay_multiplier, + a_to_proposers_p2b, + a_log_forward_reference, + ); - let p_received_max_ballot = p_max_ballot( - &proposers, - a_to_proposers_p1b.clone(), - a_to_proposers_p2b.clone(), - p_to_proposers_i_am_leader.clone(), - ); - let (p_ballot_num, p_has_largest_ballot) = - p_ballot_calc(&proposers, p_received_max_ballot.latest_tick()); + let (p_to_clients_new_leader_elected, p_to_replicas, a_log, a_to_proposers_p2b_new) = + sequence_payload( + &proposers, + &acceptors, + c_to_proposers, + r_to_acceptors_checkpoint, + p_ballot_num, + p_is_leader, + p_max_slot, + p_log_to_try_commit, + p_log_holes, + f, + a_max_ballot, + ); - let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( - &proposers, - a_to_proposers_p1b.inspect(q!(|p1b| println!("Proposer received P1b: {:?}", p1b))), - p_ballot_num.clone(), - p_has_largest_ballot, - f, - ); + a_log_complete_cycle.complete(a_log.clone()); + a_to_proposers_p2b_complete_cycle.complete(a_to_proposers_p2b_new); - let (p_to_proposers_i_am_leader_from_others, p_to_acceptors_p1a) = p_p1a( - p_ballot_num.clone(), - p_is_leader.clone(), - &proposers, - p_to_proposers_i_am_leader, - &acceptors, - i_am_leader_send_timeout, - i_am_leader_check_timeout, - i_am_leader_check_timeout_delay_multiplier, - ); - p_to_proposers_i_am_leader_complete_cycle.complete(p_to_proposers_i_am_leader_from_others); + ( + proposers, + acceptors, + p_to_clients_new_leader_elected, + p_to_replicas, + ) +} + +#[expect( + clippy::type_complexity, + clippy::too_many_arguments, + reason = "internal paxos code // TODO" +)] +fn sequence_payload<'a, P: PaxosPayload, R>( + proposers: &Cluster<'a, Proposer>, + acceptors: &Cluster<'a, Acceptor>, + c_to_proposers: Stream>, + r_to_acceptors_checkpoint: impl FnOnce( + &Cluster<'a, Acceptor>, + ) -> Stream< + (ClusterId, i32), + Unbounded, + NoTick, + Cluster<'a, Acceptor>, + >, + + p_ballot_num: Singleton>, + p_is_leader: Optional>, + p_max_slot: Optional>, + + p_log_to_try_commit: Stream, Bounded, Tick, Cluster<'a, Proposer>>, + p_log_holes: Stream, Bounded, Tick, Cluster<'a, Proposer>>, + f: usize, + a_max_ballot: Singleton>, +) -> ( + Stream>, + Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>>, + Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, + Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, +) { + let p_id = proposers.self_id(); let (p_next_slot, p_to_acceptors_p2a) = p_p2a( - &proposers, + proposers, p_max_slot, c_to_proposers, p_ballot_num.clone(), p_log_to_try_commit, p_log_holes, p_is_leader.clone(), - &acceptors, + acceptors, ); // Tell clients that leader election has completed and they can begin sending messages @@ -154,38 +191,129 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( .cross_singleton(p_ballot_num) .map(q!(move |(_is_leader, ballot_num)| Ballot { num: ballot_num, proposer_id: p_id})) // Only tell the clients once when leader election concludes .all_ticks(); - // End tell clients that leader election has completed - let p_to_replicas = p_p2b(&proposers, a_to_proposers_p2b, f); // Acceptors. acceptors .source_iter(q!(["Acceptors say hello"])) .for_each(q!(|s| println!("{}", s))); - let r_to_acceptors_checkpoint = r_to_acceptors_checkpoint(&acceptors); + let r_to_acceptors_checkpoint = r_to_acceptors_checkpoint(acceptors); // p_to_acceptors_p2a.clone().for_each(q!(|p2a: P2a| println!("Acceptor received P2a: {:?}", p2a))); - let (a_to_proposers_p1b_new, a_to_proposers_p2b_new) = acceptor( - p_to_acceptors_p1a.inspect(q!(|p1a| println!("Acceptor received P1a: {:?}", p1a))), + let (a_log, a_to_proposers_p2b) = acceptor_p2( + a_max_ballot.clone(), p_to_acceptors_p2a, r_to_acceptors_checkpoint, - &proposers, - &acceptors, + proposers, + acceptors, f, ); - a_to_proposers_p1b_complete_cycle.complete(a_to_proposers_p1b_new); - a_to_proposers_p2b_complete_cycle.complete(a_to_proposers_p2b_new); + + // End tell clients that leader election has completed + let p_to_replicas = p_p2b(proposers, a_to_proposers_p2b.clone(), f); ( - proposers, - acceptors, p_to_clients_new_leader_elected, p_to_replicas, + a_log, + a_to_proposers_p2b, + ) +} + +#[expect( + clippy::type_complexity, + clippy::too_many_arguments, + reason = "internal paxos code // TODO" +)] +fn leader_election<'a, P: PaxosPayload>( + proposers: &Cluster<'a, Proposer>, + acceptors: &Cluster<'a, Acceptor>, + f: usize, + i_am_leader_send_timeout: u64, + i_am_leader_check_timeout: u64, + i_am_leader_check_timeout_delay_multiplier: usize, + a_to_proposers_p2b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + a_log: Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, +) -> ( + Singleton>, + Optional>, + Optional>, + Stream, Bounded, Tick, Cluster<'a, Proposer>>, + Stream, Bounded, Tick, Cluster<'a, Proposer>>, + Singleton>, +) { + let (a_to_proposers_p1b_complete_cycle, a_to_proposers_p1b) = + proposers.forward_ref::>(); + let (p_to_proposers_i_am_leader_complete_cycle, p_to_proposers_i_am_leader) = + proposers.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.clone(), + a_to_proposers_p2b, + p_to_proposers_i_am_leader.clone(), + ); + let (p_ballot_num, p_has_largest_ballot) = + p_ballot_calc(proposers, p_received_max_ballot.latest_tick()); + + let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( + proposers, + a_to_proposers_p1b.inspect(q!(|p1b| println!("Proposer received P1b: {:?}", p1b))), + p_ballot_num.clone(), + p_has_largest_ballot, + f, + ); + + let (p_to_proposers_i_am_leader_from_others, p_to_acceptors_p1a) = p_p1a( + p_ballot_num.clone(), + p_is_leader.clone(), + proposers, + p_to_proposers_i_am_leader, + acceptors, + i_am_leader_send_timeout, + i_am_leader_check_timeout, + i_am_leader_check_timeout_delay_multiplier, + ); + p_to_proposers_i_am_leader_complete_cycle.complete(p_to_proposers_i_am_leader_from_others); + + let a_max_ballot = p_to_acceptors_p1a + .clone() + .inspect(q!(|p1a| println!("Acceptor received P1a: {:?}", p1a))) + .map(q!(|p1a| p1a.ballot)) + .max() + .unwrap_or(acceptors.singleton(q!(Ballot { + num: 0, + proposer_id: ClusterId::from_raw(0) + }))); + + a_to_proposers_p1b_complete_cycle.complete(acceptor_p1( + p_to_acceptors_p1a, + &a_max_ballot, + a_log, + proposers, + )); + + ( + p_ballot_num, + p_is_leader, + p_max_slot, + p_log_to_try_commit, + p_log_holes, + a_max_ballot, ) } +#[derive(Clone)] +enum CheckpointOrP2a

{ + Checkpoint(i32), + P2a(P2a

), +} + #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn acceptor<'a, P: PaxosPayload, R>( - p_to_acceptors_p1a: Stream>, +fn acceptor_p2<'a, P: PaxosPayload, R>( + a_max_ballot: Singleton>, p_to_acceptors_p2a: Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, r_to_acceptors_checkpoint: Stream< (ClusterId, i32), @@ -197,7 +325,7 @@ fn acceptor<'a, P: PaxosPayload, R>( acceptors: &Cluster<'a, Acceptor>, f: usize, ) -> ( - Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, ) { // Get the latest checkpoint sequence per replica @@ -223,35 +351,16 @@ fn acceptor<'a, P: PaxosPayload, R>( .min() .unwrap_or(acceptors.singleton(q!(-1)).latest_tick()) .delta() - .map(q!(|min_seq| ( - min_seq, - P2a { - // Create tuple with checkpoint number and dummy p2a - ballot: Ballot { - num: 0, - proposer_id: ClusterId::from_raw(0) - }, - slot: -1, - value: Default::default() - } - ))); + .map(q!(|min_seq| CheckpointOrP2a::Checkpoint(min_seq))); // .inspect(q!(|(min_seq, p2a): &(i32, P2a)| println!("Acceptor new checkpoint: {:?}", min_seq))); - let a_max_ballot = p_to_acceptors_p1a - .clone() - .map(q!(|p1a| p1a.ballot)) - .max() - .unwrap_or(acceptors.singleton(q!(Ballot { - num: 0, - proposer_id: ClusterId::from_raw(0) - }))); let a_p2as_to_place_in_log = p_to_acceptors_p2a .clone() .tick_batch() .cross_singleton(a_max_ballot.clone().latest_tick()) // Don't consider p2as if the current ballot is higher .filter_map(q!(|(p2a, max_ballot)| if p2a.ballot >= max_ballot { - Some((-1, p2a)) // Signal that this isn't a checkpoint with -1 + Some(CheckpointOrP2a::P2a(p2a)) } else { None } @@ -261,60 +370,39 @@ fn acceptor<'a, P: PaxosPayload, R>( .persist() .fold( q!(|| (-1, HashMap::new())), - q!(|(prev_checkpoint, log), (new_checkpoint, p2a)| { - if new_checkpoint != -1 { - // This is a checkpoint message. Delete all entries up to the checkpoint - for slot in *prev_checkpoint..new_checkpoint { - log.remove(&slot); + 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..new_checkpoint { + log.remove(&slot); + } + *prev_checkpoint = new_checkpoint; } - *prev_checkpoint = new_checkpoint; - } else { - // 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 - if p2a.slot > *prev_checkpoint { - match log.get(&p2a.slot) { - None => { - log.insert( - p2a.slot, - LogValue { - ballot: p2a.ballot, - value: p2a.value, - }, - ); - } - Some(prev_p2a) => { - if p2a.ballot > prev_p2a.ballot { - log.insert( - p2a.slot, - LogValue { - ballot: p2a.ballot, - value: p2a.value, - }, - ); - } - } - }; + 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 + if p2a.slot > *prev_checkpoint + && 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, + }, + ); + } } } }), ); - let a_to_proposers_p1b_new = p_to_acceptors_p1a - .tick_batch() - .cross_singleton(a_max_ballot.clone().latest_tick()) - .cross_singleton(a_log) - .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( - p1a.ballot.proposer_id, - P1b { - ballot: p1a.ballot, - max_ballot, - accepted: log - } - ))) - .all_ticks() - .send_bincode_interleaved(proposers); let a_to_proposers_p2b_new = p_to_acceptors_p2a .tick_batch() - .cross_singleton(a_max_ballot.latest_tick()) + .cross_singleton(a_max_ballot.clone().latest_tick()) .map(q!(|(p2a, max_ballot)| ( p2a.ballot.proposer_id, P2b { @@ -326,7 +414,30 @@ fn acceptor<'a, P: PaxosPayload, R>( ))) .all_ticks() .send_bincode_interleaved(proposers); - (a_to_proposers_p1b_new, a_to_proposers_p2b_new) + (a_log, a_to_proposers_p2b_new) +} + +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] +fn acceptor_p1<'a, P: PaxosPayload>( + p_to_acceptors_p1a: Stream>, + a_max_ballot: &Singleton>, + a_log: Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, + proposers: &Cluster<'a, Proposer>, +) -> Stream, Unbounded, NoTick, Cluster<'a, Proposer>> { + p_to_acceptors_p1a + .tick_batch() + .cross_singleton(a_max_ballot.clone().latest_tick()) + .cross_singleton(a_log) + .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( + p1a.ballot.proposer_id, + P1b { + ballot: p1a.ballot, + max_ballot, + accepted: log + } + ))) + .all_ticks() + .send_bincode_interleaved(proposers) } fn p_p2b<'a, P: PaxosPayload>( diff --git a/hydroflow_plus_test/src/cluster/paxos_bench.rs b/hydroflow_plus_test/src/cluster/paxos_bench.rs index 5b72192b093..37cfc078305 100644 --- a/hydroflow_plus_test/src/cluster/paxos_bench.rs +++ b/hydroflow_plus_test/src/cluster/paxos_bench.rs @@ -63,7 +63,7 @@ pub fn paxos_bench<'a>( let clients = flow.cluster::(); let replicas = flow.cluster::(); - let (c_to_proposers_complete_cycle, c_to_proposers) = clients.cycle(); + let (c_to_proposers_complete_cycle, c_to_proposers) = clients.forward_ref(); let (proposers, acceptors, p_to_clients_new_leader_elected, r_new_processed_payloads) = paxos_with_replica( @@ -115,7 +115,7 @@ fn paxos_with_replica<'a>( Stream<(ClusterId, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Replica>>, ) { let (r_to_acceptors_checkpoint_complete_cycle, r_to_acceptors_checkpoint) = - replicas.cycle::>(); + replicas.forward_ref::>(); let (proposers, acceptors, p_to_clients_new_leader_elected, p_to_replicas) = paxos_core( flow, 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 af57d72b873..adbac8d5663 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 @@ -21,96 +21,98 @@ expression: built.ir() location_kind: Cluster( 2, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , proposer_id : p_id , }) { received_max_ballot . num + 1 } else { ballot_num } } }), - input: CrossSingleton( - Tee { - inner: : Union( - 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( - Union( + input: DeferTick( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , proposer_id : p_id , }) { received_max_ballot . num + 1 } else { ballot_num } } }), + input: CrossSingleton( + Tee { + inner: : Union( + 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( Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | p1b . max_ballot }), - input: Tee { - inner: : CycleSource { - ident: Ident { - sym: cycle_1, + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | p1b . max_ballot }), + input: Tee { + inner: : CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 2, - ), }, }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | p2b . ballot }), - input: Tee { - inner: : CycleSource { - ident: Ident { - sym: cycle_2, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | p2b . ballot }), + input: Tee { + inner: : CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 2, - ), }, }, - }, - ), - Tee { - inner: : CycleSource { - ident: Ident { - sym: cycle_0, + ), + Tee { + inner: : CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 2, - ), }, - }, - ), - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, - ), - location_kind: Cluster( - 2, + ), ), }, - ), - ), - }, - Tee { - inner: : Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 2, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), + ), + }, + Tee { + inner: : Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, location_kind: Cluster( 2, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), - }, - ), - }, + }, + ), + }, + ), }, CycleSink { ident: Ident { - sym: cycle_0, + sym: cycle_2, }, location_kind: Cluster( 2, @@ -234,6 +236,195 @@ expression: built.ir() }, }, }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 3, + ), + from_key: None, + to_location: Cluster( + 2, + ), + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , 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 :: 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( + 2, + ), + from_key: None, + to_location: Cluster( + 3, + ), + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , proposer_id : p_id } } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ 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: 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 = 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) } 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( + Tee { + inner: : 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: , + }, + }, + }, + }, + ), + }, + }, + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Source { + source: Stream( + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 2, + ), + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + Tee { + inner: : Union( + 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: , + }, + }, + }, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + ), + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, + ), + }, + ), + }, + }, + }, + }, CycleSink { ident: Ident { sym: cycle_4, @@ -244,7 +435,7 @@ expression: built.ir() input: DeferTick( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Union( @@ -256,12 +447,12 @@ expression: built.ir() input: CrossSingleton( Union( Tee { - inner: : Reduce { + inner: : Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { @@ -294,7 +485,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_4, }, @@ -313,15 +504,15 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , next_slot) | next_slot + num_payloads as i32 }), input: CrossSingleton( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: : Fold { + 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_bench :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -369,10 +560,10 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: : Filter { + inner: : Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | num_payloads | * num_payloads > 0 }), input: Tee { - inner: , + inner: , }, }, }, @@ -381,7 +572,7 @@ expression: built.ir() }, }, Tee { - inner: , + inner: , }, ), }, @@ -390,7 +581,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -400,7 +591,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 , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -433,7 +624,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -444,6 +635,17 @@ expression: built.ir() ), ), }, + ForEach { + f: stageleft :: runtime_support :: fn1_type_hint :: < & str , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | s | println ! ("{}" , s) }), + input: Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; ["Acceptors say hello"] }, + ), + location_kind: Cluster( + 3, + ), + }, + }, CycleSink { ident: Ident { sym: cycle_5, @@ -456,40 +658,204 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _value) | slot }), input: Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , value)) | if count > f { Some ((slot , value)) } else { None } }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , Default :: default ()) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , value | { accum . 0 += 1 ; accum . 1 = value ; } }), input: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , core :: option :: Option < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | if p2b . victory { Some (((p2b . slot , p2b . ballot) , p2b . value)) } else { None } }), input: Tee { - inner: : Union( + inner: : Union( Tee { - inner: , - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - }, - Tee { - inner: : FilterMap { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 3, + ), + from_key: None, + to_location: Cluster( + 2, + ), + 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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_bench :: ClientPayload > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , P2b { victory : p2a . ballot == max_ballot , ballot : p2a . ballot , slot : p2a . slot , value : p2a . value }) }), + input: CrossSingleton( + Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 2, + ), + from_key: None, + to_location: Cluster( + 3, + ), + 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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_bench :: ClientPayload > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Union( + FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id , } , slot , value : entry . value , }) } else { None } }), + input: CrossSingleton( + Tee { + inner: , + }, + Tee { + inner: , + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id } , slot , value : Default :: default () } }), + input: CrossSingleton( + Difference( + FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: ops :: Range < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | 0 .. max_slot }), + input: Tee { + inner: , + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + input: Tee { + inner: , + }, + }, + ), + Tee { + inner: , + }, + ), + }, + ), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: stream :: * ; | 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 , i32 , () > ({ 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_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id } , slot : next_slot + index as i32 , value : payload } }), + input: CrossSingleton( + CrossSingleton( + Enumerate( + Tee { + inner: , + }, + ), + Tee { + inner: , + }, + ), + Tee { + inner: , + }, + ), + }, + ), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), + input: Tee { + inner: , + }, + }, + ), + }, + }, + }, + }, + }, + Tee { + inner: , + }, + ), + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + Tee { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , _p2b)) | if count == 2 * f + 1 { Some (slot) } else { None } }), input: Tee { - inner: , + inner: , }, }, }, @@ -510,194 +876,121 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | (p2b . slot , p2b) }), input: Tee { - inner: , + inner: , }, }, Tee { - inner: , + inner: , }, ), }, ), }, - ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < & str , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | s | println ! ("{}" , s) }), - input: Source { - source: Iter( - { use crate :: __staged :: cluster :: paxos :: * ; ["Acceptors say hello"] }, - ), - location_kind: Cluster( - 3, - ), - }, - }, CycleSink { ident: Ident { - sym: cycle_1, + sym: cycle_0, }, location_kind: Cluster( - 2, + 3, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 3, - ), - from_key: None, - to_location: Cluster( - 2, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", - ], + input: Tee { + inner: : Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: new ()) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { for slot in * prev_checkpoint .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = new_checkpoint ; } CheckpointOrP2a :: P2a (p2a) => { if p2a . slot > * prev_checkpoint && 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( + Union( + FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ 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: , + }, + ), }, - ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , log)) | (p1a . ballot . proposer_id , P1b { ballot : p1a . ballot , max_ballot , accepted : log }) }), - input: CrossSingleton( - CrossSingleton( - Tee { - inner: : 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: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: 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( - 2, - ), - from_key: None, - to_location: Cluster( - 3, - ), - 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , proposer_id : p_id } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ 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: 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 = 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) } 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( - Tee { - inner: : 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: , - }, - }, - }, - }, - ), - }, - }, - }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), + input: Delta( + Union( + Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , ()) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: : ReduceKeyed { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), + input: Persist( + Network { + from_location: Cluster( + 1, ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Source { - source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + from_key: None, + to_location: Cluster( + 3, + ), + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , i32) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + ], + }, ), - location_kind: Cluster( - 2, + ), + 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 :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + ], + }, ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + }, }, }, ), }, }, - }, + 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 = 1usize ; move | num_received | if num_received == f + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: , + }, + }, + }, + }, + ), }, }, }, - }, - }, - Tee { - inner: : Union( - 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: Tee { - inner: , - }, - }, - ), - }, Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, ), location_kind: Cluster( 3, @@ -705,293 +998,22 @@ expression: built.ir() }, ), ), - }, - ), - Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: new ()) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , (new_checkpoint , p2a) | { if new_checkpoint != - 1 { for slot in * prev_checkpoint .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = new_checkpoint ; } else { if p2a . slot > * prev_checkpoint { match log . get (& p2a . slot) { None => { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } Some (prev_p2a) => { if p2a . ballot > prev_p2a . ballot { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } ; } } } }), - input: Persist( - Union( - FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some ((- 1 , p2a)) } else { None } }), - input: CrossSingleton( - Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 2, - ), - from_key: None, - to_location: Cluster( - 3, - ), - 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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_bench :: ClientPayload > > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Union( - FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id , } , slot , value : entry . value , }) } else { None } }), - input: CrossSingleton( - Tee { - inner: , - }, - Tee { - inner: , - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id } , slot , value : Default :: default () } }), - input: CrossSingleton( - Difference( - FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: ops :: Range < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | 0 .. max_slot }), - input: Tee { - inner: , - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), - input: Tee { - inner: , - }, - }, - ), - Tee { - inner: , - }, - ), - }, - ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: stream :: * ; | 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 , i32 , () > ({ 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_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id } , slot : next_slot + index as i32 , value : payload } }), - input: CrossSingleton( - CrossSingleton( - Enumerate( - Tee { - inner: , - }, - ), - Tee { - inner: , - }, - ), - Tee { - inner: , - }, - ), - }, - ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), - input: Tee { - inner: , - }, - }, - ), - }, - }, - }, - }, - }, - Tee { - inner: , - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | (min_seq , P2a { ballot : Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } , slot : - 1 , value : Default :: default () }) }), - input: Delta( - Union( - Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , ()) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: : ReduceKeyed { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), - input: Persist( - Network { - from_location: Cluster( - 1, - ), - from_key: None, - to_location: Cluster( - 3, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , i32) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 1, - ), - }, - }, - }, - ), - }, - }, - 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 = 1usize ; move | num_received | if num_received == f + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: , - }, - }, - }, - }, - ), - }, - }, - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), - }, - ), - ), - ), - }, - ), ), }, ), - }, + ), }, }, }, CycleSink { ident: Ident { - sym: cycle_2, + sym: cycle_0, }, location_kind: Cluster( 2, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 3, - ), - from_key: None, - to_location: Cluster( - 2, - ), - 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_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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_bench :: ClientPayload > > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , P2b { victory : p2a . ballot == max_ballot , ballot : p2a . ballot , slot : p2a . slot , value : p2a . value }) }), - input: CrossSingleton( - Tee { - inner: , - }, - Tee { - inner: , - }, - ), - }, - }, + input: Tee { + inner: , }, }, CycleSink { @@ -1008,10 +1030,10 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), input: CrossSingleton( Tee { - inner: : Sort( + inner: : Sort( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -1051,7 +1073,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , value) | (slot , value) }), input: AntiJoin( Tee { - inner: , + inner: , }, CycleSource { ident: Ident { @@ -1080,12 +1102,12 @@ expression: built.ir() ), }, Tee { - inner: : Fold { + inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | - 1 }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | filled_slot , (sorted_payload , highest_seq) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Union( CycleSource { @@ -1122,23 +1144,23 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | (HashMap :: < u32 , String > :: new () , - 1) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | state , payload | { let kv_store = & mut state . 0 ; let last_seq = & mut state . 1 ; kv_store . insert (payload . key , payload . value) ; debug_assert ! (payload . seq == * last_seq + 1 , "Hole in log between seq {} and {}" , * last_seq , payload . seq) ; * last_seq = payload . seq ; } }), input: Persist( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1159,7 +1181,7 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , i32) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let checkpoint_frequency = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if new_highest_seq - max_checkpointed_seq >= checkpoint_frequency as i32 { Some (new_highest_seq) } else { None } }), input: CrossSingleton( Union( @@ -1188,7 +1210,7 @@ expression: built.ir() ), ), Tee { - inner: , + inner: , }, ), }, @@ -1203,7 +1225,7 @@ expression: built.ir() 1, ), input: Tee { - inner: , + inner: , }, }, CycleSink { @@ -1216,7 +1238,7 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Union( + inner: : Union( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { @@ -1252,7 +1274,7 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1268,13 +1290,13 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 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 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { - inner: , + inner: , }, }, }, @@ -1295,7 +1317,7 @@ expression: built.ir() input: Union( Union( Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_2, }, @@ -1309,9 +1331,9 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: time :: SystemTime > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | SystemTime :: now () }), input: Tee { - inner: : Delta( + inner: : Delta( Tee { - inner: : Reduce { + inner: : 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 { @@ -1367,7 +1389,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1392,10 +1414,10 @@ expression: built.ir() }, ), Tee { - inner: : Map { + 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 ()) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1408,7 +1430,7 @@ expression: built.ir() input: CrossSingleton( CrossSingleton( Tee { - inner: : Source { + inner: : Source { source: Interval( { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, ), @@ -1426,10 +1448,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: , }, ), }, @@ -1437,7 +1459,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), input: Tee { - inner: , + inner: , }, }, ), @@ -1459,7 +1481,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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, Map { @@ -1470,7 +1492,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 , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1482,7 +1504,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), input: Tee { - inner: , + inner: , }, }, ), @@ -1502,17 +1524,17 @@ expression: built.ir() FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . leader_id () , ClientPayload { key : i as u32 , value : c_id . raw_id . to_string () })) }), input: Tee { - inner: , + inner: , }, }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; move | (key , leader_ballot) | (leader_ballot . leader_id () , ClientPayload { key , value : c_id . raw_id . to_string () }) }), input: CrossSingleton( Tee { - inner: , + inner: , }, 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 c3a27f6c1d3..97f48660e85 100644 --- a/hydroflow_plus_test_local/src/local/graph_reachability.rs +++ b/hydroflow_plus_test_local/src/local/graph_reachability.rs @@ -16,7 +16,7 @@ pub fn graph_reachability<'a>( let roots = process.source_stream(roots); let edges = process.source_stream(edges); - let (set_reached_cycle, reached_cycle) = process.cycle(); + let (set_reached_cycle, reached_cycle) = process.forward_ref(); let reached = roots.union(reached_cycle); let reachable = reached diff --git a/stageleft_tool/src/lib.rs b/stageleft_tool/src/lib.rs index 35bbff0f0f0..22cdb6ef0f3 100644 --- a/stageleft_tool/src/lib.rs +++ b/stageleft_tool/src/lib.rs @@ -125,6 +125,10 @@ impl VisitMut for GenFinalPubVistor { syn::visit_mut::visit_item_enum_mut(self, i); } + fn visit_variant_mut(&mut self, _i: &mut syn::Variant) { + // variant fields do not have visibility modifiers + } + fn visit_item_struct_mut(&mut self, i: &mut syn::ItemStruct) { i.vis = parse_quote!(pub); syn::visit_mut::visit_item_struct_mut(self, i); From dfb3cf66ae85734fb600d9058b21c88059d7a856 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 8 Oct 2024 14:30:25 -0700 Subject: [PATCH 3/5] refactor(paxos): rearrange code to reflect order of phases --- hydroflow_plus_test/src/cluster/paxos.rs | 925 +++++++++--------- ...cluster__paxos_bench__tests__paxos_ir.snap | 688 ++++++------- 2 files changed, 814 insertions(+), 799 deletions(-) diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 7ebdc8160af..94a8fb5cfa7 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -97,7 +97,7 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( .source_iter(q!(["Proposers say hello"])) .for_each(q!(|s| println!("{}", s))); - let (a_to_proposers_p2b_complete_cycle, a_to_proposers_p2b) = + let (a_to_proposers_p2b_complete_cycle, a_to_proposers_p2b_forward_reference) = proposers.forward_ref::>(); let (a_log_complete_cycle, a_log_forward_reference) = acceptors.tick_forward_ref::>(); @@ -110,11 +110,11 @@ 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, + a_to_proposers_p2b_forward_reference, a_log_forward_reference, ); - let (p_to_clients_new_leader_elected, p_to_replicas, a_log, a_to_proposers_p2b_new) = + let (p_to_clients_new_leader_elected, p_to_replicas, a_log, a_to_proposers_p2b) = sequence_payload( &proposers, &acceptors, @@ -130,7 +130,7 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( ); a_log_complete_cycle.complete(a_log.clone()); - a_to_proposers_p2b_complete_cycle.complete(a_to_proposers_p2b_new); + a_to_proposers_p2b_complete_cycle.complete(a_to_proposers_p2b); ( proposers, @@ -140,85 +140,6 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( ) } -#[expect( - clippy::type_complexity, - clippy::too_many_arguments, - reason = "internal paxos code // TODO" -)] -fn sequence_payload<'a, P: PaxosPayload, R>( - proposers: &Cluster<'a, Proposer>, - acceptors: &Cluster<'a, Acceptor>, - c_to_proposers: Stream>, - r_to_acceptors_checkpoint: impl FnOnce( - &Cluster<'a, Acceptor>, - ) -> Stream< - (ClusterId, i32), - Unbounded, - NoTick, - Cluster<'a, Acceptor>, - >, - - p_ballot_num: Singleton>, - p_is_leader: Optional>, - p_max_slot: Optional>, - - p_log_to_try_commit: Stream, Bounded, Tick, Cluster<'a, Proposer>>, - p_log_holes: Stream, Bounded, Tick, Cluster<'a, Proposer>>, - f: usize, - - a_max_ballot: Singleton>, -) -> ( - Stream>, - Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>>, - Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, - Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, -) { - let p_id = proposers.self_id(); - let (p_next_slot, p_to_acceptors_p2a) = p_p2a( - proposers, - p_max_slot, - c_to_proposers, - p_ballot_num.clone(), - p_log_to_try_commit, - p_log_holes, - p_is_leader.clone(), - acceptors, - ); - - // Tell clients that leader election has completed and they can begin sending messages - let p_to_clients_new_leader_elected = p_is_leader.clone() - .continue_unless(p_next_slot) - .cross_singleton(p_ballot_num) - .map(q!(move |(_is_leader, ballot_num)| Ballot { num: ballot_num, proposer_id: p_id})) // Only tell the clients once when leader election concludes - .all_ticks(); - - // Acceptors. - acceptors - .source_iter(q!(["Acceptors say hello"])) - .for_each(q!(|s| println!("{}", s))); - let r_to_acceptors_checkpoint = r_to_acceptors_checkpoint(acceptors); - - // p_to_acceptors_p2a.clone().for_each(q!(|p2a: P2a| println!("Acceptor received P2a: {:?}", p2a))); - let (a_log, a_to_proposers_p2b) = acceptor_p2( - a_max_ballot.clone(), - p_to_acceptors_p2a, - r_to_acceptors_checkpoint, - proposers, - acceptors, - f, - ); - - // End tell clients that leader election has completed - let p_to_replicas = p_p2b(proposers, a_to_proposers_p2b.clone(), f); - - ( - p_to_clients_new_leader_elected, - p_to_replicas, - a_log, - a_to_proposers_p2b, - ) -} - #[expect( clippy::type_complexity, clippy::too_many_arguments, @@ -245,32 +166,25 @@ fn leader_election<'a, P: PaxosPayload>( proposers.forward_ref::>(); let (p_to_proposers_i_am_leader_complete_cycle, p_to_proposers_i_am_leader) = proposers.forward_ref::>(); + let (p_is_leader_complete_cycle, p_is_leader) = + proposers.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.clone(), + a_to_proposers_p1b, a_to_proposers_p2b, p_to_proposers_i_am_leader.clone(), ); let (p_ballot_num, p_has_largest_ballot) = p_ballot_calc(proposers, p_received_max_ballot.latest_tick()); - let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( - proposers, - a_to_proposers_p1b.inspect(q!(|p1b| println!("Proposer received P1b: {:?}", p1b))), - p_ballot_num.clone(), - p_has_largest_ballot, - f, - ); - let (p_to_proposers_i_am_leader_from_others, p_to_acceptors_p1a) = p_p1a( p_ballot_num.clone(), p_is_leader.clone(), proposers, - p_to_proposers_i_am_leader, acceptors, i_am_leader_send_timeout, i_am_leader_check_timeout, @@ -288,12 +202,17 @@ fn leader_election<'a, P: PaxosPayload>( proposer_id: ClusterId::from_raw(0) }))); - a_to_proposers_p1b_complete_cycle.complete(acceptor_p1( - p_to_acceptors_p1a, - &a_max_ballot, - a_log, + let a_to_proposers_p1b = acceptor_p1(p_to_acceptors_p1a, &a_max_ballot, a_log, proposers); + a_to_proposers_p1b_complete_cycle.complete(a_to_proposers_p1b.clone()); + + let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( proposers, - )); + a_to_proposers_p1b.inspect(q!(|p1b| println!("Proposer received P1b: {:?}", p1b))), + p_ballot_num.clone(), + p_has_largest_ballot, + f, + ); + p_is_leader_complete_cycle.complete(p_is_leader.clone()); ( p_ballot_num, @@ -305,282 +224,177 @@ fn leader_election<'a, P: PaxosPayload>( ) } -#[derive(Clone)] -enum CheckpointOrP2a

{ - Checkpoint(i32), - P2a(P2a

), +// Proposer logic to calculate the largest ballot received so far. +fn p_max_ballot<'a, P: PaxosPayload>( + proposers: &Cluster<'a, Proposer>, + a_to_proposers_p1b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + a_to_proposers_p2b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + p_to_proposers_i_am_leader: Stream>, +) -> Singleton> { + let p_received_p1b_ballots = a_to_proposers_p1b.clone().map(q!(|p1b| p1b.max_ballot)); + let p_received_p2b_ballots = a_to_proposers_p2b.clone().map(q!(|p2b| p2b.ballot)); + p_received_p1b_ballots + .union(p_received_p2b_ballots) + .union(p_to_proposers_i_am_leader) + .max() + .unwrap_or(proposers.singleton(q!(Ballot { + num: 0, + proposer_id: ClusterId::from_raw(0) + }))) } +// 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. #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn acceptor_p2<'a, P: PaxosPayload, R>( - a_max_ballot: Singleton>, - p_to_acceptors_p2a: Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, - r_to_acceptors_checkpoint: Stream< - (ClusterId, i32), - Unbounded, - NoTick, - Cluster<'a, Acceptor>, - >, +fn p_ballot_calc<'a>( proposers: &Cluster<'a, Proposer>, - acceptors: &Cluster<'a, Acceptor>, - f: usize, + p_received_max_ballot: Singleton>, ) -> ( - Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, - Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + Singleton>, + Optional<(Ballot, u32), Bounded, Tick, Cluster<'a, Proposer>>, ) { - // Get the latest checkpoint sequence per replica - let a_checkpoint_largest_seqs = - r_to_acceptors_checkpoint - .tick_prefix() - .reduce_keyed(q!(|curr_seq, seq| { - if seq > *curr_seq { - *curr_seq = seq; - } - })); - let a_checkpoints_quorum_reached = a_checkpoint_largest_seqs.clone().count().filter_map(q!( - move |num_received| if num_received == f + 1 { - Some(true) - } else { - None - } - )); - // Find the smallest checkpoint seq that everyone agrees to, track whenever it changes - let a_new_checkpoint = a_checkpoint_largest_seqs - .continue_if(a_checkpoints_quorum_reached) - .map(q!(|(_sender, seq)| seq)) - .min() - .unwrap_or(acceptors.singleton(q!(-1)).latest_tick()) - .delta() - .map(q!(|min_seq| CheckpointOrP2a::Checkpoint(min_seq))); - // .inspect(q!(|(min_seq, p2a): &(i32, P2a)| println!("Acceptor new checkpoint: {:?}", min_seq))); + let p_id = proposers.self_id(); + let (p_ballot_num_complete_cycle, p_ballot_num) = + proposers.tick_cycle_with_initial(proposers.singleton(q!(0)).latest_tick()); - let a_p2as_to_place_in_log = p_to_acceptors_p2a + let p_new_ballot_num = p_received_max_ballot .clone() - .tick_batch() - .cross_singleton(a_max_ballot.clone().latest_tick()) // Don't consider p2as if the current ballot is higher - .filter_map(q!(|(p2a, max_ballot)| - if p2a.ballot >= max_ballot { - Some(CheckpointOrP2a::P2a(p2a)) + .cross_singleton(p_ballot_num.clone()) + .map(q!(move |(received_max_ballot, ballot_num)| { + if received_max_ballot + > (Ballot { + num: ballot_num, + proposer_id: p_id, + }) + { + received_max_ballot.num + 1 } else { - None + ballot_num } - )); - let a_log = a_p2as_to_place_in_log - .union(a_new_checkpoint.into_stream()) - .persist() - .fold( - q!(|| (-1, 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..new_checkpoint { - log.remove(&slot); - } - *prev_checkpoint = 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 - if p2a.slot > *prev_checkpoint - && 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, - }, - ); - } - } + })); + p_ballot_num_complete_cycle.complete_next_tick(p_new_ballot_num); + + let p_has_largest_ballot = p_received_max_ballot + .clone() + .cross_singleton(p_ballot_num.clone()) + .filter(q!( + move |(received_max_ballot, ballot_num)| *received_max_ballot + <= Ballot { + num: *ballot_num, + proposer_id: p_id } - }), - ); + )); - let a_to_proposers_p2b_new = p_to_acceptors_p2a - .tick_batch() - .cross_singleton(a_max_ballot.clone().latest_tick()) - .map(q!(|(p2a, max_ballot)| ( - p2a.ballot.proposer_id, - P2b { - victory: p2a.ballot == max_ballot, - ballot: p2a.ballot, - slot: p2a.slot, - value: p2a.value - } - ))) - .all_ticks() - .send_bincode_interleaved(proposers); - (a_log, a_to_proposers_p2b_new) + // End stable leader election + (p_ballot_num, p_has_largest_ballot) } -#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn acceptor_p1<'a, P: PaxosPayload>( - p_to_acceptors_p1a: Stream>, - a_max_ballot: &Singleton>, - a_log: Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, - proposers: &Cluster<'a, Proposer>, -) -> Stream, Unbounded, NoTick, Cluster<'a, Proposer>> { - p_to_acceptors_p1a - .tick_batch() - .cross_singleton(a_max_ballot.clone().latest_tick()) - .cross_singleton(a_log) - .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( - p1a.ballot.proposer_id, - P1b { - ballot: p1a.ballot, - max_ballot, - accepted: log - } - ))) - .all_ticks() - .send_bincode_interleaved(proposers) -} +fn p_leader_expired<'a>( + p_to_proposers_i_am_leader: Stream>, + p_is_leader: Optional>, + i_am_leader_check_timeout: u64, // How often to check if heartbeat expired +) -> Optional, Bounded, Tick, Cluster<'a, Proposer>> { + let p_latest_received_i_am_leader = p_to_proposers_i_am_leader.clone().fold( + q!(|| None), + q!(|latest, _| { + // Note: May want to check received ballot against our own? + *latest = Some(Instant::now()); + }), + ); -fn p_p2b<'a, P: PaxosPayload>( - proposers: &Cluster<'a, Proposer>, - a_to_proposers_p2b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, - f: usize, -) -> Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>> { - let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = proposers.tick_cycle(); - let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = proposers.tick_cycle(); - let p_p2b = a_to_proposers_p2b - .clone() - .tick_batch() - .union(p_persisted_p2bs); - let p_count_matching_p2bs = p_p2b - .clone() - .filter_map(q!(|p2b| if p2b.victory { - // 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( - q!(|| (0, Default::default())), - q!(|accum, value| { - // TODO(shadaj): why is sender unused? should we de-dup? - accum.0 += 1; - accum.1 = value; - }), - ); - let p_p2b_quorum_reached = - p_count_matching_p2bs - .clone() - .filter_map(q!(move |((slot, _ballot), (count, value))| if count > f { - Some((slot, value)) - } else { - None - })); - let p_to_replicas = p_p2b_quorum_reached + p_latest_received_i_am_leader .clone() - .anti_join(p_broadcasted_p2b_slots) // Only tell the replicas about committed values once - .map(q!(|(slot, value)| (slot, value))) - .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) + .latest_tick() + .continue_unless(p_is_leader) + .filter(q!(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) } else { - None + true } - )); - // 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 + })) } -// Proposer logic to send p2as, outputting the next slot and the p2as to send to acceptors. -#[expect( - clippy::type_complexity, - clippy::too_many_arguments, - reason = "internal paxos code // TODO" -)] -fn p_p2a<'a, P: PaxosPayload>( - proposers: &Cluster<'a, Proposer>, - p_max_slot: Optional>, - c_to_proposers: Stream>, +// Proposer logic to send "I am leader" messages periodically to other proposers, or send p1a to acceptors if other leaders expired. +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] +fn p_p1a<'a>( p_ballot_num: Singleton>, - p_log_to_try_commit: Stream, Bounded, Tick, Cluster<'a, Proposer>>, - p_log_holes: Stream, Bounded, Tick, Cluster<'a, Proposer>>, p_is_leader: Optional>, + proposers: &Cluster<'a, Proposer>, acceptors: &Cluster<'a, Acceptor>, + i_am_leader_send_timeout: u64, // How often to heartbeat + i_am_leader_check_timeout: u64, // How often to check if heartbeat expired + i_am_leader_check_timeout_delay_multiplier: usize, /* Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts */ ) -> ( - Optional>, - Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, + Stream>, + Stream>, ) { let p_id = proposers.self_id(); - let (p_next_slot_complete_cycle, p_next_slot) = - proposers.tick_cycle::>(); - let p_next_slot_after_reconciling_p1bs = p_max_slot - .unwrap_or(proposers.singleton(q!(-1)).latest_tick()) - // .inspect(q!(|max_slot| println!("{} p_max_slot: {:?}", context.current_tick(), max_slot))) - .continue_unless(p_next_slot.clone()) - .map(q!(|max_slot| max_slot + 1)); - - // Send p2as - let p_indexed_payloads = c_to_proposers + let p_to_proposers_i_am_leader = p_is_leader .clone() - .tick_batch() - .enumerate() - .cross_singleton(p_next_slot.clone()) - // .inspect(q!(|next| println!("{} p_indexed_payloads next slot: {}", context.current_tick(), next)))) - .cross_singleton(p_ballot_num.clone()) - // .inspect(q!(|ballot_num| println!("{} p_indexed_payloads ballot_num: {}", context.current_tick(), ballot_num)))) - .map(q!(move |(((index, payload), next_slot), ballot_num)| P2a { ballot: Ballot { num: ballot_num, proposer_id: p_id }, slot: next_slot + index as i32, value: payload })); - // .inspect(q!(|p2a: &P2a| println!("{} p_indexed_payloads P2a: {:?}", context.current_tick(), p2a))); - let p_to_acceptors_p2a = p_log_to_try_commit - .union(p_log_holes) - .continue_unless(p_next_slot.clone()) // Only resend p1b stuff once. Once it's resent, next_slot will exist. - .union(p_indexed_payloads) - .continue_if(p_is_leader.clone()) + .then(p_ballot_num.clone()) + .latest() + .sample_every(q!(Duration::from_secs(i_am_leader_send_timeout))) + .map(q!(move |ballot_num| Ballot { + num: ballot_num, + proposer_id: p_id + })) + .broadcast_bincode_interleaved(proposers); + + let p_leader_expired = p_leader_expired( + p_to_proposers_i_am_leader.clone(), + p_is_leader, + i_am_leader_check_timeout, + ); + + let p_id = proposers.self_id(); + + // Add random delay depending on node ID so not everyone sends p1a at the same time + let p_to_acceptors_p1a = p_leader_expired + .then(p_ballot_num) + .continue_if( + proposers + .source_interval_delayed( + q!(Duration::from_secs( + (p_id.raw_id * i_am_leader_check_timeout_delay_multiplier as u32).into() + )), + q!(Duration::from_secs(i_am_leader_check_timeout)), + ) + .latest_tick(), + ) + .map(q!(move |ballot_num| P1a { + ballot: Ballot { + num: ballot_num, + proposer_id: p_id + } + })) .all_ticks() + .inspect(q!(|_| println!("Proposer leader expired, sending P1a"))) .broadcast_bincode_interleaved(acceptors); + (p_to_proposers_i_am_leader, p_to_acceptors_p1a) +} - let p_num_payloads = c_to_proposers.clone().tick_batch().count(); - let p_exists_payloads = p_num_payloads - .clone() - .filter(q!(|num_payloads| *num_payloads > 0)); - let p_next_slot_after_sending_payloads = p_num_payloads - .continue_if(p_exists_payloads.clone()) - .clone() - .cross_singleton(p_next_slot.clone()) - .map(q!( - |(num_payloads, next_slot)| next_slot + num_payloads as i32 - )); - let p_next_slot_if_no_payloads = p_next_slot.clone().continue_unless(p_exists_payloads); - let p_new_next_slot_calculated = 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)))) - .union(p_next_slot_if_no_payloads) - // .inspect(q!(|slot| println!("{} p_next_slot_if_no_payloads: {:?}", context.current_tick(), slot)))) - .continue_if(p_is_leader.clone()); - let p_new_next_slot_default = p_is_leader // Default next slot to 0 if there haven't been any payloads at all - .clone() - .continue_unless(p_new_next_slot_calculated.clone()) - .map(q!(|_| 0)); - // .inspect(q!(|slot| println!("{} p_new_next_slot_default: {:?}", context.current_tick(), slot))); - let p_new_next_slot = p_new_next_slot_calculated.union(p_new_next_slot_default); - p_next_slot_complete_cycle.complete_next_tick(p_new_next_slot); - (p_next_slot, p_to_acceptors_p2a) +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] +fn acceptor_p1<'a, P: PaxosPayload>( + p_to_acceptors_p1a: Stream>, + a_max_ballot: &Singleton>, + a_log: Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, + proposers: &Cluster<'a, Proposer>, +) -> Stream, Unbounded, NoTick, Cluster<'a, Proposer>> { + p_to_acceptors_p1a + .tick_batch() + .cross_singleton(a_max_ballot.clone().latest_tick()) + .cross_singleton(a_log) + .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( + p1a.ballot.proposer_id, + P1b { + ballot: p1a.ballot, + max_ballot, + accepted: log + } + ))) + .all_ticks() + .send_bincode_interleaved(proposers) } // Proposer logic for processing p1bs, determining if the proposer is now the leader, which uncommitted messages to commit, what the maximum slot is in the p1bs, and which no-ops to commit to fill log holes. @@ -675,157 +489,336 @@ fn p_p1b<'a, P: PaxosPayload>( (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) } -// Proposer logic to calculate the largest ballot received so far. -fn p_max_ballot<'a, P: PaxosPayload>( +#[expect( + clippy::type_complexity, + clippy::too_many_arguments, + reason = "internal paxos code // TODO" +)] +fn sequence_payload<'a, P: PaxosPayload, R>( proposers: &Cluster<'a, Proposer>, - a_to_proposers_p1b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, - a_to_proposers_p2b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, - p_to_proposers_i_am_leader: Stream>, -) -> Singleton> { - let p_received_p1b_ballots = a_to_proposers_p1b.clone().map(q!(|p1b| p1b.max_ballot)); - let p_received_p2b_ballots = a_to_proposers_p2b.clone().map(q!(|p2b| p2b.ballot)); - p_received_p1b_ballots - .union(p_received_p2b_ballots) - .union(p_to_proposers_i_am_leader) - .max() - .unwrap_or(proposers.singleton(q!(Ballot { - num: 0, - proposer_id: ClusterId::from_raw(0) - }))) -} + acceptors: &Cluster<'a, Acceptor>, + c_to_proposers: Stream>, + r_to_acceptors_checkpoint: impl FnOnce( + &Cluster<'a, Acceptor>, + ) -> Stream< + (ClusterId, i32), + Unbounded, + NoTick, + Cluster<'a, Acceptor>, + >, -// 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. -#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn p_ballot_calc<'a>( - proposers: &Cluster<'a, Proposer>, - p_received_max_ballot: Singleton>, + p_ballot_num: Singleton>, + p_is_leader: Optional>, + p_max_slot: Optional>, + + p_log_to_try_commit: Stream, Bounded, Tick, Cluster<'a, Proposer>>, + p_log_holes: Stream, Bounded, Tick, Cluster<'a, Proposer>>, + f: usize, + + a_max_ballot: Singleton>, ) -> ( - Singleton>, - Optional<(Ballot, u32), Bounded, Tick, Cluster<'a, Proposer>>, + Stream>, + Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>>, + Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, + Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, ) { let p_id = proposers.self_id(); - let (p_ballot_num_complete_cycle, p_ballot_num) = - proposers.tick_cycle_with_initial(proposers.singleton(q!(0)).latest_tick()); + let (p_next_slot, p_to_acceptors_p2a) = p_p2a( + proposers, + p_max_slot, + c_to_proposers, + p_ballot_num.clone(), + p_log_to_try_commit, + p_log_holes, + p_is_leader.clone(), + acceptors, + ); - let p_new_ballot_num = p_received_max_ballot - .clone() - .cross_singleton(p_ballot_num.clone()) - .map(q!(move |(received_max_ballot, ballot_num)| { - if received_max_ballot - > (Ballot { - num: ballot_num, - proposer_id: p_id, - }) - { - received_max_ballot.num + 1 - } else { - ballot_num - } - })); - p_ballot_num_complete_cycle.complete_next_tick(p_new_ballot_num); + // Tell clients that leader election has completed and they can begin sending messages + let p_to_clients_new_leader_elected = p_is_leader.clone() + .continue_unless(p_next_slot) + .cross_singleton(p_ballot_num) + .map(q!(move |(_is_leader, ballot_num)| Ballot { num: ballot_num, proposer_id: p_id})) // Only tell the clients once when leader election concludes + .all_ticks(); - let p_has_largest_ballot = p_received_max_ballot - .clone() - .cross_singleton(p_ballot_num.clone()) - .filter(q!( - move |(received_max_ballot, ballot_num)| *received_max_ballot - <= Ballot { - num: *ballot_num, - proposer_id: p_id - } - )); + // Acceptors. + acceptors + .source_iter(q!(["Acceptors say hello"])) + .for_each(q!(|s| println!("{}", s))); + let r_to_acceptors_checkpoint = r_to_acceptors_checkpoint(acceptors); - // End stable leader election - (p_ballot_num, p_has_largest_ballot) + // p_to_acceptors_p2a.clone().for_each(q!(|p2a: P2a| println!("Acceptor received P2a: {:?}", p2a))); + let (a_log, a_to_proposers_p2b) = acceptor_p2( + a_max_ballot.clone(), + p_to_acceptors_p2a, + r_to_acceptors_checkpoint, + proposers, + acceptors, + f, + ); + + // End tell clients that leader election has completed + let p_to_replicas = p_p2b(proposers, a_to_proposers_p2b.clone(), f); + + ( + p_to_clients_new_leader_elected, + p_to_replicas, + a_log, + a_to_proposers_p2b, + ) } -// Proposer logic to send "I am leader" messages periodically to other proposers, or send p1a to acceptors if other leaders expired. +#[derive(Clone)] +enum CheckpointOrP2a

{ + Checkpoint(i32), + P2a(P2a

), +} + +// Proposer logic to send p2as, outputting the next slot and the p2as to send to acceptors. #[expect( - clippy::too_many_arguments, clippy::type_complexity, + clippy::too_many_arguments, reason = "internal paxos code // TODO" )] -fn p_p1a<'a>( +fn p_p2a<'a, P: PaxosPayload>( + proposers: &Cluster<'a, Proposer>, + p_max_slot: Optional>, + c_to_proposers: Stream>, p_ballot_num: Singleton>, + p_log_to_try_commit: Stream, Bounded, Tick, Cluster<'a, Proposer>>, + p_log_holes: Stream, Bounded, Tick, Cluster<'a, Proposer>>, p_is_leader: Optional>, - proposers: &Cluster<'a, Proposer>, - p_to_proposers_i_am_leader: Stream>, acceptors: &Cluster<'a, Acceptor>, - i_am_leader_send_timeout: u64, // How often to heartbeat - i_am_leader_check_timeout: u64, // How often to check if heartbeat expired - i_am_leader_check_timeout_delay_multiplier: usize, /* Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts */ ) -> ( - Stream>, - Stream>, + Optional>, + Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, ) { let p_id = proposers.self_id(); - let p_to_proposers_i_am_leader_new = p_is_leader + let (p_next_slot_complete_cycle, p_next_slot) = + proposers.tick_cycle::>(); + let p_next_slot_after_reconciling_p1bs = p_max_slot + .unwrap_or(proposers.singleton(q!(-1)).latest_tick()) + // .inspect(q!(|max_slot| println!("{} p_max_slot: {:?}", context.current_tick(), max_slot))) + .continue_unless(p_next_slot.clone()) + .map(q!(|max_slot| max_slot + 1)); + + // Send p2as + let p_indexed_payloads = c_to_proposers .clone() - .then(p_ballot_num.clone()) - .latest() - .sample_every(q!(Duration::from_secs(i_am_leader_send_timeout))) - .map(q!(move |ballot_num| Ballot { - num: ballot_num, - proposer_id: p_id - })) - .broadcast_bincode_interleaved(proposers); + .tick_batch() + .enumerate() + .cross_singleton(p_next_slot.clone()) + // .inspect(q!(|next| println!("{} p_indexed_payloads next slot: {}", context.current_tick(), next)))) + .cross_singleton(p_ballot_num.clone()) + // .inspect(q!(|ballot_num| println!("{} p_indexed_payloads ballot_num: {}", context.current_tick(), ballot_num)))) + .map(q!(move |(((index, payload), next_slot), ballot_num)| P2a { ballot: Ballot { num: ballot_num, proposer_id: p_id }, slot: next_slot + index as i32, value: payload })); + // .inspect(q!(|p2a: &P2a| println!("{} p_indexed_payloads P2a: {:?}", context.current_tick(), p2a))); + let p_to_acceptors_p2a = p_log_to_try_commit + .union(p_log_holes) + .continue_unless(p_next_slot.clone()) // Only resend p1b stuff once. Once it's resent, next_slot will exist. + .union(p_indexed_payloads) + .continue_if(p_is_leader.clone()) + .all_ticks() + .broadcast_bincode_interleaved(acceptors); - let p_leader_expired = p_leader_expired( - p_to_proposers_i_am_leader, - p_is_leader, - i_am_leader_check_timeout, - ); + let p_num_payloads = c_to_proposers.clone().tick_batch().count(); + let p_exists_payloads = p_num_payloads + .clone() + .filter(q!(|num_payloads| *num_payloads > 0)); + let p_next_slot_after_sending_payloads = p_num_payloads + .continue_if(p_exists_payloads.clone()) + .clone() + .cross_singleton(p_next_slot.clone()) + .map(q!( + |(num_payloads, next_slot)| next_slot + num_payloads as i32 + )); + let p_next_slot_if_no_payloads = p_next_slot.clone().continue_unless(p_exists_payloads); + let p_new_next_slot_calculated = 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)))) + .union(p_next_slot_if_no_payloads) + // .inspect(q!(|slot| println!("{} p_next_slot_if_no_payloads: {:?}", context.current_tick(), slot)))) + .continue_if(p_is_leader.clone()); + let p_new_next_slot_default = p_is_leader // Default next slot to 0 if there haven't been any payloads at all + .clone() + .continue_unless(p_new_next_slot_calculated.clone()) + .map(q!(|_| 0)); + // .inspect(q!(|slot| println!("{} p_new_next_slot_default: {:?}", context.current_tick(), slot))); + let p_new_next_slot = p_new_next_slot_calculated.union(p_new_next_slot_default); + p_next_slot_complete_cycle.complete_next_tick(p_new_next_slot); + (p_next_slot, p_to_acceptors_p2a) +} - let p_id = proposers.self_id(); +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] +fn acceptor_p2<'a, P: PaxosPayload, R>( + a_max_ballot: Singleton>, + p_to_acceptors_p2a: Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, + r_to_acceptors_checkpoint: Stream< + (ClusterId, i32), + Unbounded, + NoTick, + Cluster<'a, Acceptor>, + >, + proposers: &Cluster<'a, Proposer>, + acceptors: &Cluster<'a, Acceptor>, + f: usize, +) -> ( + Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, + Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, +) { + // Get the latest checkpoint sequence per replica + let a_checkpoint_largest_seqs = + r_to_acceptors_checkpoint + .tick_prefix() + .reduce_keyed(q!(|curr_seq, seq| { + if seq > *curr_seq { + *curr_seq = seq; + } + })); + let a_checkpoints_quorum_reached = a_checkpoint_largest_seqs.clone().count().filter_map(q!( + move |num_received| if num_received == f + 1 { + Some(true) + } else { + None + } + )); + // Find the smallest checkpoint seq that everyone agrees to, track whenever it changes + let a_new_checkpoint = a_checkpoint_largest_seqs + .continue_if(a_checkpoints_quorum_reached) + .map(q!(|(_sender, seq)| seq)) + .min() + .unwrap_or(acceptors.singleton(q!(-1)).latest_tick()) + .delta() + .map(q!(|min_seq| CheckpointOrP2a::Checkpoint(min_seq))); + // .inspect(q!(|(min_seq, p2a): &(i32, P2a)| println!("Acceptor new checkpoint: {:?}", min_seq))); - // Add random delay depending on node ID so not everyone sends p1a at the same time - let p_to_acceptors_p1a = p_leader_expired - .then(p_ballot_num) - .continue_if( - proposers - .source_interval_delayed( - q!(Duration::from_secs( - (p_id.raw_id * i_am_leader_check_timeout_delay_multiplier as u32).into() - )), - q!(Duration::from_secs(i_am_leader_check_timeout)), - ) - .latest_tick(), - ) - .map(q!(move |ballot_num| P1a { - ballot: Ballot { - num: ballot_num, - proposer_id: p_id + let a_p2as_to_place_in_log = p_to_acceptors_p2a + .clone() + .tick_batch() + .cross_singleton(a_max_ballot.clone().latest_tick()) // Don't consider p2as if the current ballot is higher + .filter_map(q!(|(p2a, max_ballot)| + if p2a.ballot >= max_ballot { + Some(CheckpointOrP2a::P2a(p2a)) + } else { + None } - })) + )); + let a_log = a_p2as_to_place_in_log + .union(a_new_checkpoint.into_stream()) + .persist() + .fold( + q!(|| (-1, 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..new_checkpoint { + log.remove(&slot); + } + *prev_checkpoint = 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 + if p2a.slot > *prev_checkpoint + && 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, + }, + ); + } + } + } + }), + ); + + let a_to_proposers_p2b = p_to_acceptors_p2a + .tick_batch() + .cross_singleton(a_max_ballot.clone().latest_tick()) + .map(q!(|(p2a, max_ballot)| ( + p2a.ballot.proposer_id, + P2b { + victory: p2a.ballot == max_ballot, + ballot: p2a.ballot, + slot: p2a.slot, + value: p2a.value + } + ))) .all_ticks() - .inspect(q!(|_| println!("Proposer leader expired, sending P1a"))) - .broadcast_bincode_interleaved(acceptors); - (p_to_proposers_i_am_leader_new, p_to_acceptors_p1a) + .send_bincode_interleaved(proposers); + (a_log, a_to_proposers_p2b) } -fn p_leader_expired<'a>( - p_to_proposers_i_am_leader: Stream>, - p_is_leader: Optional>, - i_am_leader_check_timeout: u64, // How often to check if heartbeat expired -) -> Optional, Bounded, Tick, Cluster<'a, Proposer>> { - let p_latest_received_i_am_leader = p_to_proposers_i_am_leader.clone().fold( - q!(|| None), - q!(|latest, _| { - // Note: May want to check received ballot against our own? - *latest = Some(Instant::now()); - }), - ); - - p_latest_received_i_am_leader +fn p_p2b<'a, P: PaxosPayload>( + proposers: &Cluster<'a, Proposer>, + a_to_proposers_p2b: Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, + f: usize, +) -> Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>> { + let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = proposers.tick_cycle(); + let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = proposers.tick_cycle(); + let p_p2b = a_to_proposers_p2b .clone() - .latest_tick() - .continue_unless(p_is_leader) - .filter(q!(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) + .tick_batch() + .union(p_persisted_p2bs); + let p_count_matching_p2bs = p_p2b + .clone() + .filter_map(q!(|p2b| if p2b.victory { + // 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( + q!(|| (0, Default::default())), + q!(|accum, value| { + // TODO(shadaj): why is sender unused? should we de-dup? + accum.0 += 1; + accum.1 = value; + }), + ); + let p_p2b_quorum_reached = + p_count_matching_p2bs + .clone() + .filter_map(q!(move |((slot, _ballot), (count, value))| if count > f { + Some((slot, value)) } else { - true + None + })); + let p_to_replicas = p_p2b_quorum_reached + .clone() + .anti_join(p_broadcasted_p2b_slots) // Only tell the replicas about committed values once + .map(q!(|(slot, value)| (slot, value))) + .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 } 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 adbac8d5663..5614a8d634b 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 @@ -16,7 +16,7 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_3, + sym: cycle_4, }, location_kind: Cluster( 2, @@ -88,7 +88,7 @@ expression: built.ir() inner: : Union( CycleSource { ident: Ident { - sym: cycle_3, + sym: cycle_4, }, location_kind: Cluster( 2, @@ -117,119 +117,80 @@ expression: built.ir() location_kind: Cluster( 2, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: 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( - 2, - ), - from_key: None, - to_location: Cluster( - 2, - ), - 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 ()) }", - ], - }, + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: 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( + 2, ), - ), - 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 ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 2, ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > > (__hydroflow_plus_cluster_ids_2) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | Ballot { num : ballot_num , proposer_id : p_id } }), + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > > (__hydroflow_plus_cluster_ids_2) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: , - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (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 = 1usize ; move | num_received | if num_received > f { 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (p1b , ballot_num) | p1b . ballot . num == * ballot_num && p1b . ballot . proposer_id == p_id }), - input: CrossSingleton( - Persist( - Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), - input: Tee { - inner: , - }, - }, - ), - Tee { - inner: , - }, - ), - }, - }, - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: : Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , proposer_id : p_id } }), - input: CrossSingleton( - Tee { - inner: , - }, - Tee { - inner: , - }, - ), - }, - }, + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | Ballot { num : ballot_num , proposer_id : p_id } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: , + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: : CycleSource { + ident: Ident { + sym: cycle_3, }, - ), + location_kind: Cluster( + 2, + ), + }, }, }, - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Source { - source: Interval( - stageleft :: runtime_support :: type_hint :: < core :: time :: Duration > ({ use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_send_timeout = 1u64 ; Duration :: from_secs (i_am_leader_send_timeout) }), - ), - location_kind: Cluster( - 2, ), }, - }, - ), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Source { + source: Interval( + stageleft :: runtime_support :: type_hint :: < core :: time :: Duration > ({ use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_send_timeout = 1u64 ; Duration :: from_secs (i_am_leader_send_timeout) }), + ), + location_kind: Cluster( + 2, + ), + }, + }, + ), + }, }, }, }, @@ -243,191 +204,252 @@ expression: built.ir() location_kind: Cluster( 2, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 3, - ), - from_key: None, - to_location: Cluster( - 2, - ), - to_key: None, - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", - ], - }, + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + input: 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 :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 2, ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , 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 :: 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( - 2, - ), - from_key: None, - to_location: Cluster( - 3, - ), - 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 ()) }", - ], - }, + to_key: None, + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , 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 :: 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( + 2, ), - ), - 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 ()) }", - ], - }, + from_key: None, + to_location: Cluster( + 3, ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , proposer_id : p_id } } }), + 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 :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . 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: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ 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: 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 = 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) } 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( - Tee { - inner: : 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: , - }, - ), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , proposer_id : p_id } } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ 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: 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 = 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) } 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( + Tee { + inner: : 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: , + 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: , + }, }, }, }, - }, - ), + ), + }, }, }, - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Source { - source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, - ), - location_kind: Cluster( - 2, ), }, - }, - ), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Source { + source: Stream( + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 2, + ), + }, + }, + ), + }, }, }, }, }, }, }, - }, - Tee { - inner: : Union( - 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: , + Tee { + inner: : Union( + 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: , + }, }, }, - }, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, - ), - location_kind: Cluster( - 3, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), ), + }, + ), + CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 3, ), }, ), - CycleSource { - ident: Ident { - sym: cycle_0, + }, + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (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 = 1usize ; move | num_received | if num_received > f { 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ 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 < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (p1b , ballot_num) | p1b . ballot . num == * ballot_num && p1b . ballot . proposer_id == p_id }), + input: CrossSingleton( + Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), + input: Tee { + inner: , + }, + }, + ), + Tee { + inner: , + }, + ), + }, + }, }, - location_kind: Cluster( - 3, - ), }, - ), - }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: : Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , proposer_id : p_id } }), + input: CrossSingleton( + Tee { + inner: , + }, + Tee { + inner: , + }, + ), + }, + }, + }, + ), }, }, }, CycleSink { ident: Ident { - sym: cycle_4, + sym: cycle_5, }, location_kind: Cluster( 2, @@ -435,7 +457,7 @@ expression: built.ir() input: DeferTick( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Union( @@ -447,18 +469,18 @@ expression: built.ir() input: CrossSingleton( Union( Tee { - inner: : Reduce { + inner: : Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | p1b . accepted . into_iter () }), input: Tee { - inner: , + inner: , }, }, }, @@ -485,9 +507,9 @@ 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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { - sym: cycle_4, + sym: cycle_5, }, location_kind: Cluster( 2, @@ -504,15 +526,15 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , next_slot) | next_slot + num_payloads as i32 }), input: CrossSingleton( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: : Fold { + 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_bench :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -560,10 +582,10 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: : Filter { + inner: : Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | num_payloads | * num_payloads > 0 }), input: Tee { - inner: , + inner: , }, }, }, @@ -572,7 +594,7 @@ expression: built.ir() }, }, Tee { - inner: , + inner: , }, ), }, @@ -581,7 +603,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -591,7 +613,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 , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -602,7 +624,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ), @@ -614,7 +636,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -624,7 +646,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -648,7 +670,7 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_5, + sym: cycle_6, }, location_kind: Cluster( 2, @@ -658,18 +680,18 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _value) | slot }), input: Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , value)) | if count > f { Some ((slot , value)) } else { None } }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , Default :: default ()) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , value | { accum . 0 += 1 ; accum . 1 = value ; } }), input: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , core :: option :: Option < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | if p2b . victory { Some (((p2b . slot , p2b . ballot) , p2b . value)) } else { None } }), input: Tee { - inner: : Union( + inner: : Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -705,7 +727,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , P2b { victory : p2a . ballot == max_ballot , ballot : p2a . ballot , slot : p2a . slot , value : p2a . value }) }), input: CrossSingleton( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -751,7 +773,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id , } , slot , value : entry . value , }) } else { None } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { inner: , @@ -765,13 +787,13 @@ expression: built.ir() FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: ops :: Range < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | 0 .. max_slot }), input: Tee { - inner: , + inner: , }, }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: , + inner: , }, }, ), @@ -789,7 +811,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -802,11 +824,11 @@ expression: built.ir() CrossSingleton( Enumerate( Tee { - inner: , + inner: , }, ), Tee { - inner: , + inner: , }, ), Tee { @@ -818,7 +840,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ), @@ -837,7 +859,7 @@ expression: built.ir() }, CycleSource { ident: Ident { - sym: cycle_6, + sym: cycle_7, }, location_kind: Cluster( 2, @@ -852,10 +874,10 @@ expression: built.ir() }, }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , _p2b)) | if count == 2 * f + 1 { Some (slot) } else { None } }), input: Tee { - inner: , + inner: , }, }, }, @@ -864,7 +886,7 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_6, + sym: cycle_7, }, location_kind: Cluster( 2, @@ -876,11 +898,11 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | (p2b . slot , p2b) }), input: Tee { - inner: , + inner: , }, }, Tee { - inner: , + inner: , }, ), }, @@ -894,7 +916,7 @@ expression: built.ir() 3, ), input: Tee { - inner: : Fold { + inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: new ()) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { for slot in * prev_checkpoint .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = new_checkpoint ; } CheckpointOrP2a :: P2a (p2a) => { if p2a . slot > * prev_checkpoint && 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( @@ -903,7 +925,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { inner: , @@ -922,7 +944,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , ()) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: : ReduceKeyed { + inner: : ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), input: Persist( Network { @@ -978,7 +1000,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 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1013,7 +1035,7 @@ expression: built.ir() 2, ), input: Tee { - inner: , + inner: , }, }, CycleSink { @@ -1030,10 +1052,10 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), input: CrossSingleton( Tee { - inner: : Sort( + inner: : Sort( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -1073,11 +1095,11 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , value) | (slot , value) }), input: AntiJoin( Tee { - inner: , + inner: , }, CycleSource { ident: Ident { - sym: cycle_5, + sym: cycle_6, }, location_kind: Cluster( 2, @@ -1102,12 +1124,12 @@ expression: built.ir() ), }, Tee { - inner: : Fold { + inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | - 1 }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | filled_slot , (sorted_payload , highest_seq) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Union( CycleSource { @@ -1144,23 +1166,23 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | (HashMap :: < u32 , String > :: new () , - 1) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | state , payload | { let kv_store = & mut state . 0 ; let last_seq = & mut state . 1 ; kv_store . insert (payload . key , payload . value) ; debug_assert ! (payload . seq == * last_seq + 1 , "Hole in log between seq {} and {}" , * last_seq , payload . seq) ; * last_seq = payload . seq ; } }), input: Persist( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1181,7 +1203,7 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , i32) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let checkpoint_frequency = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if new_highest_seq - max_checkpointed_seq >= checkpoint_frequency as i32 { Some (new_highest_seq) } else { None } }), input: CrossSingleton( Union( @@ -1210,7 +1232,7 @@ expression: built.ir() ), ), Tee { - inner: , + inner: , }, ), }, @@ -1225,7 +1247,7 @@ expression: built.ir() 1, ), input: Tee { - inner: , + inner: , }, }, CycleSink { @@ -1238,7 +1260,7 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Union( + inner: : Union( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { @@ -1274,7 +1296,7 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1290,13 +1312,13 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 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 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { - inner: , + inner: , }, }, }, @@ -1317,7 +1339,7 @@ expression: built.ir() input: Union( Union( Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_2, }, @@ -1331,9 +1353,9 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: time :: SystemTime > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | SystemTime :: now () }), input: Tee { - inner: : Delta( + inner: : Delta( Tee { - inner: : Reduce { + inner: : 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 { @@ -1379,7 +1401,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -1389,7 +1411,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1414,10 +1436,10 @@ expression: built.ir() }, ), Tee { - inner: : Map { + 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 ()) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1430,7 +1452,7 @@ expression: built.ir() input: CrossSingleton( CrossSingleton( Tee { - inner: : Source { + inner: : Source { source: Interval( { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, ), @@ -1448,10 +1470,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: , }, ), }, @@ -1459,7 +1481,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), input: Tee { - inner: , + inner: , }, }, ), @@ -1481,7 +1503,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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, Map { @@ -1492,7 +1514,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 , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1504,7 +1526,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), input: Tee { - inner: , + inner: , }, }, ), @@ -1524,17 +1546,17 @@ expression: built.ir() FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . leader_id () , ClientPayload { key : i as u32 , value : c_id . raw_id . to_string () })) }), input: Tee { - inner: , + inner: , }, }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; move | (key , leader_ballot) | (leader_ballot . leader_id () , ClientPayload { key , value : c_id . raw_id . to_string () }) }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, From 5f0c305deb9186ef23016087bb30015316d7dae9 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 8 Oct 2024 14:30:25 -0700 Subject: [PATCH 4/5] fix(paxos): be more careful about which parts of proposer and acceptor have to be maintained atomically --- hydroflow_plus/src/location.rs | 10 ++ hydroflow_plus_test/src/cluster/paxos.rs | 87 ++++----- ...cluster__paxos_bench__tests__paxos_ir.snap | 170 +++++++++--------- 3 files changed, 141 insertions(+), 126 deletions(-) diff --git a/hydroflow_plus/src/location.rs b/hydroflow_plus/src/location.rs index 95aae37ce28..d1ea7c58fd3 100644 --- a/hydroflow_plus/src/location.rs +++ b/hydroflow_plus/src/location.rs @@ -115,6 +115,16 @@ pub trait Location<'a> { ) } + fn singleton_each_tick( + &self, + e: impl Quoted<'a, T>, + ) -> Singleton + where + Self: Sized, + { + self.singleton(e).latest_tick() + } + fn singleton_first_tick( &self, e: impl Quoted<'a, T>, diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 94a8fb5cfa7..4d60b35e683 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -160,7 +160,7 @@ fn leader_election<'a, P: PaxosPayload>( Optional>, Stream, Bounded, Tick, Cluster<'a, Proposer>>, Stream, Bounded, Tick, Cluster<'a, Proposer>>, - Singleton>, + Singleton>, ) { let (a_to_proposers_p1b_complete_cycle, a_to_proposers_p1b) = proposers.forward_ref::>(); @@ -192,17 +192,8 @@ fn leader_election<'a, P: PaxosPayload>( ); p_to_proposers_i_am_leader_complete_cycle.complete(p_to_proposers_i_am_leader_from_others); - let a_max_ballot = p_to_acceptors_p1a - .clone() - .inspect(q!(|p1a| println!("Acceptor received P1a: {:?}", p1a))) - .map(q!(|p1a| p1a.ballot)) - .max() - .unwrap_or(acceptors.singleton(q!(Ballot { - num: 0, - proposer_id: ClusterId::from_raw(0) - }))); - - let a_to_proposers_p1b = acceptor_p1(p_to_acceptors_p1a, &a_max_ballot, a_log, proposers); + let (a_max_ballot, a_to_proposers_p1b) = + acceptor_p1(acceptors, p_to_acceptors_p1a, a_log, proposers); a_to_proposers_p1b_complete_cycle.complete(a_to_proposers_p1b.clone()); let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( @@ -254,7 +245,7 @@ fn p_ballot_calc<'a>( ) { let p_id = proposers.self_id(); let (p_ballot_num_complete_cycle, p_ballot_num) = - proposers.tick_cycle_with_initial(proposers.singleton(q!(0)).latest_tick()); + proposers.tick_cycle_with_initial(proposers.singleton_each_tick(q!(0))); let p_new_ballot_num = p_received_max_ballot .clone() @@ -302,7 +293,6 @@ fn p_leader_expired<'a>( ); p_latest_received_i_am_leader - .clone() .latest_tick() .continue_unless(p_is_leader) .filter(q!(move |latest_received_i_am_leader| { @@ -376,25 +366,42 @@ fn p_p1a<'a>( #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn acceptor_p1<'a, P: PaxosPayload>( + acceptors: &Cluster<'a, Acceptor>, p_to_acceptors_p1a: Stream>, - a_max_ballot: &Singleton>, a_log: Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, proposers: &Cluster<'a, Proposer>, -) -> Stream, Unbounded, NoTick, Cluster<'a, Proposer>> { - p_to_acceptors_p1a - .tick_batch() - .cross_singleton(a_max_ballot.clone().latest_tick()) - .cross_singleton(a_log) - .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( - p1a.ballot.proposer_id, - P1b { - ballot: p1a.ballot, - max_ballot, - accepted: log - } - ))) - .all_ticks() - .send_bincode_interleaved(proposers) +) -> ( + Singleton>, + Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, +) { + let p_to_acceptors_p1a = p_to_acceptors_p1a.tick_batch(); + 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(acceptors.singleton_each_tick(q!(Ballot { + num: 0, + proposer_id: ClusterId::from_raw(0) + }))); + + ( + a_max_ballot.clone(), + p_to_acceptors_p1a + .cross_singleton(a_max_ballot) + .cross_singleton(a_log) + .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( + p1a.ballot.proposer_id, + P1b { + ballot: p1a.ballot, + max_ballot, + accepted: log + } + ))) + .all_ticks() + .send_bincode_interleaved(proposers), + ) } // Proposer logic for processing p1bs, determining if the proposer is now the leader, which uncommitted messages to commit, what the maximum slot is in the p1bs, and which no-ops to commit to fill log holes. @@ -515,7 +522,7 @@ fn sequence_payload<'a, P: PaxosPayload, R>( p_log_holes: Stream, Bounded, Tick, Cluster<'a, Proposer>>, f: usize, - a_max_ballot: Singleton>, + a_max_ballot: Singleton>, ) -> ( Stream>, Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>>, @@ -597,7 +604,7 @@ fn p_p2a<'a, P: PaxosPayload>( let (p_next_slot_complete_cycle, p_next_slot) = proposers.tick_cycle::>(); let p_next_slot_after_reconciling_p1bs = p_max_slot - .unwrap_or(proposers.singleton(q!(-1)).latest_tick()) + .unwrap_or(proposers.singleton_each_tick(q!(-1))) // .inspect(q!(|max_slot| println!("{} p_max_slot: {:?}", context.current_tick(), max_slot))) .continue_unless(p_next_slot.clone()) .map(q!(|max_slot| max_slot + 1)); @@ -652,7 +659,7 @@ fn p_p2a<'a, P: PaxosPayload>( #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn acceptor_p2<'a, P: PaxosPayload, R>( - a_max_ballot: Singleton>, + a_max_ballot: Singleton>, p_to_acceptors_p2a: Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, r_to_acceptors_checkpoint: Stream< (ClusterId, i32), @@ -667,6 +674,8 @@ fn acceptor_p2<'a, P: PaxosPayload, R>( Singleton<(i32, HashMap>), Bounded, Tick, Cluster<'a, Acceptor>>, Stream, Unbounded, NoTick, Cluster<'a, Proposer>>, ) { + let p_to_acceptors_p2a_batch = p_to_acceptors_p2a.tick_batch(); + // Get the latest checkpoint sequence per replica let a_checkpoint_largest_seqs = r_to_acceptors_checkpoint @@ -688,15 +697,14 @@ fn acceptor_p2<'a, P: PaxosPayload, R>( .continue_if(a_checkpoints_quorum_reached) .map(q!(|(_sender, seq)| seq)) .min() - .unwrap_or(acceptors.singleton(q!(-1)).latest_tick()) + .unwrap_or(acceptors.singleton_each_tick(q!(-1))) .delta() .map(q!(|min_seq| CheckpointOrP2a::Checkpoint(min_seq))); // .inspect(q!(|(min_seq, p2a): &(i32, P2a)| println!("Acceptor new checkpoint: {:?}", min_seq))); - let a_p2as_to_place_in_log = p_to_acceptors_p2a + let a_p2as_to_place_in_log = p_to_acceptors_p2a_batch .clone() - .tick_batch() - .cross_singleton(a_max_ballot.clone().latest_tick()) // Don't consider p2as if the current ballot is higher + .cross_singleton(a_max_ballot.clone()) // Don't consider p2as if the current ballot is higher .filter_map(q!(|(p2a, max_ballot)| if p2a.ballot >= max_ballot { Some(CheckpointOrP2a::P2a(p2a)) @@ -739,9 +747,8 @@ fn acceptor_p2<'a, P: PaxosPayload, R>( }), ); - let a_to_proposers_p2b = p_to_acceptors_p2a - .tick_batch() - .cross_singleton(a_max_ballot.clone().latest_tick()) + let a_to_proposers_p2b = p_to_acceptors_p2a_batch + .cross_singleton(a_max_ballot) .map(q!(|(p2a, max_ballot)| ( p2a.ballot.proposer_id, P2b { 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 5614a8d634b..3479dce1d41 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 @@ -296,16 +296,14 @@ expression: built.ir() 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( - Tee { - inner: : 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: , - }, - ), - }, + 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 | () }), @@ -346,7 +344,7 @@ expression: built.ir() }, }, Tee { - inner: : Union( + inner: : Union( 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( @@ -396,7 +394,7 @@ expression: built.ir() 2, ), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( FilterMap { @@ -405,7 +403,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 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (p1b , ballot_num) | p1b . ballot . num == * ballot_num && p1b . ballot . proposer_id == p_id }), @@ -430,7 +428,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: : Filter { + inner: : Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , proposer_id : p_id } }), input: CrossSingleton( Tee { @@ -457,7 +455,7 @@ expression: built.ir() input: DeferTick( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Union( @@ -469,18 +467,18 @@ expression: built.ir() input: CrossSingleton( Union( Tee { - inner: : Reduce { + inner: : Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | p1b . accepted . into_iter () }), input: Tee { - inner: , + inner: , }, }, }, @@ -507,7 +505,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_5, }, @@ -526,15 +524,15 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , next_slot) | next_slot + num_payloads as i32 }), input: CrossSingleton( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: : Fold { + 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_bench :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -582,10 +580,10 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: : Filter { + inner: : Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | num_payloads | * num_payloads > 0 }), input: Tee { - inner: , + inner: , }, }, }, @@ -594,7 +592,7 @@ expression: built.ir() }, }, Tee { - inner: , + inner: , }, ), }, @@ -603,7 +601,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -613,7 +611,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 , usize , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -624,7 +622,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ), @@ -636,7 +634,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -646,7 +644,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -680,18 +678,18 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _value) | slot }), input: Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , value)) | if count > f { Some ((slot , value)) } else { None } }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , Default :: default ()) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , value | { accum . 0 += 1 ; accum . 1 = value ; } }), input: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , core :: option :: Option < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | if p2b . victory { Some (((p2b . slot , p2b . ballot) , p2b . value)) } else { None } }), input: Tee { - inner: : Union( + inner: : Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -727,7 +725,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , P2b { victory : p2a . ballot == max_ballot , ballot : p2a . ballot , slot : p2a . slot , value : p2a . value }) }), input: CrossSingleton( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -773,7 +771,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , proposer_id : p_id , } , slot , value : entry . value , }) } else { None } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { inner: , @@ -787,13 +785,13 @@ expression: built.ir() FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: ops :: Range < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | 0 .. max_slot }), input: Tee { - inner: , + inner: , }, }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: , + inner: , }, }, ), @@ -811,7 +809,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -824,11 +822,11 @@ expression: built.ir() CrossSingleton( Enumerate( Tee { - inner: , + inner: , }, ), Tee { - inner: , + inner: , }, ), Tee { @@ -840,7 +838,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ), @@ -850,7 +848,7 @@ expression: built.ir() }, }, Tee { - inner: , + inner: , }, ), }, @@ -874,10 +872,10 @@ expression: built.ir() }, }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload)) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | ((slot , _ballot) , (count , _p2b)) | if count == 2 * f + 1 { Some (slot) } else { None } }), input: Tee { - inner: , + inner: , }, }, }, @@ -898,11 +896,11 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2b | (p2b . slot , p2b) }), input: Tee { - inner: , + inner: , }, }, Tee { - inner: , + inner: , }, ), }, @@ -916,7 +914,7 @@ expression: built.ir() 3, ), input: Tee { - inner: : Fold { + inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: new ()) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >) , hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { for slot in * prev_checkpoint .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = new_checkpoint ; } CheckpointOrP2a :: P2a (p2a) => { if p2a . slot > * prev_checkpoint && 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( @@ -925,10 +923,10 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: CheckpointOrP2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -944,7 +942,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , ()) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: : ReduceKeyed { + inner: : ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), input: Persist( Network { @@ -1000,7 +998,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 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1035,7 +1033,7 @@ expression: built.ir() 2, ), input: Tee { - inner: , + inner: , }, }, CycleSink { @@ -1052,10 +1050,10 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), input: CrossSingleton( Tee { - inner: : Sort( + inner: : Sort( Union( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -1095,7 +1093,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , value) | (slot , value) }), input: AntiJoin( Tee { - inner: , + inner: , }, CycleSource { ident: Ident { @@ -1124,12 +1122,12 @@ expression: built.ir() ), }, Tee { - inner: : Fold { + inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | - 1 }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | filled_slot , (sorted_payload , highest_seq) | { let next_slot = std :: cmp :: max (* filled_slot , highest_seq) ; * filled_slot = if sorted_payload . seq == next_slot + 1 { sorted_payload . seq } else { * filled_slot } ; } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Union( CycleSource { @@ -1166,23 +1164,23 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | (HashMap :: < u32 , String > :: new () , - 1) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | state , payload | { let kv_store = & mut state . 0 ; let last_seq = & mut state . 1 ; kv_store . insert (payload . key , payload . value) ; debug_assert ! (payload . seq == * last_seq + 1 , "Hole in log between seq {} and {}" , * last_seq , payload . seq) ; * last_seq = payload . seq ; } }), input: Persist( Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1203,7 +1201,7 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , i32) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let checkpoint_frequency = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if new_highest_seq - max_checkpointed_seq >= checkpoint_frequency as i32 { Some (new_highest_seq) } else { None } }), input: CrossSingleton( Union( @@ -1232,7 +1230,7 @@ expression: built.ir() ), ), Tee { - inner: , + inner: , }, ), }, @@ -1247,7 +1245,7 @@ expression: built.ir() 1, ), input: Tee { - inner: , + inner: , }, }, CycleSink { @@ -1260,7 +1258,7 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Union( + inner: : Union( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { @@ -1296,7 +1294,7 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1312,13 +1310,13 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 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 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { - inner: , + inner: , }, }, }, @@ -1339,7 +1337,7 @@ expression: built.ir() input: Union( Union( Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_2, }, @@ -1353,9 +1351,9 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: time :: SystemTime > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | SystemTime :: now () }), input: Tee { - inner: : Delta( + inner: : Delta( Tee { - inner: : Reduce { + inner: : 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 { @@ -1401,7 +1399,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: , + inner: , }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), @@ -1411,7 +1409,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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1436,10 +1434,10 @@ expression: built.ir() }, ), Tee { - inner: : Map { + 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 ()) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1452,7 +1450,7 @@ expression: built.ir() input: CrossSingleton( CrossSingleton( Tee { - inner: : Source { + inner: : Source { source: Interval( { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, ), @@ -1470,10 +1468,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: , }, ), }, @@ -1481,7 +1479,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), input: Tee { - inner: , + inner: , }, }, ), @@ -1503,7 +1501,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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, Map { @@ -1514,7 +1512,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 , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1526,7 +1524,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), input: Tee { - inner: , + inner: , }, }, ), @@ -1546,17 +1544,17 @@ expression: built.ir() FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . leader_id () , ClientPayload { key : i as u32 , value : c_id . raw_id . to_string () })) }), input: Tee { - inner: , + inner: , }, }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; move | (key , leader_ballot) | (leader_ballot . leader_id () , ClientPayload { key , value : c_id . raw_id . to_string () }) }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, From 059479d3f4468605b7d9592c4086a2994467681a Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 8 Oct 2024 14:30:25 -0700 Subject: [PATCH 5/5] refactor(paxos): minor additional cleanups --- hydroflow_plus_test/src/cluster/paxos.rs | 12 +++++----- ...cluster__paxos_bench__tests__paxos_ir.snap | 22 +++++++++---------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 4d60b35e683..83f502b17e7 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -90,13 +90,14 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( let proposers = flow.cluster::(); let acceptors = flow.cluster::(); - let c_to_proposers = c_to_proposers(&proposers); - - // Proposers. proposers .source_iter(q!(["Proposers say hello"])) .for_each(q!(|s| println!("{}", s))); + acceptors + .source_iter(q!(["Acceptors say hello"])) + .for_each(q!(|s| println!("{}", s))); + let (a_to_proposers_p2b_complete_cycle, a_to_proposers_p2b_forward_reference) = proposers.forward_ref::>(); let (a_log_complete_cycle, a_log_forward_reference) = @@ -114,6 +115,8 @@ pub fn paxos_core<'a, P: PaxosPayload, R>( a_log_forward_reference, ); + let c_to_proposers = c_to_proposers(&proposers); + let (p_to_clients_new_leader_elected, p_to_replicas, a_log, a_to_proposers_p2b) = sequence_payload( &proposers, @@ -549,9 +552,6 @@ fn sequence_payload<'a, P: PaxosPayload, R>( .all_ticks(); // Acceptors. - acceptors - .source_iter(q!(["Acceptors say hello"])) - .for_each(q!(|s| println!("{}", s))); let r_to_acceptors_checkpoint = r_to_acceptors_checkpoint(acceptors); // p_to_acceptors_p2a.clone().for_each(q!(|p2a: P2a| println!("Acceptor received P2a: {:?}", p2a))); 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 3479dce1d41..0b782243511 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 @@ -14,6 +14,17 @@ expression: built.ir() ), }, }, + ForEach { + f: stageleft :: runtime_support :: fn1_type_hint :: < & str , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | s | println ! ("{}" , s) }), + input: Source { + source: Iter( + { use crate :: __staged :: cluster :: paxos :: * ; ["Acceptors say hello"] }, + ), + location_kind: Cluster( + 3, + ), + }, + }, CycleSink { ident: Ident { sym: cycle_4, @@ -655,17 +666,6 @@ expression: built.ir() ), ), }, - ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < & str , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | s | println ! ("{}" , s) }), - input: Source { - source: Iter( - { use crate :: __staged :: cluster :: paxos :: * ; ["Acceptors say hello"] }, - ), - location_kind: Cluster( - 3, - ), - }, - }, CycleSink { ident: Ident { sym: cycle_6,