From 51a3f55e7723c8c8e4acd14e8bafb9fc6d29a2a4 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Fri, 6 Sep 2024 15:57:25 -0700 Subject: [PATCH] refactor(hydroflow_plus_test): separate Paxos core from benchmark (#1448) --- hydroflow_plus_test/examples/paxos.rs | 21 +- hydroflow_plus_test/src/cluster/mod.rs | 1 + hydroflow_plus_test/src/cluster/paxos.rs | 527 +- .../src/cluster/paxos_bench.rs | 470 + ...luster__paxos_bench__tests__paxos_ir.snap} | 10123 ++++++++-------- 5 files changed, 5822 insertions(+), 5320 deletions(-) create mode 100644 hydroflow_plus_test/src/cluster/paxos_bench.rs rename hydroflow_plus_test/src/cluster/snapshots/{hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap => hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap} (79%) diff --git a/hydroflow_plus_test/examples/paxos.rs b/hydroflow_plus_test/examples/paxos.rs index afa83d4b221..cfd326b6b7d 100644 --- a/hydroflow_plus_test/examples/paxos.rs +++ b/hydroflow_plus_test/examples/paxos.rs @@ -16,16 +16,17 @@ async fn main() { let i_am_leader_check_timeout = 10; // Sec let i_am_leader_check_timeout_delay_multiplier = 15; - let (proposers, acceptors, clients, replicas) = hydroflow_plus_test::cluster::paxos::paxos( - &builder, - f, - num_clients_per_node, - median_latency_window_size, - checkpoint_frequency, - i_am_leader_send_timeout, - i_am_leader_check_timeout, - i_am_leader_check_timeout_delay_multiplier, - ); + let (proposers, acceptors, clients, replicas) = + hydroflow_plus_test::cluster::paxos_bench::paxos_bench( + &builder, + f, + num_clients_per_node, + median_latency_window_size, + checkpoint_frequency, + i_am_leader_send_timeout, + i_am_leader_check_timeout, + i_am_leader_check_timeout_delay_multiplier, + ); let rustflags = "-C opt-level=3 -C codegen-units=1 -C strip=none -C debuginfo=2 -C lto=off"; diff --git a/hydroflow_plus_test/src/cluster/mod.rs b/hydroflow_plus_test/src/cluster/mod.rs index ab7e8d3bb25..1e3bc5374a7 100644 --- a/hydroflow_plus_test/src/cluster/mod.rs +++ b/hydroflow_plus_test/src/cluster/mod.rs @@ -2,4 +2,5 @@ pub mod compute_pi; pub mod many_to_many; pub mod map_reduce; pub mod paxos; +pub mod paxos_bench; pub mod simple_cluster; diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index 9deb18345c1..a3b94407407 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -1,34 +1,20 @@ -use std::cell::RefCell; use std::collections::HashMap; -use std::rc::Rc; -use std::time::{Duration, SystemTime}; +use std::time::Duration; use hydroflow_plus::*; +use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; use stageleft::*; use tokio::time::Instant; +use super::paxos_bench::LeaderElected; + pub struct Proposer {} pub struct Acceptor {} -pub struct Client {} -pub struct Replica {} -#[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug)] -struct ClientPayload { - key: u32, - value: String, -} - -#[derive(Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord, Clone, Debug)] -struct ReplicaPayload { - // Note: Important that seq is the first member of the struct for sorting - seq: i32, - key: u32, - value: String, -} - -pub trait Address { - fn get_id(&self) -> u32; +pub trait PaxosPayload: + Serialize + DeserializeOwned + PartialEq + Eq + Default + Clone + std::fmt::Debug +{ } #[derive(Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Hash)] @@ -38,8 +24,8 @@ pub struct Ballot { pub id: u32, } -impl Address for Ballot { - fn get_id(&self) -> u32 { +impl LeaderElected for Ballot { + fn leader_id(&self) -> u32 { self.id } } @@ -50,69 +36,62 @@ struct P1a { } #[derive(Serialize, Deserialize, Clone, Debug)] -struct LogValue { +struct LogValue

{ ballot: Ballot, - value: ClientPayload, + value: P, } #[derive(Serialize, Deserialize, Clone, Debug)] -struct P1b { +struct P1b

{ ballot: Ballot, max_ballot: Ballot, - accepted: HashMap, + accepted: HashMap>, } #[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug)] -struct P2a { +struct P2a

{ ballot: Ballot, slot: i32, - value: ClientPayload, + value: P, } #[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug)] -struct P2b { +struct P2b

{ ballot: Ballot, max_ballot: Ballot, slot: i32, - value: ClientPayload, + value: P, } -// Important: By convention, all relations that represent booleans either have a single "true" value or nothing. -// This allows us to use the continue_if_exists() and continue_if_empty() operators as if they were if (true) and if (false) statements. -#[expect(clippy::too_many_arguments, reason = "internal paxos code // TODO")] -pub fn paxos( - flow: &FlowBuilder<'_>, +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] +pub fn paxos_core<'a, P: PaxosPayload>( + flow: &FlowBuilder<'a>, + r_to_acceptors_checkpoint: impl FnOnce( + &Cluster, + ) -> Stream< + 'a, + (u32, i32), + Unbounded, + NoTick, + Cluster, + >, + c_to_proposers: impl FnOnce( + &Cluster, + ) -> Stream<'a, P, Unbounded, NoTick, Cluster>, f: usize, - num_clients_per_node: usize, - median_latency_window_size: usize, /* How many latencies to keep in the window for calculating the median */ - checkpoint_frequency: usize, // How many sequence numbers to commit before checkpointing - i_am_leader_send_timeout: u64, // How often to heartbeat - i_am_leader_check_timeout: u64, // How often to check if heartbeat expired - i_am_leader_check_timeout_delay_multiplier: usize, /* Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts */ + i_am_leader_send_timeout: u64, + i_am_leader_check_timeout: u64, + i_am_leader_check_timeout_delay_multiplier: usize, ) -> ( Cluster, Cluster, - Cluster, - Cluster, + Stream<'a, Ballot, Unbounded, NoTick, Cluster>, + Stream<'a, (i32, P), Unbounded, NoTick, Cluster>, ) { let proposers = flow.cluster::(); let acceptors = flow.cluster::(); - let clients = flow.cluster::(); - let replicas = flow.cluster::(); - let (r_to_clients_payload_applied_cycle, r_to_clients_payload_applied) = flow.cycle(&clients); - let (p_to_clients_leader_elected_cycle, p_to_clients_leader_elected) = flow.cycle(&clients); - - let c_to_proposers = client( - &clients, - p_to_clients_leader_elected, - r_to_clients_payload_applied, - flow, - num_clients_per_node, - median_latency_window_size, - f, - ) - .send_bincode_interleaved(&proposers); + let c_to_proposers = c_to_proposers(&proposers); // Proposers. flow.source_iter(&proposers, q!(["Proposers say hello"])) @@ -120,14 +99,11 @@ pub fn paxos( let p_id = flow.cluster_self_id(&proposers); let (p_to_proposers_i_am_leader_complete_cycle, p_to_proposers_i_am_leader) = - flow.cycle::>>(&proposers); + flow.cycle::>(&proposers); let (a_to_proposers_p1b_complete_cycle, a_to_proposers_p1b) = - flow.cycle::>(&proposers); - a_to_proposers_p1b - .clone() - .for_each(q!(|(_, p1b)| println!("Proposer received P1b: {:?}", p1b))); + flow.cycle::>(&proposers); let (a_to_proposers_p2b_complete_cycle, a_to_proposers_p2b) = - flow.cycle::>(&proposers); + flow.cycle::>(&proposers); // 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))); @@ -145,7 +121,7 @@ pub fn paxos( let (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) = p_p1b( flow, &proposers, - a_to_proposers_p1b, + a_to_proposers_p1b.inspect(q!(|(_, p1b)| println!("Proposer received P1b: {:?}", p1b))), p_ballot_num.clone(), p_has_largest_ballot, f, @@ -181,24 +157,19 @@ pub fn paxos( .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 - .all_ticks() - .broadcast_bincode_interleaved(&clients); - p_to_clients_leader_elected_cycle.complete(p_to_clients_new_leader_elected); + .all_ticks(); // End tell clients that leader election has completed - let p_to_replicas = p_p2b(flow, &proposers, a_to_proposers_p2b, &replicas, f); + let p_to_replicas = p_p2b(flow, &proposers, a_to_proposers_p2b, f); // Acceptors. flow.source_iter(&acceptors, q!(["Acceptors say hello"])) .for_each(q!(|s| println!("{}", s))); - let (r_to_acceptors_checkpoint_complete_cycle, r_to_acceptors_checkpoint) = - flow.cycle(&acceptors); - p_to_acceptors_p1a - .clone() - .for_each(q!(|p1a| println!("Acceptor received P1a: {:?}", p1a))); + 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( flow, - p_to_acceptors_p1a, + p_to_acceptors_p1a.inspect(q!(|p1a| println!("Acceptor received P1a: {:?}", p1a))), p_to_acceptors_p2a, r_to_acceptors_checkpoint, &proposers, @@ -208,28 +179,26 @@ pub fn paxos( a_to_proposers_p1b_complete_cycle.complete(a_to_proposers_p1b_new); a_to_proposers_p2b_complete_cycle.complete(a_to_proposers_p2b_new); - // Replicas. - let (r_to_acceptors_checkpoint_new, r_new_processed_payloads) = - replica(flow, &replicas, p_to_replicas, checkpoint_frequency); - r_to_clients_payload_applied_cycle.complete(r_new_processed_payloads.send_bincode(&clients)); - r_to_acceptors_checkpoint_complete_cycle - .complete(r_to_acceptors_checkpoint_new.broadcast_bincode(&acceptors)); - - (proposers, acceptors, clients, replicas) + ( + proposers, + acceptors, + p_to_clients_new_leader_elected, + p_to_replicas, + ) } #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn acceptor<'a>( +fn acceptor<'a, P: PaxosPayload>( flow: &FlowBuilder<'a>, p_to_acceptors_p1a: Stream<'a, P1a, Unbounded, NoTick, Cluster>, - p_to_acceptors_p2a: Stream<'a, P2a, Unbounded, NoTick, Cluster>, + p_to_acceptors_p2a: Stream<'a, P2a

, Unbounded, NoTick, Cluster>, r_to_acceptors_checkpoint: Stream<'a, (u32, i32), Unbounded, NoTick, Cluster>, proposers: &Cluster, acceptors: &Cluster, f: usize, ) -> ( - Stream<'a, (u32, P1b), Unbounded, NoTick, Cluster>, - Stream<'a, (u32, P2b), Unbounded, NoTick, Cluster>, + Stream<'a, (u32, P1b

), Unbounded, NoTick, Cluster>, + Stream<'a, (u32, P2b

), Unbounded, NoTick, Cluster>, ) { // Get the latest checkpoint sequence per replica let a_checkpoint_largest_seqs = @@ -260,10 +229,7 @@ fn acceptor<'a>( // Create tuple with checkpoint number and dummy p2a ballot: Ballot { num: 0, id: 0 }, slot: -1, - value: ClientPayload { - key: 0, - value: "".to_string(), - } + value: Default::default() } ))); // .inspect(q!(|(min_seq, p2a): &(i32, P2a)| println!("Acceptor new checkpoint: {:?}", min_seq))); @@ -286,9 +252,9 @@ fn acceptor<'a>( )); let a_log = a_p2as_to_place_in_log .union(a_new_checkpoint.into_stream()) - .all_ticks() + .persist() .fold( - q!(|| (-1, HashMap::::new())), + 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 @@ -329,7 +295,7 @@ fn acceptor<'a>( let a_to_proposers_p1b_new = p_to_acceptors_p1a .tick_batch() .cross_singleton(a_max_ballot.clone().latest_tick()) - .cross_singleton(a_log.latest_tick()) + .cross_singleton(a_log) .map(q!(|((p1a, max_ballot), (_prev_checkpoint, log))| ( p1a.ballot.id, P1b { @@ -357,13 +323,12 @@ fn acceptor<'a>( (a_to_proposers_p1b_new, a_to_proposers_p2b_new) } -fn p_p2b<'a>( +fn p_p2b<'a, P: PaxosPayload>( flow: &FlowBuilder<'a>, proposers: &Cluster, - a_to_proposers_p2b: Stream<'a, (u32, P2b), Unbounded, NoTick, Cluster>, - replicas: &Cluster, + a_to_proposers_p2b: Stream<'a, (u32, P2b

), Unbounded, NoTick, Cluster>, f: usize, -) -> Stream<'a, ReplicaPayload, Unbounded, NoTick, Cluster> { +) -> Stream<'a, (i32, P), Unbounded, NoTick, Cluster> { let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = flow.tick_cycle(proposers); let (p_persisted_p2bs_complete_cycle, p_persisted_p2bs) = flow.tick_cycle(proposers); @@ -386,10 +351,7 @@ fn p_p2b<'a>( ballot: Ballot { num: 0, id: 0 }, max_ballot: Ballot { num: 0, id: 0 }, slot: 0, - value: ClientPayload { - key: 0, - value: "0".to_string() - } + value: Default::default() } )), q!(|accum, (_sender, p2b)| { @@ -403,9 +365,8 @@ fn p_p2b<'a>( 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))| ReplicaPayload { seq: p2b.slot, key: p2b.value.key, value: p2b.value.value })) - .all_ticks() - .broadcast_bincode_interleaved(replicas); + .map(q!(|(_slot, (_count, p2b))| (p2b.slot, p2b.value))) + .all_ticks(); let p_p2b_all_commit_slots = p_count_matching_p2bs @@ -438,19 +399,19 @@ fn p_p2b<'a>( clippy::too_many_arguments, reason = "internal paxos code // TODO" )] -fn p_p2a<'a>( +fn p_p2a<'a, P: PaxosPayload>( flow: &FlowBuilder<'a>, proposers: &Cluster, p_max_slot: Optional<'a, i32, Bounded, Tick, Cluster>, - c_to_proposers: Stream<'a, ClientPayload, Unbounded, NoTick, Cluster>, + c_to_proposers: Stream<'a, P, Unbounded, NoTick, Cluster>, p_ballot_num: Singleton<'a, u32, Bounded, Tick, Cluster>, - p_log_to_try_commit: Stream<'a, P2a, Bounded, Tick, Cluster>, - p_log_holes: Stream<'a, P2a, Bounded, Tick, Cluster>, + p_log_to_try_commit: Stream<'a, P2a

, Bounded, Tick, Cluster>, + p_log_holes: Stream<'a, P2a

, Bounded, Tick, Cluster>, p_is_leader: Optional<'a, bool, Bounded, Tick, Cluster>, acceptors: &Cluster, ) -> ( Optional<'a, i32, Bounded, Tick, Cluster>, - Stream<'a, P2a, Unbounded, NoTick, Cluster>, + Stream<'a, P2a

, Unbounded, NoTick, Cluster>, ) { let p_id = flow.cluster_self_id(proposers); let (p_next_slot_complete_cycle, p_next_slot) = @@ -511,18 +472,18 @@ fn p_p2a<'a>( // 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. #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn p_p1b<'a>( +fn p_p1b<'a, P: PaxosPayload>( flow: &FlowBuilder<'a>, proposers: &Cluster, - a_to_proposers_p1b: Stream<'a, (u32, P1b), Unbounded, NoTick, Cluster>, + a_to_proposers_p1b: Stream<'a, (u32, P1b

), Unbounded, NoTick, Cluster>, p_ballot_num: Singleton<'a, u32, Bounded, Tick, Cluster>, p_has_largest_ballot: Optional<'a, (Ballot, u32), Bounded, Tick, Cluster>, f: usize, ) -> ( Optional<'a, bool, Bounded, Tick, Cluster>, - Stream<'a, P2a, Bounded, Tick, Cluster>, + Stream<'a, P2a

, Bounded, Tick, Cluster>, Optional<'a, i32, Bounded, Tick, Cluster>, - Stream<'a, P2a, Bounded, Tick, Cluster>, + Stream<'a, P2a

, Bounded, Tick, Cluster>, ) { let p_id = flow.cluster_self_id(proposers); let p_relevant_p1bs = a_to_proposers_p1b @@ -549,7 +510,7 @@ fn p_p1b<'a>( 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: 0 }, value: ClientPayload { key: 0, value: "".to_string() } })), q!(|curr_entry, new_entry| { + .fold_keyed(q!(|| (0, LogValue { ballot: Ballot { num: 0, id: 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 @@ -601,326 +562,17 @@ fn p_p1b<'a>( id: p_id }, slot, - value: ClientPayload { - key: 0, - value: "0".to_string() - } + value: Default::default() })); (p_is_leader, p_log_to_try_commit, p_max_slot, p_log_holes) } -// Replicas. All relations for replicas will be prefixed with r. Expects ReplicaPayload on p_to_replicas, outputs a stream of (client address, ReplicaPayload) after processing. -#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn replica<'a>( - flow: &FlowBuilder<'a>, - replicas: &Cluster, - p_to_replicas: Stream<'a, ReplicaPayload, Unbounded, NoTick, Cluster>, - checkpoint_frequency: usize, -) -> ( - Stream<'a, i32, Unbounded, NoTick, Cluster>, - Stream<'a, (u32, ReplicaPayload), Unbounded, NoTick, Cluster>, -) { - let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = flow.tick_cycle(replicas); - // p_to_replicas.inspect(q!(|payload: ReplicaPayload| println!("Replica received payload: {:?}", payload))); - let r_sorted_payloads = p_to_replicas - .clone() - .tick_batch() - .union(r_buffered_payloads) // Combine with all payloads that we've received and not processed yet - .sort(); - // Create a cycle since we'll use this seq before we define it - let (r_highest_seq_complete_cycle, r_highest_seq) = - flow.tick_cycle::>(replicas); - let empty_slot = flow.singleton_first_tick(replicas, q!(-1)); - // Either the max sequence number executed so far or -1. Need to union otherwise r_highest_seq is empty and joins with it will fail - let r_highest_seq_with_default = r_highest_seq.union(empty_slot); - // Find highest the sequence number of any payload that can be processed in this tick. This is the payload right before a hole. - let r_highest_seq_processable_payload = r_sorted_payloads - .clone() - .cross_singleton(r_highest_seq_with_default) - .fold( - q!(|| -1), - q!(|filled_slot, (sorted_payload, highest_seq)| { - // Note: This function only works if the input is sorted on 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 - }; - }), - ); - // Find all payloads that can and cannot be processed in this tick. - let r_processable_payloads = r_sorted_payloads - .clone() - .cross_singleton(r_highest_seq_processable_payload.clone()) - .filter(q!( - |(sorted_payload, highest_seq)| sorted_payload.seq <= *highest_seq - )) - .map(q!(|(sorted_payload, _)| { sorted_payload })); - let r_new_non_processable_payloads = r_sorted_payloads - .clone() - .cross_singleton(r_highest_seq_processable_payload.clone()) - .filter(q!( - |(sorted_payload, highest_seq)| sorted_payload.seq > *highest_seq - )) - .map(q!(|(sorted_payload, _)| { sorted_payload })); - // Save these, we can process them once the hole has been filled - r_buffered_payloads_complete_cycle.complete_next_tick(r_new_non_processable_payloads); - - let r_kv_store = r_processable_payloads - .clone() - .persist() // Optimization: all_ticks() + fold() = fold, where the state of the previous fold is saved and persisted values are deleted. - .fold(q!(|| (HashMap::::new(), -1)), q!(|state, 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; - // println!("Replica kv store: {:?}", kv_store); - })); - // Update the highest seq for the next tick - let r_new_highest_seq = r_kv_store.map(q!(|(_kv_store, highest_seq)| highest_seq)); - r_highest_seq_complete_cycle.complete_next_tick(r_new_highest_seq.clone().into()); - - // Send checkpoints to the acceptors when we've processed enough payloads - let (r_checkpointed_seqs_complete_cycle, r_checkpointed_seqs) = - flow.tick_cycle::>(replicas); - let r_max_checkpointed_seq = r_checkpointed_seqs - .persist() - .max() - .unwrap_or(flow.singleton(replicas, q!(-1)).latest_tick()); - let r_checkpoint_seq_new = r_max_checkpointed_seq - .cross_singleton(r_new_highest_seq) - .filter_map(q!( - move |(max_checkpointed_seq, new_highest_seq)| if new_highest_seq - max_checkpointed_seq - >= checkpoint_frequency as i32 - { - Some(new_highest_seq) - } else { - None - } - )); - r_checkpointed_seqs_complete_cycle.complete_next_tick(r_checkpoint_seq_new.clone()); - - // Tell clients that the payload has been committed. All ReplicaPayloads contain the client's machine ID (to string) as value. - let r_to_clients = p_to_replicas.map(q!(|payload| ( - payload.value.parse::().unwrap(), - payload - ))); - (r_checkpoint_seq_new.all_ticks(), r_to_clients) -} - -// Clients. All relations for clients will be prefixed with c. All ClientPayloads will contain the virtual client number as key and the client's machine ID (to string) as value. Expects p_to_clients_leader_elected containing Ballots whenever the leader is elected, and r_to_clients_payload_applied containing ReplicaPayloads whenever a payload is committed. Outputs (leader address, ClientPayload) when a new leader is elected or when the previous payload is committed. -fn client<'a, B: Address + Ord + std::fmt::Debug + Clone>( - clients: &Cluster, - p_to_clients_leader_elected: Stream<'a, B, Unbounded, NoTick, Cluster>, - r_to_clients_payload_applied: Stream< - 'a, - (u32, ReplicaPayload), - Unbounded, - NoTick, - Cluster, - >, - flow: &FlowBuilder<'a>, - num_clients_per_node: usize, - median_latency_window_size: usize, - f: usize, -) -> Stream<'a, (u32, ClientPayload), Unbounded, NoTick, Cluster> { - let c_id = flow.cluster_self_id(clients); - p_to_clients_leader_elected - .clone() - .for_each(q!(|ballot| println!( - "Client notified that leader was elected: {:?}", - ballot - ))); - // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); - // Only keep the latest leader - let c_max_leader_ballot = p_to_clients_leader_elected.max(); - let c_new_leader_ballot = c_max_leader_ballot.clone().latest_tick().delta(); - // Whenever the leader changes, make all clients send a message - let c_new_payloads_when_leader_elected = - c_new_leader_ballot - .clone() - .flat_map(q!(move |leader_ballot| (0..num_clients_per_node).map( - move |i| ( - leader_ballot.get_id(), - ClientPayload { - key: i as u32, - value: c_id.to_string() - } - ) - ))); - // Whenever replicas confirm that a payload was committed, collected it and wait for a quorum - let (c_pending_quorum_payloads_complete_cycle, c_pending_quorum_payloads) = - flow.tick_cycle(clients); - let c_received_payloads = r_to_clients_payload_applied - .tick_batch() - .map(q!(|(sender, replica_payload)| ( - replica_payload.key, - sender - ))) - .union(c_pending_quorum_payloads); - let c_received_quorum_payloads = c_received_payloads - .clone() - .fold_keyed( - q!(|| 0), - q!(|curr_count, _sender| { - *curr_count += 1; // Assumes the same replica will only send commit once - }), - ) - .filter_map(q!(move |(key, count)| { - if count == f + 1 { - Some(key) - } else { - None - } - })); - let c_new_pending_quorum_payloads = - c_received_payloads.anti_join(c_received_quorum_payloads.clone()); - c_pending_quorum_payloads_complete_cycle.complete_next_tick(c_new_pending_quorum_payloads); - // Whenever all replicas confirm that a payload was committed, send another payload - let c_new_payloads_when_committed = c_received_quorum_payloads - .clone() - .cross_singleton(c_max_leader_ballot.clone().latest_tick()) - .map(q!(move |(key, leader_ballot)| ( - leader_ballot.get_id(), - ClientPayload { - key, - value: c_id.to_string() - } - ))); - let c_to_proposers = c_new_payloads_when_leader_elected - .union(c_new_payloads_when_committed) - .all_ticks(); - - // Track statistics - let (c_timers_complete_cycle, c_timers) = - flow.tick_cycle::>(clients); - let c_new_timers_when_leader_elected = c_new_leader_ballot - .map(q!(|_| SystemTime::now())) - .flat_map(q!( - move |now| (0..num_clients_per_node).map(move |virtual_id| (virtual_id, now)) - )); - let c_updated_timers = c_received_quorum_payloads - .clone() - .map(q!(|key| (key as usize, SystemTime::now()))); - let c_new_timers = c_timers - .clone() // Update c_timers in tick+1 so we can record differences during this tick (to track latency) - .union(c_new_timers_when_leader_elected) - .union(c_updated_timers.clone()) - .reduce_keyed(q!(|curr_time, new_time| { - if new_time > *curr_time { - *curr_time = new_time; - } - })); - c_timers_complete_cycle.complete_next_tick(c_new_timers); - - let c_stats_output_timer = flow.source_interval(clients, q!(Duration::from_secs(1))); - - let c_latency_reset = c_stats_output_timer - .clone() - .latest_tick() - .map(q!(|_| None)) - .defer_tick(); - - let c_latencies = c_timers - .join(c_updated_timers) - .map(q!(|(_virtual_id, (prev_time, curr_time))| Some( - curr_time.duration_since(prev_time).unwrap().as_micros() - ))) - .union(c_latency_reset.into_stream()) - .all_ticks() - .fold( - // Create window with ring buffer using vec + wraparound index - // TODO: Would be nice if I could use vec![] instead, but that doesn't work in HF+ with RuntimeData *median_latency_window_size - q!(move || ( - Rc::new(RefCell::new(Vec::::with_capacity( - median_latency_window_size - ))), - 0usize, - false - )), - q!(move |(latencies, write_index, has_any_value), latency| { - let mut latencies_mut = latencies.borrow_mut(); - if let Some(latency) = latency { - // Insert into latencies - if let Some(prev_latency) = latencies_mut.get_mut(*write_index) { - *prev_latency = latency; - } else { - latencies_mut.push(latency); - } - *has_any_value = true; - // Increment write index and wrap around - *write_index += 1; - if *write_index == median_latency_window_size { - *write_index = 0; - } - } else { - // reset latencies - latencies_mut.clear(); - *write_index = 0; - *has_any_value = false; - } - }), - ); - let c_throughput_new_batch = c_received_quorum_payloads - .clone() - .count() - .continue_unless(c_stats_output_timer.clone().latest_tick()) - .map(q!(|batch_size| (batch_size, false))); - - let c_throughput_reset = c_stats_output_timer - .clone() - .latest_tick() - .map(q!(|_| (0, true))) - .defer_tick(); - - let c_throughput = c_throughput_new_batch - .union(c_throughput_reset) - .all_ticks() - .fold( - q!(|| (0, 0)), - q!(|(total, num_ticks), (batch_size, reset)| { - if reset { - *total = 0; - *num_ticks = 0; - } else { - *total += batch_size as u32; - *num_ticks += 1; - } - }), - ); - c_stats_output_timer - .cross_singleton(c_latencies) - .cross_singleton(c_throughput) - .tick_samples() - .for_each(q!(move |( - (_, (latencies, _write_index, has_any_value)), - (throughput, num_ticks), - )| { - let mut latencies_mut = latencies.borrow_mut(); - let median_latency = if has_any_value { - let (_, median, _) = - latencies_mut.select_nth_unstable(median_latency_window_size / 2); - *median - } else { - 0 - }; - println!("Median latency: {}ms", median_latency as f64 / 1000.0); - println!("Throughput: {} requests/s", throughput); - println!("Num ticks per second: {}", num_ticks); - })); - // End track statistics - c_to_proposers -} - // Proposer logic to calculate the largest ballot received so far. -fn p_max_ballot<'a>( +fn p_max_ballot<'a, P: PaxosPayload>( flow: &FlowBuilder<'a>, proposers: &Cluster, - a_to_proposers_p1b: Stream<'a, (u32, P1b), Unbounded, NoTick, Cluster>, - a_to_proposers_p2b: Stream<'a, (u32, P2b), Unbounded, NoTick, Cluster>, + a_to_proposers_p1b: Stream<'a, (u32, P1b

), Unbounded, NoTick, Cluster>, + a_to_proposers_p2b: Stream<'a, (u32, P2b

), Unbounded, NoTick, Cluster>, p_to_proposers_i_am_leader: Stream<'a, Ballot, Unbounded, NoTick, Cluster>, ) -> Singleton<'a, Ballot, Unbounded, NoTick, Cluster> { let p_received_p1b_ballots = a_to_proposers_p1b @@ -1055,20 +707,3 @@ fn p_p1a<'a>( .broadcast_bincode_interleaved(acceptors); (p_to_proposers_i_am_leader_new, p_to_acceptors_p1a) } - -#[cfg(test)] -mod tests { - use hydroflow_plus_deploy::DeployRuntime; - use stageleft::RuntimeData; - - #[test] - fn paxos_ir() { - let builder = hydroflow_plus::FlowBuilder::new(); - let _ = super::paxos(&builder, 1, 1, 1, 1, 1, 1, 1); - let built = builder.with_default_optimize(); - - insta::assert_debug_snapshot!(built.ir()); - - let _ = built.compile::(&RuntimeData::new("FAKE")); - } -} diff --git a/hydroflow_plus_test/src/cluster/paxos_bench.rs b/hydroflow_plus_test/src/cluster/paxos_bench.rs new file mode 100644 index 00000000000..c53938d9406 --- /dev/null +++ b/hydroflow_plus_test/src/cluster/paxos_bench.rs @@ -0,0 +1,470 @@ +use std::cell::RefCell; +use std::collections::HashMap; +use std::rc::Rc; +use std::time::{Duration, SystemTime}; + +use hydroflow_plus::*; +use serde::{Deserialize, Serialize}; +use stageleft::*; + +use super::paxos::{paxos_core, Acceptor, Ballot, PaxosPayload, Proposer}; + +pub trait LeaderElected: Ord + Clone { + fn leader_id(&self) -> u32; +} + +pub struct Replica {} + +pub struct Client {} + +#[derive(Serialize, serde::Deserialize, PartialEq, Eq, PartialOrd, Ord, Clone, Debug)] +pub struct ReplicaPayload { + // Note: Important that seq is the first member of the struct for sorting + pub seq: i32, + pub key: u32, + pub value: String, +} + +#[derive(Serialize, Deserialize, PartialEq, Eq, Clone, Debug)] +pub struct ClientPayload { + pub key: u32, + pub value: String, +} + +impl Default for ClientPayload { + fn default() -> Self { + Self { + key: 0, + value: "".to_string(), + } + } +} + +impl PaxosPayload for ClientPayload {} + +// Important: By convention, all relations that represent booleans either have a single "true" value or nothing. +// This allows us to use the continue_if_exists() and continue_if_empty() operators as if they were if (true) and if (false) statements. +#[expect(clippy::too_many_arguments, reason = "internal paxos code // TODO")] +pub fn paxos_bench( + flow: &FlowBuilder, + f: usize, + num_clients_per_node: usize, + median_latency_window_size: usize, /* How many latencies to keep in the window for calculating the median */ + checkpoint_frequency: usize, // How many sequence numbers to commit before checkpointing + i_am_leader_send_timeout: u64, // How often to heartbeat + i_am_leader_check_timeout: u64, // How often to check if heartbeat expired + i_am_leader_check_timeout_delay_multiplier: usize, /* Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts */ +) -> ( + Cluster, + Cluster, + Cluster, + Cluster, +) { + let clients = flow.cluster::(); + let replicas = flow.cluster::(); + + let (c_to_proposers_complete_cycle, c_to_proposers) = flow.cycle(&clients); + + let (proposers, acceptors, p_to_clients_new_leader_elected, r_new_processed_payloads) = + paxos_with_replica( + flow, + &replicas, + c_to_proposers, + f, + i_am_leader_send_timeout, + i_am_leader_check_timeout, + i_am_leader_check_timeout_delay_multiplier, + checkpoint_frequency, + ); + + c_to_proposers_complete_cycle.complete(bench_client( + &clients, + p_to_clients_new_leader_elected.broadcast_bincode_interleaved(&clients), + r_new_processed_payloads.send_bincode(&clients), + flow, + num_clients_per_node, + median_latency_window_size, + f, + )); + + (proposers, acceptors, clients, replicas) +} + +#[expect( + clippy::type_complexity, + clippy::too_many_arguments, + reason = "internal paxos code // TODO" +)] +fn paxos_with_replica<'a>( + flow: &FlowBuilder<'a>, + replicas: &Cluster, + c_to_proposers: Stream<'a, (u32, ClientPayload), Unbounded, NoTick, Cluster>, + f: usize, + i_am_leader_send_timeout: u64, + i_am_leader_check_timeout: u64, + i_am_leader_check_timeout_delay_multiplier: usize, + checkpoint_frequency: usize, +) -> ( + Cluster, + Cluster, + Stream<'a, Ballot, Unbounded, NoTick, Cluster>, + Stream<'a, (u32, ReplicaPayload), Unbounded, NoTick, Cluster>, +) { + let (r_to_acceptors_checkpoint_complete_cycle, r_to_acceptors_checkpoint) = + flow.cycle::>(replicas); + + let (proposers, acceptors, p_to_clients_new_leader_elected, p_to_replicas) = paxos_core( + flow, + |acceptors| r_to_acceptors_checkpoint.broadcast_bincode(acceptors), + |proposers| c_to_proposers.send_bincode_interleaved(proposers), + f, + i_am_leader_send_timeout, + i_am_leader_check_timeout, + i_am_leader_check_timeout_delay_multiplier, + ); + + let (r_to_acceptors_checkpoint_new, r_new_processed_payloads) = replica( + flow, + replicas, + p_to_replicas + .map(q!(|(slot, data)| ReplicaPayload { + seq: slot, + key: data.key, + value: data.value + })) + .broadcast_bincode_interleaved(replicas), + checkpoint_frequency, + ); + + r_to_acceptors_checkpoint_complete_cycle.complete(r_to_acceptors_checkpoint_new); + + ( + proposers, + acceptors, + p_to_clients_new_leader_elected, + r_new_processed_payloads, + ) +} + +// Replicas. All relations for replicas will be prefixed with r. Expects ReplicaPayload on p_to_replicas, outputs a stream of (client address, ReplicaPayload) after processing. +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] +pub fn replica<'a>( + flow: &FlowBuilder<'a>, + replicas: &Cluster, + p_to_replicas: Stream<'a, ReplicaPayload, Unbounded, NoTick, Cluster>, + checkpoint_frequency: usize, +) -> ( + Stream<'a, i32, Unbounded, NoTick, Cluster>, + Stream<'a, (u32, ReplicaPayload), Unbounded, NoTick, Cluster>, +) { + let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = flow.tick_cycle(replicas); + // p_to_replicas.inspect(q!(|payload: ReplicaPayload| println!("Replica received payload: {:?}", payload))); + let r_sorted_payloads = p_to_replicas + .clone() + .tick_batch() + .union(r_buffered_payloads) // Combine with all payloads that we've received and not processed yet + .sort(); + // Create a cycle since we'll use this seq before we define it + let (r_highest_seq_complete_cycle, r_highest_seq) = + flow.tick_cycle::>(replicas); + let empty_slot = flow.singleton_first_tick(replicas, q!(-1)); + // Either the max sequence number executed so far or -1. Need to union otherwise r_highest_seq is empty and joins with it will fail + let r_highest_seq_with_default = r_highest_seq.union(empty_slot); + // Find highest the sequence number of any payload that can be processed in this tick. This is the payload right before a hole. + let r_highest_seq_processable_payload = r_sorted_payloads + .clone() + .cross_singleton(r_highest_seq_with_default) + .fold( + q!(|| -1), + q!(|filled_slot, (sorted_payload, highest_seq)| { + // Note: This function only works if the input is sorted on 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 + }; + }), + ); + // Find all payloads that can and cannot be processed in this tick. + let r_processable_payloads = r_sorted_payloads + .clone() + .cross_singleton(r_highest_seq_processable_payload.clone()) + .filter(q!( + |(sorted_payload, highest_seq)| sorted_payload.seq <= *highest_seq + )) + .map(q!(|(sorted_payload, _)| { sorted_payload })); + let r_new_non_processable_payloads = r_sorted_payloads + .clone() + .cross_singleton(r_highest_seq_processable_payload.clone()) + .filter(q!( + |(sorted_payload, highest_seq)| sorted_payload.seq > *highest_seq + )) + .map(q!(|(sorted_payload, _)| { sorted_payload })); + // Save these, we can process them once the hole has been filled + r_buffered_payloads_complete_cycle.complete_next_tick(r_new_non_processable_payloads); + + let r_kv_store = r_processable_payloads + .clone() + .persist() // Optimization: all_ticks() + fold() = fold, where the state of the previous fold is saved and persisted values are deleted. + .fold(q!(|| (HashMap::::new(), -1)), q!(|state, 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; + // println!("Replica kv store: {:?}", kv_store); + })); + // Update the highest seq for the next tick + let r_new_highest_seq = r_kv_store.map(q!(|(_kv_store, highest_seq)| highest_seq)); + r_highest_seq_complete_cycle.complete_next_tick(r_new_highest_seq.clone().into()); + + // Send checkpoints to the acceptors when we've processed enough payloads + let (r_checkpointed_seqs_complete_cycle, r_checkpointed_seqs) = + flow.tick_cycle::>(replicas); + let r_max_checkpointed_seq = r_checkpointed_seqs + .persist() + .max() + .unwrap_or(flow.singleton(replicas, q!(-1)).latest_tick()); + let r_checkpoint_seq_new = r_max_checkpointed_seq + .cross_singleton(r_new_highest_seq) + .filter_map(q!( + move |(max_checkpointed_seq, new_highest_seq)| if new_highest_seq - max_checkpointed_seq + >= checkpoint_frequency as i32 + { + Some(new_highest_seq) + } else { + None + } + )); + r_checkpointed_seqs_complete_cycle.complete_next_tick(r_checkpoint_seq_new.clone()); + + // Tell clients that the payload has been committed. All ReplicaPayloads contain the client's machine ID (to string) as value. + let r_to_clients = p_to_replicas.map(q!(|payload| ( + payload.value.parse::().unwrap(), + payload + ))); + (r_checkpoint_seq_new.all_ticks(), r_to_clients) +} + +// Clients. All relations for clients will be prefixed with c. All ClientPayloads will contain the virtual client number as key and the client's machine ID (to string) as value. Expects p_to_clients_leader_elected containing Ballots whenever the leader is elected, and r_to_clients_payload_applied containing ReplicaPayloads whenever a payload is committed. Outputs (leader address, ClientPayload) when a new leader is elected or when the previous payload is committed. +fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( + clients: &Cluster, + p_to_clients_leader_elected: Stream<'a, B, Unbounded, NoTick, Cluster>, + r_to_clients_payload_applied: Stream< + 'a, + (u32, ReplicaPayload), + Unbounded, + NoTick, + Cluster, + >, + flow: &FlowBuilder<'a>, + num_clients_per_node: usize, + median_latency_window_size: usize, + f: usize, +) -> Stream<'a, (u32, ClientPayload), Unbounded, NoTick, Cluster> { + let c_id = flow.cluster_self_id(clients); + // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); + // Only keep the latest leader + let c_max_leader_ballot = p_to_clients_leader_elected + .inspect(q!(|ballot| println!( + "Client notified that leader was elected: {:?}", + ballot + ))) + .max(); + let c_new_leader_ballot = c_max_leader_ballot.clone().latest_tick().delta(); + // Whenever the leader changes, make all clients send a message + let c_new_payloads_when_leader_elected = + c_new_leader_ballot + .clone() + .flat_map(q!(move |leader_ballot| (0..num_clients_per_node).map( + move |i| ( + leader_ballot.leader_id(), + ClientPayload { + key: i as u32, + value: c_id.to_string() + } + ) + ))); + // Whenever replicas confirm that a payload was committed, collected it and wait for a quorum + let (c_pending_quorum_payloads_complete_cycle, c_pending_quorum_payloads) = + flow.tick_cycle(clients); + let c_received_payloads = r_to_clients_payload_applied + .tick_batch() + .map(q!(|(sender, replica_payload)| ( + replica_payload.key, + sender + ))) + .union(c_pending_quorum_payloads); + let c_received_quorum_payloads = c_received_payloads + .clone() + .fold_keyed( + q!(|| 0), + q!(|curr_count, _sender| { + *curr_count += 1; // Assumes the same replica will only send commit once + }), + ) + .filter_map(q!(move |(key, count)| { + if count == f + 1 { + Some(key) + } else { + None + } + })); + let c_new_pending_quorum_payloads = + c_received_payloads.anti_join(c_received_quorum_payloads.clone()); + c_pending_quorum_payloads_complete_cycle.complete_next_tick(c_new_pending_quorum_payloads); + // Whenever all replicas confirm that a payload was committed, send another payload + let c_new_payloads_when_committed = c_received_quorum_payloads + .clone() + .cross_singleton(c_max_leader_ballot.clone().latest_tick()) + .map(q!(move |(key, leader_ballot)| ( + leader_ballot.leader_id(), + ClientPayload { + key, + value: c_id.to_string() + } + ))); + let c_to_proposers = c_new_payloads_when_leader_elected + .union(c_new_payloads_when_committed) + .all_ticks(); + + // Track statistics + let (c_timers_complete_cycle, c_timers) = + flow.tick_cycle::>(clients); + let c_new_timers_when_leader_elected = c_new_leader_ballot + .map(q!(|_| SystemTime::now())) + .flat_map(q!( + move |now| (0..num_clients_per_node).map(move |virtual_id| (virtual_id, now)) + )); + let c_updated_timers = c_received_quorum_payloads + .clone() + .map(q!(|key| (key as usize, SystemTime::now()))); + let c_new_timers = c_timers + .clone() // Update c_timers in tick+1 so we can record differences during this tick (to track latency) + .union(c_new_timers_when_leader_elected) + .union(c_updated_timers.clone()) + .reduce_keyed(q!(|curr_time, new_time| { + if new_time > *curr_time { + *curr_time = new_time; + } + })); + c_timers_complete_cycle.complete_next_tick(c_new_timers); + + let c_stats_output_timer = flow.source_interval(clients, q!(Duration::from_secs(1))); + + let c_latency_reset = c_stats_output_timer + .clone() + .latest_tick() + .map(q!(|_| None)) + .defer_tick(); + + let c_latencies = c_timers + .join(c_updated_timers) + .map(q!(|(_virtual_id, (prev_time, curr_time))| Some( + curr_time.duration_since(prev_time).unwrap().as_micros() + ))) + .union(c_latency_reset.into_stream()) + .all_ticks() + .fold( + // Create window with ring buffer using vec + wraparound index + // TODO: Would be nice if I could use vec![] instead, but that doesn't work in HF+ with RuntimeData *median_latency_window_size + q!(move || ( + Rc::new(RefCell::new(Vec::::with_capacity( + median_latency_window_size + ))), + 0usize, + false + )), + q!(move |(latencies, write_index, has_any_value), latency| { + let mut latencies_mut = latencies.borrow_mut(); + if let Some(latency) = latency { + // Insert into latencies + if let Some(prev_latency) = latencies_mut.get_mut(*write_index) { + *prev_latency = latency; + } else { + latencies_mut.push(latency); + } + *has_any_value = true; + // Increment write index and wrap around + *write_index += 1; + if *write_index == median_latency_window_size { + *write_index = 0; + } + } else { + // reset latencies + latencies_mut.clear(); + *write_index = 0; + *has_any_value = false; + } + }), + ); + let c_throughput_new_batch = c_received_quorum_payloads + .clone() + .count() + .continue_unless(c_stats_output_timer.clone().latest_tick()) + .map(q!(|batch_size| (batch_size, false))); + + let c_throughput_reset = c_stats_output_timer + .clone() + .latest_tick() + .map(q!(|_| (0, true))) + .defer_tick(); + + let c_throughput = c_throughput_new_batch + .union(c_throughput_reset) + .all_ticks() + .fold( + q!(|| (0, 0)), + q!(|(total, num_ticks), (batch_size, reset)| { + if reset { + *total = 0; + *num_ticks = 0; + } else { + *total += batch_size as u32; + *num_ticks += 1; + } + }), + ); + + c_stats_output_timer + .cross_singleton(c_latencies) + .cross_singleton(c_throughput) + .tick_samples() + .for_each(q!(move |( + (_, (latencies, _write_index, has_any_value)), + (throughput, num_ticks), + )| { + let mut latencies_mut = latencies.borrow_mut(); + let median_latency = if has_any_value { + let (_, median, _) = + latencies_mut.select_nth_unstable(median_latency_window_size / 2); + *median + } else { + 0 + }; + println!("Median latency: {}ms", median_latency as f64 / 1000.0); + println!("Throughput: {} requests/s", throughput); + println!("Num ticks per second: {}", num_ticks); + })); + // End track statistics + c_to_proposers +} + +#[cfg(test)] +mod tests { + use hydroflow_plus_deploy::DeployRuntime; + use stageleft::RuntimeData; + + #[test] + fn paxos_ir() { + let builder = hydroflow_plus::FlowBuilder::new(); + let _ = super::paxos_bench(&builder, 1, 1, 1, 1, 1, 1, 1); + let built = builder.with_default_optimize(); + + insta::assert_debug_snapshot!(built.ir()); + + let _ = built.compile::(&RuntimeData::new("FAKE")); + } +} diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap similarity index 79% rename from hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap rename to hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap index 8aa6532267a..699752d6a65 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos__tests__paxos_ir.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap @@ -1,753 +1,625 @@ --- -source: hydroflow_plus_test/src/cluster/paxos.rs +source: hydroflow_plus_test/src/cluster/paxos_bench.rs expression: built.ir() --- [ ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, + 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 :: * ; ["Proposers say hello"] }, + ), + location_kind: Cluster( + 2, + ), }, }, CycleSink { ident: Ident { - sym: cycle_2, + sym: cycle_3, }, location_kind: Cluster( 2, ), - input: DeferTick( - AntiJoin( + 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_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 } } }), + input: CrossSingleton( Tee { inner: RefCell { value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, + 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( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), location_kind: Cluster( 2, ), }, - }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Union( CycleSource { ident: Ident { - sym: cycle_2, + sym: cycle_3, }, location_kind: Cluster( 2, ), }, - ), - }, - }, - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), }, - }, - }, + ), + ), }, }, ), - ), - }, - CycleSink { - ident: Ident { - sym: cycle_3, }, - location_kind: Cluster( - 2, - ), - input: DeferTick( - ReduceKeyed { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < std :: time :: SystemTime , std :: time :: SystemTime , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_time , new_time | { if new_time > * curr_time { * curr_time = new_time ; } } }), - input: Union( - Union( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_3, - }, + }, + 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: RefCell { + value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), location_kind: Cluster( 2, ), }, - }, - }, - FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < std :: time :: SystemTime , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos :: * ; let num_clients_per_node = 1usize ; move | now | (0 .. num_clients_per_node) . map (move | virtual_id | (virtual_id , now)) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: time :: SystemTime > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | SystemTime :: now () }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , (usize , std :: time :: SystemTime) > ({ use crate :: __staged :: cluster :: paxos :: * ; | key | (key as usize , SystemTime :: now ()) }), - input: Tee { + Tee { inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { + value: 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: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), + value: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 2, + ), + }, }, }, - }, + ), }, }, }, - }, - }, - }, - ), - }, - ), - }, - ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((() , (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < u128 > > > , usize , bool)) , (u32 , i32)) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; let median_latency_window_size = 1usize ; move | ((_ , (latencies , _write_index , has_any_value)) , (throughput , num_ticks) ,) | { let mut latencies_mut = latencies . borrow_mut () ; let median_latency = if has_any_value { let (_ , median , _) = latencies_mut . select_nth_unstable (median_latency_window_size / 2) ; * median } else { 0 } ; println ! ("Median latency: {}ms" , median_latency as f64 / 1000.0) ; println ! ("Throughput: {} requests/s" , throughput) ; println ! ("Num ticks per second: {}" , num_ticks) ; } }), - input: CrossSingleton( - CrossSingleton( - Tee { - inner: RefCell { - value: Source { - source: Interval( - { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, ), - location_kind: Cluster( - 2, - ), - }, - }, - }, - Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < u128 > > > , usize , bool) > ({ use crate :: __staged :: cluster :: paxos :: * ; let median_latency_window_size = 1usize ; move | | (Rc :: new (RefCell :: new (Vec :: < u128 > :: with_capacity (median_latency_window_size))) , 0usize , false) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < u128 > > > , usize , bool) , core :: option :: Option < u128 > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; let median_latency_window_size = 1usize ; move | (latencies , write_index , has_any_value) , latency | { let mut latencies_mut = latencies . borrow_mut () ; if let Some (latency) = latency { if let Some (prev_latency) = latencies_mut . get_mut (* write_index) { * prev_latency = latency ; } else { latencies_mut . push (latency) ; } * has_any_value = true ; * write_index += 1 ; if * write_index == median_latency_window_size { * write_index = 0 ; } } else { latencies_mut . clear () ; * write_index = 0 ; * has_any_value = false ; } } }), - input: Persist( - Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (std :: time :: SystemTime , std :: time :: SystemTime)) , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_virtual_id , (prev_time , curr_time)) | Some (curr_time . duration_since (prev_time) . unwrap () . as_micros ()) }), - input: Join( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , (usize , std :: time :: SystemTime) > ({ use crate :: __staged :: cluster :: paxos :: * ; | key | (key as usize , SystemTime :: now ()) }), - input: Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, + 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: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, ), }, }, - CycleSource { - ident: Ident { - sym: cycle_2, + }, + }, + ), + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), }, - location_kind: Cluster( - 2, - ), }, ), }, }, }, }, - }, + ), }, }, }, }, - ), - }, - DeferTick( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | None }), - input: Tee { - inner: RefCell { - value: Source { - source: Interval( - { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, - ), - location_kind: Cluster( - 2, - ), - }, - }, - }, }, - ), + }, ), - ), - }, - ), - Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , i32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , 0) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , i32) , (usize , bool) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (total , num_ticks) , (batch_size , reset) | { if reset { * total = 0 ; * num_ticks = 0 ; } else { * total += batch_size as u32 ; * num_ticks += 1 ; } } }), - input: Persist( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos :: * ; | batch_size | (batch_size , false) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: RefCell { - value: Source { - source: Interval( - { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, - ), - location_kind: Cluster( - 2, - ), - }, - }, - }, - }, - }, - }, - ), - }, - }, - DeferTick( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | (0 , true) }), - input: Tee { - inner: RefCell { - value: Source { - source: Interval( - { use crate :: __staged :: cluster :: paxos :: * ; Duration :: from_secs (1) }, - ), - location_kind: Cluster( - 2, - ), - }, - }, - }, - }, - ), - ), - ), - }, - ), - }, - ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < & str , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | s | println ! ("{}" , s) }), - input: Source { - source: Iter( - { use crate :: __staged :: cluster :: paxos :: * ; ["Proposers say hello"] }, - ), - location_kind: Cluster( - 0, - ), - }, - }, - ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), + }, }, }, }, }, CycleSink { ident: Ident { - sym: cycle_3, + sym: cycle_0, }, location_kind: Cluster( - 0, + 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_cluster_self_id_0 ; 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 } } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: 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( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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, + ), + to_location: Cluster( + 2, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_2 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ballot_num | 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( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, }, + location_kind: Cluster( + 2, + ), }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), location_kind: Cluster( - 0, + 2, ), }, - }, - }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, ), - }, + ), }, }, - ), - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - 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: RefCell { - value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, - ), - location_kind: Cluster( - 0, - ), - }, - Tee { - inner: RefCell { - value: Fold { - init: 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: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), + 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, + ), + }, }, - }, + ), }, - ), - 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: RefCell { - value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, }, - location_kind: Cluster( - 0, - ), }, }, }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, + }, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), location_kind: Cluster( - 0, + 2, ), }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), ), - }, + ), }, - ), - }, + }, + ), }, }, }, - ), - }, + }, + ), }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: 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 :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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( Union( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, }, + location_kind: Cluster( + 2, + ), }, }, }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 0, - ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, }, + location_kind: Cluster( + 2, + ), }, }, }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, }, + location_kind: Cluster( + 2, + ), }, }, - ), + }, ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, ), - }, + ), }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), location_kind: Cluster( - 0, + 2, ), }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), ), - }, + ), }, - ), - }, + }, + ), }, }, }, - ), - }, + }, + ), }, }, }, }, - }, - ), + ), + }, }, }, }, @@ -755,350 +627,92 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_0, + sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 0, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num | 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: DeferTick( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ 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: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, + Union( + Union( 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( - 0, - ), - }, - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - }, - }, - ), - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { + Union( + Tee { inner: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { + value: 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: RefCell { - value: 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( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: 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 : 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 :: < ((u32 , 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 () }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { + ), + Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, ), - }, - }, - }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, + ), }, - location_kind: Cluster( - 0, - ), }, - }, + ), }, - ), - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - }, - }, - ), - }, - }, - }, - }, - ), - }, - }, - }, - }, - }, - }, - CycleSink { - ident: Ident { - sym: cycle_4, - }, - location_kind: Cluster( - 0, - ), - input: DeferTick( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Union( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , ()) , i32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Union( - Tee { - inner: RefCell { - value: 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)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, + }, }, }, }, @@ -1114,7 +728,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -1133,7 +747,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -1156,24 +770,24 @@ expression: built.ir() inner: RefCell { value: 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 :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -1184,105 +798,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -1300,24 +828,24 @@ expression: built.ir() inner: RefCell { value: 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 :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -1328,105 +856,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -1449,7 +891,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -1467,7 +909,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -1487,24 +929,24 @@ expression: built.ir() inner: RefCell { value: 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 :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -1515,105 +957,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -1644,22 +1000,29 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -1672,7 +1035,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -1681,7 +1044,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -1701,7 +1064,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -1712,7 +1075,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { value: CycleSource { @@ -1720,14 +1083,14 @@ expression: built.ir() sym: cycle_1, }, location_kind: Cluster( - 0, + 2, ), }, }, }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), input: Tee { inner: RefCell { value: CycleSource { @@ -1735,7 +1098,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -1749,7 +1112,7 @@ expression: built.ir() sym: cycle_0, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -1763,7 +1126,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -1778,7 +1141,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -1787,7 +1150,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -1825,22 +1188,29 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -1853,7 +1223,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -1862,7 +1232,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -1882,7 +1252,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -1893,7 +1263,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { value: CycleSource { @@ -1901,14 +1271,14 @@ expression: built.ir() sym: cycle_1, }, location_kind: Cluster( - 0, + 2, ), }, }, }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), input: Tee { inner: RefCell { value: CycleSource { @@ -1916,7 +1286,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -1930,7 +1300,7 @@ expression: built.ir() sym: cycle_0, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -1944,7 +1314,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -1959,7 +1329,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -1968,7 +1338,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -2009,29 +1379,36 @@ expression: built.ir() value: 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)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + 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: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 ; } } } }), + 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 : 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , 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 () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -2044,7 +1421,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -2053,7 +1430,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -2078,7 +1455,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -2097,7 +1474,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -2120,24 +1497,24 @@ expression: built.ir() inner: RefCell { value: 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 :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -2148,105 +1525,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -2264,24 +1555,24 @@ expression: built.ir() inner: RefCell { value: 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 :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -2292,105 +1583,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -2413,7 +1618,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -2431,7 +1636,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -2451,24 +1656,24 @@ expression: built.ir() inner: RefCell { value: 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 :: ClientPayload , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + 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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -2479,105 +1684,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -2608,22 +1727,29 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -2636,7 +1762,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -2645,7 +1771,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -2665,7 +1791,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -2676,7 +1802,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { value: CycleSource { @@ -2684,14 +1810,14 @@ expression: built.ir() sym: cycle_1, }, location_kind: Cluster( - 0, + 2, ), }, }, }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), input: Tee { inner: RefCell { value: CycleSource { @@ -2699,7 +1825,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -2713,7 +1839,7 @@ expression: built.ir() sym: cycle_0, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -2727,7 +1853,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -2742,7 +1868,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -2751,7 +1877,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -2783,296 +1909,26 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_1, + sym: cycle_5, }, location_kind: Cluster( 2, ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 2, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_2 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , 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: RefCell { - value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - }, - }, - ), - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: 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( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - }, - }, - ), - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_4, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - }, - }, - }, - ), - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - }, - }, - }, - CycleSink { - ident: Ident { - sym: cycle_5, - }, - location_kind: Cluster( - 0, - ), input: DeferTick( Difference( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , (_count , _p2b)) | slot }), + 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 }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), + 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 }), input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ 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 :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), input: Tee { inner: RefCell { value: Union( @@ -3083,7 +1939,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -3093,7 +1949,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Cluster( - 0, + 2, ), }, ), @@ -3110,14 +1966,14 @@ expression: built.ir() Tee { inner: RefCell { value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , 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 } }), + 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 } }), input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ 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 :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), input: Tee { inner: RefCell { value: Union( @@ -3128,7 +1984,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -3138,7 +1994,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Cluster( - 0, + 2, ), }, ), @@ -3159,14 +2015,14 @@ expression: built.ir() sym: cycle_6, }, location_kind: Cluster( - 0, + 2, ), input: DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (sender , p2b)) | (sender , p2b) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (sender , p2b)) | (sender , p2b) }), input: AntiJoin( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | (p2b . slot , (sender , p2b)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | (p2b . slot , (sender , p2b)) }), input: Tee { inner: RefCell { value: Union( @@ -3177,7 +2033,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -3187,7 +2043,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Cluster( - 0, + 2, ), }, ), @@ -3197,14 +2053,14 @@ expression: built.ir() Tee { inner: RefCell { value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , 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 } }), + 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 } }), input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ 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 :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), input: Tee { inner: RefCell { value: Union( @@ -3215,7 +2071,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -3225,7 +2081,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Cluster( - 0, + 2, ), }, ), @@ -3249,283 +2105,334 @@ expression: built.ir() { use crate :: __staged :: cluster :: paxos :: * ; ["Acceptors say hello"] }, ), location_kind: Cluster( - 1, + 3, ), }, }, - ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), - input: Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 1, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; 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: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + input: Network { + from_location: Cluster( + 3, + ), + to_location: Cluster( + 2, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: 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 () ; (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 > >)) , (u32 , 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: RefCell { + value: 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 :: < (u32 , 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, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), ), - }, - }, - 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: RefCell { - value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, - ), - location_kind: Cluster( - 0, - ), - }, - Tee { - inner: RefCell { - value: Fold { - init: 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: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __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: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, }, + location_kind: Cluster( + 2, + ), }, - }, - ), - 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: RefCell { - value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + 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: RefCell { + value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 2, + ), + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, }, + location_kind: Cluster( + 2, + ), }, }, }, ), }, }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + }, + ), + 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: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( - Tee { - inner: RefCell { - value: 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( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, + 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, - }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), }, - location_kind: Cluster( - 0, - ), }, - }, + ), }, - ), - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, - ), + }, }, - ), + }, ), }, }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, ), }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), + }, }, }, ), @@ -3533,341 +2440,321 @@ expression: built.ir() }, }, }, - ), + }, }, - }, + ), }, }, }, }, - ), - }, + }, + ), }, }, }, }, - ), + }, }, }, }, - }, - }, - }, - }, - }, - CycleSink { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - input: Network { - from_location: Cluster( - 1, - ), - to_location: Cluster( - 0, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: Map { - f: 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 >)) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) > ({ 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 1, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; 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: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, + value: 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: RefCell { + value: 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 :: < (u32 , 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, ), - ), - }, - }, - 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: RefCell { - value: 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 } } }), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, - ), - location_kind: Cluster( - 0, - ), - }, + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __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: RefCell { - value: 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: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), - }, - }, + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), }, ), - }, + ), }, }, - ), - 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + 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: RefCell { + value: 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( - 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, + CrossSingleton( + Source { + source: Stream( + { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 2, + ), + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 2, + ), + }, }, }, - }, + ), }, - ), + }, }, - }, + ), Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: 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( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + 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: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + }, }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), }, - location_kind: Cluster( - 0, - ), }, - }, - }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, ), }, }, }, - ), + }, ), }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + ), + }, }, - ), - ), - }, + }, + }, + ), }, - ), + }, }, }, }, @@ -3878,581 +2765,289 @@ expression: built.ir() }, }, }, - }, - ), + ), + }, }, }, }, }, }, - ), + }, }, }, - }, + ), }, - }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), + ), }, }, - Tee { - inner: RefCell { - value: 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 1, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; 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 }), + ), + 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: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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, + ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: 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 () ; (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 < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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( - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, + 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_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: RefCell { + value: 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 : 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 :: < ((u32 , 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 () }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + ), + }, + }, + }, + }, + }, }, - location_kind: Cluster( - 0, - ), }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - 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: RefCell { - value: 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( + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( Source { - source: Stream( - { use hydroflow_plus :: __staged :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, - Tee { - inner: RefCell { - value: 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: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - }, - }, - }, ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + ), + }, + }, + ), + }, + 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_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: RefCell { + value: 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: RefCell { - value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { + value: 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 : 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 :: < ((u32 , 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 () }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), + value: CycleSource { + ident: Ident { + sym: cycle_1, }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, + location_kind: Cluster( + 2, ), - ), + }, }, }, - ), + }, }, }, - }, - }, - ), - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: 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( - Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - ), + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, + location_kind: Cluster( + 2, ), - ), - }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), location_kind: Cluster( - 0, + 2, ), }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), ), - }, + ), }, - ), - }, + }, + ), }, }, }, - ), + }, }, }, }, }, }, }, - ), - }, - }, - }, - }, - }, - ), - }, - }, - }, - }, - }, - }, - }, - }, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 1, - ), - }, - ), - ), - }, - }, - ), - Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (- 1 , HashMap :: < i32 , LogValue > :: new ()) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue >) , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a) , () > ({ 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 :: Ballot) , core :: option :: Option < (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some ((- 1 , p2a)) } else { None } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2a) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 1, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ 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 :: P2a > ({ 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)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; 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: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : ClientPayload { key : 0 , value : "0" . to_string () } } }), - 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: RefCell { - value: 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)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), - location_kind: Cluster( - 0, - ), - }, - ), - ), - }, - }, - ), - }, - }, - }, - }, - }, - }, - }, - }, - }, - }, - }, + }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + 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: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 ; } } } }), + 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 : 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , 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 () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -4465,7 +3060,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -4474,7 +3069,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -4499,7 +3094,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -4508,7 +3103,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -4532,7 +3127,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -4543,27 +3138,27 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }), + 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_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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -4574,105 +3169,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -4685,7 +3194,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -4699,7 +3208,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -4708,7 +3217,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -4732,22 +3241,29 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -4760,7 +3276,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -4769,7 +3285,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -4789,7 +3305,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -4800,7 +3316,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { value: CycleSource { @@ -4808,14 +3324,14 @@ expression: built.ir() sym: cycle_1, }, location_kind: Cluster( - 0, + 2, ), }, }, }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), input: Tee { inner: RefCell { value: CycleSource { @@ -4823,7 +3339,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -4837,7 +3353,7 @@ expression: built.ir() sym: cycle_0, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -4851,7 +3367,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -4866,7 +3382,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -4875,7 +3391,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -4909,296 +3425,306 @@ expression: built.ir() 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 1, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", - ], - }, + value: 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 :: < (u32 , 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 () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", - ], - }, + 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 < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __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: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, location_kind: Cluster( - 0, + 2, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), + }, }, - }, - 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: RefCell { - value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, - ), - location_kind: Cluster( - 0, - ), - }, - Tee { - inner: RefCell { - value: Fold { - init: 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: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, + 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: RefCell { + value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 2, + ), + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, }, - }, - ), + ), + }, }, }, - }, - ), - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , bool , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Tee { - inner: RefCell { - value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, + ), + 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: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, }, - location_kind: Cluster( - 0, - ), }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), + }, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, location_kind: Cluster( - 0, + 2, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), + }, }, - }, - ), + ), + }, }, }, }, - }, - ), + ), + }, }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: 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 :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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 :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, }, }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, }, }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, }, - }, - ), - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, + ), ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), + }, }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, location_kind: Cluster( - 0, + 2, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), + }, }, - }, - ), + ), + }, }, }, }, - }, - ), + ), + }, }, }, }, }, }, - }, - ), + ), + }, }, }, }, }, - }, - ), + ), + }, }, }, }, @@ -5215,7 +3741,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( - 1, + 3, ), }, ), @@ -5225,7 +3751,7 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a) > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | (min_seq , P2a { ballot : Ballot { num : 0 , id : 0 } , slot : - 1 , value : ClientPayload { key : 0 , value : "" . to_string () , } }) }), + 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 : 0 } , slot : - 1 , value : Default :: default () }) }), input: Delta( Union( Reduce { @@ -5240,13 +3766,45 @@ expression: built.ir() value: 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( - CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( + Network { + from_location: Cluster( 1, ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + }, + }, }, ), }, @@ -5264,13 +3822,45 @@ expression: built.ir() value: 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( - CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( + Network { + from_location: Cluster( 1, ), + to_location: Cluster( + 3, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + }, + }, }, ), }, @@ -5289,7 +3879,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, ), location_kind: Cluster( - 1, + 3, ), }, ), @@ -5308,21 +3898,21 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), input: Network { from_location: Cluster( - 1, + 3, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", ], }, ), @@ -5333,31 +3923,31 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (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 :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . id , P2b { ballot : p2a . ballot , max_ballot , slot : p2a . slot , value : p2a . value }) }), + 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) , (u32 , 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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2a) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 0, + 2, ), to_location: Cluster( - 1, + 3, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", ], }, ), @@ -5368,46 +3958,53 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (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 , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + 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 < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + 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 :: P2a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + 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)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; 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 } }), + 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_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: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 ; } } } }), + 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 : 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , 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 () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -5420,7 +4017,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5429,7 +4026,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5452,7 +4049,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5461,7 +4058,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5471,7 +4068,7 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : ClientPayload { key : 0 , value : "0" . to_string () } } }), + 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_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 { @@ -5481,29 +4078,36 @@ expression: built.ir() value: 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)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + 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: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 ; } } } }), + 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 : 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , 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 () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -5516,7 +4120,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5525,7 +4129,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5546,29 +4150,36 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue)) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + 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: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : ClientPayload { key : 0 , value : "" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue) , hydroflow_plus_test :: cluster :: paxos :: LogValue , () > ({ 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 ; } } } }), + 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 : 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 :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , 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 () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -5581,7 +4192,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5590,7 +4201,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5615,7 +4226,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5624,7 +4235,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5648,7 +4259,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -5659,27 +4270,27 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }), + 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_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: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( - 2, + 0, ), to_location: Cluster( - 0, + 2, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -5690,105 +4301,19 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), ), - input: Union( - 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 :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . get_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), - input: Tee { - inner: RefCell { - value: Delta( - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), - }, - }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; let c_id = __hydroflow_plus_cluster_self_id_2 ; move | (key , leader_ballot) | (leader_ballot . get_id () , ClientPayload { key , value : c_id . to_string () }) }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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 :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), - input: Tee { - inner: RefCell { - value: Union( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 2, - ), - }, - ), - }, - }, - }, - }, - }, - }, - Tee { - inner: RefCell { - value: Reduce { - f: 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( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), - }, - }, - }, - ), - }, - }, - }, - ), + input: CycleSource { + ident: Ident { + sym: cycle_0, }, - ), + location_kind: Cluster( + 0, + ), + }, }, }, }, @@ -5801,7 +4326,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -5815,7 +4340,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5824,7 +4349,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5848,22 +4373,29 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, }, @@ -5876,7 +4408,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5885,7 +4417,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5905,7 +4437,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -5916,7 +4448,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { value: CycleSource { @@ -5924,14 +4456,14 @@ expression: built.ir() sym: cycle_1, }, location_kind: Cluster( - 0, + 2, ), }, }, }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), input: Tee { inner: RefCell { value: CycleSource { @@ -5939,7 +4471,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -5953,7 +4485,7 @@ expression: built.ir() sym: cycle_0, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -5967,7 +4499,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -5982,7 +4514,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Cluster( - 0, + 2, ), }, Persist( @@ -5991,7 +4523,7 @@ expression: built.ir() { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, ), location_kind: Cluster( - 0, + 2, ), }, ), @@ -6025,282 +4557,1826 @@ expression: built.ir() 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 1, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", - ], - }, + value: 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 :: < (u32 , 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 () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", - ], - }, + 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 < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __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: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, location_kind: Cluster( - 0, + 2, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), + }, }, - }, - 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: RefCell { - value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_0 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, - ), - location_kind: Cluster( - 0, - ), - }, - Tee { - inner: RefCell { - value: Fold { - init: 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: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - ), + 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: RefCell { + value: 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 :: builder :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + ), + location_kind: Cluster( + 2, + ), + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + }, }, }, - }, - ), - 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: RefCell { - value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: Unique( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), - input: Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_0 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Persist( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + ), + 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: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, }, - location_kind: Cluster( - 0, - ), }, }, - }, - ), - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, location_kind: Cluster( - 0, + 2, ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), + }, }, - }, - ), + ), + }, }, }, }, - }, - ), + ), + }, }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), - input: Tee { - inner: RefCell { - value: 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_cluster_self_id_0 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: 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 :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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 :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_1, + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, }, }, - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), - input: Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, }, }, - }, - ), - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_0, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, }, - }, - ), - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 0, + ), ), }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), - ), + }, }, - }, - Tee { - inner: RefCell { - value: Union( - CycleSource { - ident: Ident { - sym: cycle_3, - }, - location_kind: Cluster( - 0, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, - ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, location_kind: Cluster( - 0, + 2, ), }, - ), - ), - }, - }, - ), + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 3, + ), + }, + ), + ), + }, + }, + ), + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + input: DeferTick( + 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 1, + ), + }, + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + ), + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + ), + }, + CycleSink { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 1, + ), + input: DeferTick( + Tee { + inner: RefCell { + value: 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: RefCell { + value: 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 1, + ), + }, + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + ), + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + CycleSink { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 1, + ), + input: DeferTick( + Tee { + inner: RefCell { + value: 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( + 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: Persist( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 1, + ), + }, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + ), + location_kind: Cluster( + 1, + ), + }, + ), + ), + Tee { + inner: RefCell { + value: 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: RefCell { + value: 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 1, + ), + }, + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + ), + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + }, + ), + }, + CycleSink { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 1, + ), + input: Tee { + inner: RefCell { + value: 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( + 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: Persist( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 1, + ), + }, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + ), + location_kind: Cluster( + 1, + ), + }, + ), + ), + Tee { + inner: RefCell { + value: 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: RefCell { + value: 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: Sort( + Union( + Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + Union( + CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 1, + ), + }, + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos_bench :: * ; - 1 } ; [e] }, + ), + location_kind: Cluster( + 1, + ), + }, + ), + ), + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + }, + }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + input: DeferTick( + AntiJoin( + Tee { + inner: RefCell { + value: Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + input: Network { + from_location: Cluster( + 1, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + input: Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + Tee { + inner: RefCell { + value: 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 , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + input: Tee { + inner: RefCell { + value: Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + input: Network { + from_location: Cluster( + 1, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + input: Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, }, + location_kind: Cluster( + 2, + ), }, }, }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, ), }, }, @@ -6308,13 +6384,21 @@ expression: built.ir() }, }, }, - ), + }, }, }, - }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), }, }, - ), + }, }, }, }, @@ -6322,340 +6406,577 @@ expression: built.ir() }, }, }, - }, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, - ), - location_kind: Cluster( - 1, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, ), }, - ), - ), + }, + }, }, }, - ), - }, - }, + }, + ), + ), }, CycleSink { ident: Ident { - sym: cycle_0, + sym: cycle_2, }, location_kind: Cluster( - 3, + 0, ), input: DeferTick( - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) | { sorted_payload } }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), - input: CrossSingleton( + ReduceKeyed { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < std :: time :: SystemTime , std :: time :: SystemTime , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_time , new_time | { if new_time > * curr_time { * curr_time = new_time ; } } }), + input: Union( + Union( Tee { inner: RefCell { - value: Sort( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( - Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < std :: time :: SystemTime , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let num_clients_per_node = 1usize ; move | now | (0 .. num_clients_per_node) . map (move | virtual_id | (virtual_id , now)) }), + 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: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), + input: Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , 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: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + ), }, - location_kind: Cluster( - 0, - ), }, }, }, - CycleSource { - ident: Ident { - sym: cycle_6, + ), + }, + }, + }, + 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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 2, + ), }, - location_kind: Cluster( - 0, - ), }, - ), + }, }, }, }, + ), + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), }, }, - }, + ), }, }, }, - CycleSource { - ident: Ident { - sym: cycle_5, - }, - location_kind: Cluster( - 0, - ), - }, - ), + }, }, - }, + ), }, }, }, - }, - CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 3, - ), - }, - ), - ), + ), + }, + }, }, }, - Tee { - inner: RefCell { - value: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | 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: RefCell { - value: Sort( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), + ), + Tee { + inner: RefCell { + value: 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: RefCell { + value: 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 , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + input: Tee { + inner: RefCell { + value: Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + input: Network { + from_location: Cluster( + 1, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( - Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + input: Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), }, - location_kind: Cluster( - 0, - ), }, }, }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 0, - ), - }, - ), + }, }, }, }, }, - }, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, + ), }, }, }, }, - CycleSource { - ident: Ident { - sym: cycle_5, - }, - location_kind: Cluster( - 0, - ), - }, - ), + }, }, }, }, }, }, - }, - CycleSource { - ident: Ident { - sym: cycle_0, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), }, - location_kind: Cluster( - 3, - ), - }, - ), - ), - }, - }, - Union( - CycleSource { - ident: Ident { - sym: cycle_1, + ), + }, }, - location_kind: Cluster( - 3, - ), - }, - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), }, - ), - ), + }, + }, }, }, }, - ), - }, + }, + ), }, ), }, - CycleSink { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 3, - ), - input: DeferTick( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_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 :: * ; | | (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 :: ReplicaPayload , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) | { sorted_payload } }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: Sort( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + ForEach { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((() , (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < u128 > > > , usize , bool)) , (u32 , i32)) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let median_latency_window_size = 1usize ; move | ((_ , (latencies , _write_index , has_any_value)) , (throughput , num_ticks) ,) | { let mut latencies_mut = latencies . borrow_mut () ; let median_latency = if has_any_value { let (_ , median , _) = latencies_mut . select_nth_unstable (median_latency_window_size / 2) ; * median } else { 0 } ; println ! ("Median latency: {}ms" , median_latency as f64 / 1000.0) ; println ! ("Throughput: {} requests/s" , throughput) ; println ! ("Num ticks per second: {}" , num_ticks) ; } }), + input: CrossSingleton( + CrossSingleton( + Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 0, + ), + }, + }, + }, + Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < u128 > > > , usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let median_latency_window_size = 1usize ; move | | (Rc :: new (RefCell :: new (Vec :: < u128 > :: with_capacity (median_latency_window_size))) , 0usize , false) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < u128 > > > , usize , bool) , core :: option :: Option < u128 > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let median_latency_window_size = 1usize ; move | (latencies , write_index , has_any_value) , latency | { let mut latencies_mut = latencies . borrow_mut () ; if let Some (latency) = latency { if let Some (prev_latency) = latencies_mut . get_mut (* write_index) { * prev_latency = latency ; } else { latencies_mut . push (latency) ; } * has_any_value = true ; * write_index += 1 ; if * write_index == median_latency_window_size { * write_index = 0 ; } } else { latencies_mut . clear () ; * write_index = 0 ; * has_any_value = false ; } } }), + input: Persist( + Union( + Map { + f: 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 0, + ), + }, + }, + }, + Tee { + inner: RefCell { + value: 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: RefCell { + value: 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 , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + input: Tee { + inner: RefCell { + value: Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { from_location: Cluster( - 0, + 1, ), to_location: Cluster( - 3, + 0, ), serialize_pipeline: Some( Operator( Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6666,169 +6987,89 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( - Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 0, - ), - }, - ), - }, - }, - }, - }, + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + input: Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], }, - }, - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_5, - }, - location_kind: Cluster( - 0, - ), - }, - ), - }, - }, - }, - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 3, - ), - }, - ), - ), - }, - }, - Tee { - inner: RefCell { - value: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | 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: RefCell { - value: Sort( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, + ), ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( - Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { ident: Ident { - sym: cycle_2, + sym: cycle_6, }, location_kind: Cluster( - 0, + 2, ), }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 0, ), }, - ), + }, }, }, }, @@ -6836,174 +7077,171 @@ expression: built.ir() }, }, }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_5, - }, - location_kind: Cluster( - 0, + CycleSource { + ident: Ident { + sym: cycle_5, + }, + location_kind: Cluster( + 2, + ), + }, ), }, - ), + }, }, }, }, }, }, }, - CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 3, - ), - }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, ), - ), - }, - }, - Union( - CycleSource { - ident: Ident { - sym: cycle_1, }, - location_kind: Cluster( - 3, - ), - }, - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), - }, - ), - ), + ), + }, + }, }, }, }, - ), + }, }, }, }, - }, - ), - }, - }, - }, - }, - ), - }, - CycleSink { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 3, - ), - input: DeferTick( - Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , i32) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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( - 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: Persist( - CycleSource { - ident: Ident { - sym: cycle_2, + ), + }, + DeferTick( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < () , core :: option :: Option < u128 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), + input: Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 0, + ), }, - location_kind: Cluster( - 3, - ), }, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), }, - ), + }, ), - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_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 :: * ; | | (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 :: ReplicaPayload , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) | { sorted_payload } }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: Sort( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), + ), + ), + }, + ), + Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , i32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | (0 , 0) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , i32) , (usize , bool) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (total , num_ticks) , (batch_size , reset) | { if reset { * total = 0 ; * num_ticks = 0 ; } else { * total += batch_size as u32 ; * num_ticks += 1 ; } } }), + input: Persist( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | batch_size | (batch_size , false) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: RefCell { + value: 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 , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + input: Tee { + inner: RefCell { + value: Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + input: Network { + from_location: Cluster( + 1, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + input: Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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 :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), + 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) }), input: AntiJoin( Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), + 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 }), input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ 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 :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), input: Tee { inner: RefCell { value: Union( @@ -7014,7 +7252,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Cluster( - 0, + 2, ), }, }, @@ -7024,7 +7262,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Cluster( - 0, + 2, ), }, ), @@ -7042,7 +7280,7 @@ expression: built.ir() sym: cycle_5, }, location_kind: Cluster( - 0, + 2, ), }, ), @@ -7052,156 +7290,355 @@ expression: built.ir() }, }, }, - CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 3, - ), - }, - ), - ), + }, + }, }, }, - Tee { - inner: RefCell { - value: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | 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: RefCell { - value: Sort( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | c | * c == 0 }), + input: Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + }, + }, + ), + }, + }, + DeferTick( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < () , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), + input: Tee { + inner: RefCell { + value: Source { + source: Interval( + { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) }, + ), + location_kind: Cluster( + 0, + ), + }, + }, + }, + }, + ), + ), + ), + }, + ), + }, + CycleSink { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 0, + ), + input: Union( + 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_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 . to_string () })) }), + input: Tee { + inner: RefCell { + value: Delta( + Tee { + inner: RefCell { + value: Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), + input: Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , 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: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( Tee { inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 0, - ), - }, - ), - }, + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + }, + }, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), + }, + }, + ), + }, + }, + }, + }, + ), + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: RefCell { + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, }, + location_kind: Cluster( + 2, + ), }, }, }, }, + ), + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - }, + ), + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), }, - CycleSource { - ident: Ident { - sym: cycle_5, - }, + ), + ), + }, + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), location_kind: Cluster( - 0, + 2, ), }, ), - }, + ), }, }, - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_0, + ), }, - location_kind: Cluster( - 3, - ), }, - ), - ), - }, + }, + }, + ), }, - Union( - CycleSource { - ident: Ident { - sym: cycle_1, + }, + }, + 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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, + }, + location_kind: Cluster( + 2, + ), + }, }, - location_kind: Cluster( - 3, - ), - }, - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), }, + }, + }, + }, + ), + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, ), - }, + ), }, }, ), @@ -7209,8 +7646,8 @@ expression: built.ir() }, }, }, - ), - }, + }, + ), }, }, }, @@ -7218,117 +7655,145 @@ expression: built.ir() }, }, }, - ), - }, - CycleSink { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - input: Network { - from_location: Cluster( - 3, - ), - to_location: Cluster( - 2, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), - input: Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( - Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = __hydroflow_plus_cluster_self_id_0 ; move | (key , leader_ballot) | (leader_ballot . leader_id () , ClientPayload { key , value : c_id . to_string () }) }), + input: CrossSingleton( + Tee { + inner: RefCell { + value: 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 , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + input: Tee { + inner: RefCell { + value: Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + input: Network { + from_location: Cluster( + 1, + ), + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + input: Tee { + inner: RefCell { + value: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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( + 2, + ), + to_location: Cluster( + 1, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: 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) }), + input: AntiJoin( + Tee { + inner: RefCell { + value: 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 }), + input: Tee { + inner: RefCell { + value: 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 : 0 } , max_ballot : Ballot { num : 0 , id : 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 >) , (u32 , 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 ; } }), + input: FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , 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 } }), + input: Tee { + inner: RefCell { + value: Union( + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + CycleSource { + ident: Ident { + sym: cycle_6, + }, + location_kind: Cluster( + 2, + ), + }, + ), + }, + }, + }, + }, + }, + }, + }, + }, + }, + CycleSource { ident: Ident { - sym: cycle_2, + sym: cycle_5, }, location_kind: Cluster( - 0, + 2, ), }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 0, ), }, - ), + }, }, }, }, @@ -7337,374 +7802,304 @@ expression: built.ir() }, }, }, - }, - CycleSource { - ident: Ident { - sym: cycle_5, + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Cluster( + 0, + ), }, - location_kind: Cluster( - 0, - ), - }, - ), + ), + }, }, }, }, }, }, - }, - }, - }, - }, - CycleSink { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 1, - ), - input: Network { - from_location: Cluster( - 3, - ), - to_location: Cluster( - 1, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Tee { - inner: RefCell { - value: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , i32) , core :: option :: Option < i32 > > ({ use crate :: __staged :: cluster :: paxos :: * ; 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( - 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: Persist( - CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 3, + Tee { + inner: RefCell { + value: Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), + input: Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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, ), - }, - ), - }, - Persist( - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), - }, - ), - ), - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , std :: string :: String > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_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 :: * ; | | (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 :: ReplicaPayload , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | 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: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , _) | { sorted_payload } }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), - input: CrossSingleton( - Tee { - inner: RefCell { - value: Sort( - Union( - Tee { - inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( - Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 0, - ), - }, - ), - }, + to_location: Cluster( + 0, + ), + serialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + ], + }, + ), + ), + instantiate_fn: , + deserialize_pipeline: Some( + Operator( + Operator { + path: "map", + args: [ + "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + ], + }, + ), + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , 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: RefCell { + value: 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 , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Unique( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + input: Tee { + inner: RefCell { + value: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , 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_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( + Persist( + Tee { + inner: RefCell { + value: Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, }, + location_kind: Cluster( + 2, + ), }, }, }, }, - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_5, + }, + }, + ), + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), }, - location_kind: Cluster( - 0, - ), }, ), }, }, }, }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 3, ), }, - ), - ), - }, - }, - Tee { - inner: RefCell { - value: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | | - 1 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , (hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , i32) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | 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 { + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () }), + input: Tee { inner: RefCell { - value: Sort( - Union( + value: 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_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + input: CrossSingleton( Tee { inner: RefCell { - value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - to_location: Cluster( - 3, - ), - serialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& data) . unwrap () . into ()) }", - ], - }, - ), - ), - instantiate_fn: , - deserialize_pipeline: Some( - Operator( - Operator { - path: "map", - args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > (& b) . unwrap ()) }", - ], - }, - ), - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , hydroflow_plus_test :: cluster :: paxos :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (_count , p2b)) | ReplicaPayload { seq : p2b . slot , key : p2b . value . key , value : p2b . value . value } }), - input: AntiJoin( - Tee { - inner: RefCell { - value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (i32 , (usize , hydroflow_plus_test :: cluster :: paxos :: P2b)) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | (_slot , (count , _p2b)) | * count > f }), - input: Tee { - inner: RefCell { - value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : ClientPayload { key : 0 , value : "0" . to_string () } }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), - input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), - input: Tee { - inner: RefCell { - value: Union( - Tee { - inner: RefCell { - value: CycleSource { - ident: Ident { - sym: cycle_2, - }, - location_kind: Cluster( - 0, - ), - }, - }, - }, - CycleSource { - ident: Ident { - sym: cycle_6, - }, - location_kind: Cluster( - 0, - ), - }, - ), - }, - }, - }, - }, + value: 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( + Union( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_1, }, + location_kind: Cluster( + 2, + ), }, }, }, }, - CycleSource { - ident: Ident { - sym: cycle_5, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , 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 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_2, + }, + location_kind: Cluster( + 2, + ), + }, + }, }, - location_kind: Cluster( - 0, - ), }, ), - }, - }, + Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + }, + ), + ), }, - }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), }, }, - CycleSource { - ident: Ident { - sym: cycle_0, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), + ), }, - location_kind: Cluster( - 3, - ), }, ), - ), - }, - }, - Union( - CycleSource { - ident: Ident { - sym: cycle_1, }, - location_kind: Cluster( - 3, - ), }, - Source { - source: Iter( - { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; - 1 } ; [e] }, - ), - location_kind: Cluster( - 3, - ), + }, + }, + ), + }, + }, + }, + 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 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: RefCell { + value: CycleSource { + ident: Ident { + sym: cycle_4, }, - ), - ), + location_kind: Cluster( + 2, + ), + }, + }, }, }, }, + }, + ), + }, + Tee { + inner: RefCell { + value: Union( + CycleSource { + ident: Ident { + sym: cycle_3, + }, + location_kind: Cluster( + 2, + ), + }, + Persist( + Source { + source: Iter( + { use hydroflow_plus :: __staged :: builder :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e] }, + ), + location_kind: Cluster( + 2, + ), + }, + ), ), }, }, - }, + ), }, - ), + }, }, }, }, - }, - ), + ), + }, }, }, - }, + ), }, - }, + ), }, ]