diff --git a/hydro_test/examples/paxos.rs b/hydro_test/examples/paxos.rs index 07d24c0ba00..bd8ca2d69ce 100644 --- a/hydro_test/examples/paxos.rs +++ b/hydro_test/examples/paxos.rs @@ -3,6 +3,7 @@ use std::sync::Arc; use hydro_deploy::gcp::GcpNetwork; use hydro_deploy::{Deployment, Host}; use hydro_lang::deploy::TrybuildHost; +use hydro_test::cluster::paxos::PaxosConfig; use tokio::sync::RwLock; type HostCreator = Box Arc>; @@ -43,13 +44,15 @@ async fn main() { let (proposers, acceptors, clients, replicas) = hydro_test::cluster::paxos_bench::paxos_bench( &builder, - f, num_clients_per_node, median_latency_window_size, checkpoint_frequency, - i_am_leader_send_timeout, - i_am_leader_check_timeout, - i_am_leader_check_timeout_delay_multiplier, + PaxosConfig { + f, + i_am_leader_send_timeout, + i_am_leader_check_timeout, + i_am_leader_check_timeout_delay_multiplier, + }, ); let rustflags = "-C opt-level=3 -C codegen-units=1 -C strip=none -C debuginfo=2 -C lto=off"; diff --git a/hydro_test/src/cluster/bench_client.rs b/hydro_test/src/cluster/bench_client.rs new file mode 100644 index 00000000000..788a61c8da2 --- /dev/null +++ b/hydro_test/src/cluster/bench_client.rs @@ -0,0 +1,164 @@ +use std::cell::RefCell; +use std::rc::Rc; +use std::time::Duration; + +use hydro_lang::*; +use tokio::time::Instant; + +pub struct Client {} + +pub fn bench_client<'a>( + clients: &Cluster<'a, Client>, + transaction_cycle: impl FnOnce( + Stream<(u32, u32), Cluster<'a, Client>, Unbounded>, + ) -> Stream<(u32, u32), Cluster<'a, Client>, Unbounded, NoOrder>, + num_clients_per_node: usize, + median_latency_window_size: usize, +) { + let client_tick = clients.tick(); + // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); + + // Set up an initial set of payloads on the first tick + let start_this_tick = client_tick.singleton_first_tick(q!(())); + + let c_new_payloads_on_start = start_this_tick.clone().flat_map_ordered(q!(move |_| (0 + ..num_clients_per_node) + .map(move |i| ( + (CLUSTER_SELF_ID.raw_id * (num_clients_per_node as u32)) + i as u32, + 0 + )))); + + let (c_to_proposers_complete_cycle, c_to_proposers) = + clients.forward_ref::>(); + let c_received_quorum_payloads = unsafe { + // SAFETY: because the transaction processor is required to handle arbitrary reordering + // across *different* keys, we are safe because delaying a transaction result for a key + // will only affect when the next request for that key is emitted with respect to other + // keys + transaction_cycle(c_to_proposers) + .timestamped(&client_tick) + .tick_batch() + }; + + // Whenever all replicas confirm that a payload was committed, send another payload + let c_new_payloads_when_committed = c_received_quorum_payloads + .clone() + .map(q!(|payload| (payload.0, payload.1 + 1))); + c_to_proposers_complete_cycle.complete( + c_new_payloads_on_start + .chain(unsafe { + // SAFETY: we don't send a new write for the same key until the previous one is committed, + // so this contains only a single write per key, and we don't care about order + // across keys + c_new_payloads_when_committed.assume_ordering::() + }) + .all_ticks() + .drop_timestamp(), + ); + + // Track statistics + let (c_timers_complete_cycle, c_timers) = + client_tick.cycle::>(); + let c_new_timers_when_leader_elected = start_this_tick + .map(q!(|_| Instant::now())) + .flat_map_ordered(q!( + move |now| (0..num_clients_per_node).map(move |virtual_id| (virtual_id, now)) + )); + let c_updated_timers = c_received_quorum_payloads + .clone() + .map(q!(|(key, _prev_count)| (key as usize, Instant::now()))); + let c_new_timers = c_timers + .clone() // Update c_timers in tick+1 so we can record differences during this tick (to track latency) + .chain(c_new_timers_when_leader_elected) + .chain(c_updated_timers.clone()) + .reduce_keyed_commutative(q!(|curr_time, new_time| { + if new_time > *curr_time { + *curr_time = new_time; + } + })); + c_timers_complete_cycle.complete_next_tick(c_new_timers); + + let c_stats_output_timer = unsafe { + // SAFETY: intentionally sampling statistics + clients + .source_interval(q!(Duration::from_secs(1))) + .timestamped(&client_tick) + .tick_batch() + } + .first(); + + let c_latency_reset = c_stats_output_timer.clone().map(q!(|_| None)).defer_tick(); + + let c_latencies = c_timers + .join(c_updated_timers) + .map(q!(|(_virtual_id, (prev_time, curr_time))| Some( + curr_time.duration_since(prev_time) + ))) + .chain(c_latency_reset.into_stream()) + .all_ticks() + .flatten_ordered() + .fold_commutative( + // Create window with ring buffer using vec + wraparound index + // TODO: Would be nice if I could use vec![] instead, but that doesn't work in Hydro with RuntimeData *median_latency_window_size + q!(move || ( + Rc::new(RefCell::new(Vec::::with_capacity( + median_latency_window_size + ))), + 0usize, + )), + q!(move |(latencies, write_index), latency| { + let mut latencies_mut = latencies.borrow_mut(); + if *write_index < latencies_mut.len() { + latencies_mut[*write_index] = latency; + } else { + latencies_mut.push(latency); + } + // Increment write index and wrap around + *write_index = (*write_index + 1) % median_latency_window_size; + }), + ) + .map(q!(|(latencies, _)| latencies)); + + let c_throughput_new_batch = c_received_quorum_payloads + .clone() + .count() + .continue_unless(c_stats_output_timer.clone()) + .map(q!(|batch_size| (batch_size, false))); + + let c_throughput_reset = c_stats_output_timer + .clone() + .map(q!(|_| (0, true))) + .defer_tick(); + + let c_throughput = c_throughput_new_batch + .union(c_throughput_reset) + .all_ticks() + .fold( + q!(|| 0), + q!(|total, (batch_size, reset)| { + if reset { + *total = 0; + } else { + *total += batch_size; + } + }), + ); + + unsafe { + // SAFETY: intentionally sampling statistics + c_latencies.zip(c_throughput).latest_tick() + } + .continue_if(c_stats_output_timer) + .all_ticks() + .for_each(q!(move |(latencies, throughput)| { + let mut latencies_mut = latencies.borrow_mut(); + if latencies_mut.len() > 0 { + let middle_idx = latencies_mut.len() / 2; + let (_, median, _) = latencies_mut.select_nth_unstable(middle_idx); + println!("Median latency: {}ms", median.as_micros() as f64 / 1000.0); + } + + println!("Throughput: {} requests/s", throughput); + })); + // End track statistics +} diff --git a/hydro_test/src/cluster/paxos_kv.rs b/hydro_test/src/cluster/kv_replica.rs similarity index 74% rename from hydro_test/src/cluster/paxos_kv.rs rename to hydro_test/src/cluster/kv_replica.rs index ea5084ace78..946a618182b 100644 --- a/hydro_test/src/cluster/paxos_kv.rs +++ b/hydro_test/src/cluster/kv_replica.rs @@ -6,8 +6,6 @@ use hydro_lang::*; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; -use super::paxos::{paxos_core, Acceptor, Ballot, Proposer}; - pub struct Replica {} pub trait KvKey: Serialize + DeserializeOwned + Hash + Eq + Clone + Debug {} @@ -41,70 +39,22 @@ impl PartialOrd for SequencedKv { } } -/// Sets up a linearizable key-value store using Paxos. -/// -/// # Safety -/// Notifications for leader election are non-deterministic. When the leader is changing, -/// writes may be dropped by the old leader. -#[expect( - clippy::type_complexity, - clippy::too_many_arguments, - reason = "internal paxos code // TODO" -)] -pub unsafe fn paxos_kv<'a, K: KvKey, V: KvValue>( - proposers: &Cluster<'a, Proposer>, - acceptors: &Cluster<'a, Acceptor>, - replicas: &Cluster<'a, Replica>, - c_to_proposers: Stream, Cluster<'a, Proposer>, Unbounded>, - f: usize, - i_am_leader_send_timeout: u64, - i_am_leader_check_timeout: u64, - i_am_leader_check_timeout_delay_multiplier: usize, - checkpoint_frequency: usize, -) -> ( - Stream, Unbounded>, - Stream, Cluster<'a, Replica>, Unbounded>, -) { - let (r_to_acceptors_checkpoint_complete_cycle, r_to_acceptors_checkpoint) = - replicas.forward_ref::>(); - - let (p_to_clients_new_leader_elected, p_to_replicas) = unsafe { - // SAFETY: Leader election non-determinism and non-deterministic dropping of writes is documented. - paxos_core( - proposers, - acceptors, - r_to_acceptors_checkpoint.broadcast_bincode(acceptors), - c_to_proposers, - f, - i_am_leader_send_timeout, - i_am_leader_check_timeout, - i_am_leader_check_timeout_delay_multiplier, - ) - }; - - let (r_to_acceptors_checkpoint_new, r_new_processed_payloads) = replica( - replicas, - p_to_replicas - .map(q!(|(slot, kv)| SequencedKv { seq: slot, kv })) - .broadcast_bincode_interleaved(replicas), - checkpoint_frequency, - ); - - r_to_acceptors_checkpoint_complete_cycle.complete(r_to_acceptors_checkpoint_new); - - (p_to_clients_new_leader_elected, r_new_processed_payloads) -} - // Replicas. All relations for replicas will be prefixed with r. Expects ReplicaPayload on p_to_replicas, outputs a stream of (client address, ReplicaPayload) after processing. #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -pub fn replica<'a, K: KvKey, V: KvValue>( +pub fn kv_replica<'a, K: KvKey, V: KvValue>( replicas: &Cluster<'a, Replica>, - p_to_replicas: Stream, Cluster<'a, Replica>, Unbounded, NoOrder>, + p_to_replicas: impl Into< + Stream<(usize, Option>), Cluster<'a, Replica>, Unbounded, NoOrder>, + >, checkpoint_frequency: usize, ) -> ( Stream, Unbounded>, Stream, Cluster<'a, Replica>, Unbounded>, ) { + let p_to_replicas = p_to_replicas + .into() + .map(q!(|(slot, kv)| SequencedKv { seq: slot, kv })); + let replica_tick = replicas.tick(); let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = replica_tick.cycle(); diff --git a/hydro_test/src/cluster/mod.rs b/hydro_test/src/cluster/mod.rs index b4e5be9bb00..9b3e74eff9d 100644 --- a/hydro_test/src/cluster/mod.rs +++ b/hydro_test/src/cluster/mod.rs @@ -1,8 +1,10 @@ +pub mod bench_client; pub mod compute_pi; +pub mod kv_replica; pub mod many_to_many; pub mod map_reduce; pub mod paxos; pub mod paxos_bench; -pub mod paxos_kv; +pub mod paxos_with_client; pub mod simple_cluster; pub mod two_pc; diff --git a/hydro_test/src/cluster/paxos.rs b/hydro_test/src/cluster/paxos.rs index 90dbe97a63c..2d40915ea9f 100644 --- a/hydro_test/src/cluster/paxos.rs +++ b/hydro_test/src/cluster/paxos.rs @@ -12,6 +12,18 @@ use serde::{Deserialize, Serialize}; pub struct Proposer {} pub struct Acceptor {} +#[derive(Clone, Copy)] +pub struct PaxosConfig { + /// Maximum number of faulty nodes + pub f: usize, + /// How often to send "I am leader" heartbeats + pub i_am_leader_send_timeout: u64, + /// How often to check if the leader has expired + pub i_am_leader_check_timeout: u64, + /// Initial delay, multiplied by proposer pid, to stagger proposers checking for timeouts + pub i_am_leader_check_timeout_delay_multiplier: usize, +} + pub trait PaxosPayload: Serialize + DeserializeOwned + PartialEq + Eq + Clone + Debug {} impl PaxosPayload for T {} @@ -64,11 +76,7 @@ struct P2a

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

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

), Cluster<'a, Proposer>, Unbounded, NoOrder> { + unsafe { + // SAFETY: Non-deterministic leader notifications are handled in `cur_leader_id`. We do not + // care about the order in which key writes are processed, which is the non-determinism in + // `sequenced_payloads`. + + paxos_core( + proposers, + acceptors, + replica_checkpoint, + |new_leader_elected| { + let cur_leader_id = new_leader_elected + .broadcast_bincode_interleaved(clients) + .inspect(q!(|ballot| println!( + "Client notified that leader was elected: {:?}", + ballot + ))) + .max() + .map(q!(|ballot: Ballot| ballot.proposer_id)); + + let payloads_at_proposer = { + // SAFETY: the risk here is that we send a batch of requests + // with a stale leader ID, but because the leader ID comes from the + // network there is no way to guarantee that it is up to date. This + // is documented non-determinism. + + let client_tick = clients.tick(); + let payload_batch = payloads.timestamped(&client_tick).tick_batch(); + + let latest_leader = cur_leader_id.timestamped(&client_tick).latest_tick(); + + let (unsent_payloads_complete, unsent_payloads) = + client_tick.cycle::>(); + + let all_payloads = unsent_payloads.chain(payload_batch); + + unsent_payloads_complete.complete_next_tick( + all_payloads.clone().continue_unless(latest_leader.clone()), + ); + + all_payloads.cross_singleton(latest_leader).all_ticks() + } + .map(q!(move |(payload, leader_id)| (leader_id, payload))) + .send_bincode_interleaved(proposers); + + let payloads_at_proposer = { + // SAFETY: documented non-determinism in interleaving of client payloads + payloads_at_proposer.assume_ordering() + }; + + payloads_at_proposer + }, + paxos_config, + ) + .1 + } +} diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap index f80be9a238f..eb4f676ebc9 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap @@ -30,7 +30,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -93,7 +93,7 @@ expression: built.ir() sym: cycle_4, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -165,7 +165,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -198,7 +198,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -210,7 +210,7 @@ expression: built.ir() input: Tee { inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , usize) > ({ use hydro_std :: __staged :: quorum :: * ; move | | (0 , 0) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { inner: : Chain( CycleSource { @@ -218,7 +218,7 @@ expression: built.ir() sym: cycle_5, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -226,9 +226,9 @@ expression: built.ir() }, Tee { inner: : Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 1, @@ -239,14 +239,14 @@ expression: built.ir() ), to_key: None, serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > (& data) . unwrap () . into ()) }, + | (id , data) : (hydro_lang :: ClusterId < _ > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > (& data) . unwrap () . into ()) }, ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) > (& b) . unwrap ()) }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >)) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((ballot , max_ballot) , log) | (ballot . proposer_id , (ballot , if ballot == max_ballot { Ok (log) } else { Err (max_ballot) })) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((ballot , max_ballot) , log) | (ballot . proposer_id , (ballot , if ballot == max_ballot { Ok (log) } else { Err (max_ballot) })) }), input: CrossSingleton( CrossSingleton( Tee { @@ -365,7 +365,7 @@ expression: built.ir() sym: cycle_0, }, location_kind: Tick( - 3, + 2, Cluster( 1, ), @@ -398,7 +398,7 @@ expression: built.ir() sym: cycle_5, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -419,7 +419,7 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -429,19 +429,19 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), input: CrossSingleton( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), input: Tee { inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | ((quorum_ballot , quorum_accepted) , my_ballot) | if quorum_ballot == my_ballot { Some (quorum_accepted) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | ((quorum_ballot , quorum_accepted) , my_ballot) | if quorum_ballot == my_ballot { Some (quorum_accepted) } else { None } }), input: CrossSingleton( Reduce { - f: { let key_fn = stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }) ; move | curr , new | { if key_fn (& new) > key_fn (& * curr) { * curr = new ; } } }, + f: { let key_fn = stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }) ; move | curr , new | { if key_fn (& new) > key_fn (& * curr) { * curr = new ; } } }, input: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | | vec ! [] }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | logs , log | { logs . push (log) ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | | vec ! [] }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | logs , log | { logs . push (log) ; } }), input: Persist( FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >)) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (v) => Some ((key , v)) , Err (_) => None , } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (v) => Some ((key , v)) , Err (_) => None , } }), input: AntiJoin( AntiJoin( Tee { @@ -459,7 +459,7 @@ expression: built.ir() sym: cycle_6, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -510,19 +510,145 @@ expression: built.ir() input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , ballot) | ballot }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), input: Tee { inner: , }, }, }, }, + CycleSink { + ident: Ident { + sym: cycle_1, + }, + location_kind: Tick( + 7, + Cluster( + 2, + ), + ), + input: DeferTick( + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: : Chain( + CycleSource { + ident: Ident { + sym: cycle_1, + }, + location_kind: Tick( + 7, + Cluster( + 2, + ), + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; move | (key , value) | KvPayload { key , value : (CLUSTER_SELF_ID__free , value) } }), + input: CycleSource { + ident: Ident { + sym: cycle_0, + }, + location_kind: Cluster( + 2, + ), + }, + }, + ), + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: stream :: * ; | c | * c == 0 }), + input: Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; | ballot : Ballot | ballot . proposer_id }), + input: Reduce { + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), + input: Persist( + Inspect { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 0, + ), + from_key: None, + to_location: Cluster( + 2, + ), + to_key: None, + serialize_fn: Some( + | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }, + ), + instantiate_fn: , + deserialize_fn: Some( + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }, + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: bench_client :: Client >] > (__hydro_lang_cluster_ids_2) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: , + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), + input: Tee { + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), + input: CrossSingleton( + Tee { + inner: , + }, + Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), + input: Filter { + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: optional :: * ; | c | * c == 0 }), + input: Fold { + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), + input: DeferTick( + Tee { + inner: , + }, + ), + }, + }, + }, + ), + }, + }, + }, + ), + }, + }, + }, + }, + }, + ), + }, + }, + }, + }, + }, + }, + ), + }, + ), + }, CycleSink { ident: Ident { sym: cycle_7, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -532,20 +658,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , base_slot) | base_slot + num_payloads }), input: CrossSingleton( Tee { - inner: : Fold { + inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , usize) , (usize , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((index , payload) , base_slot) | (base_slot + index , payload) }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , usize) , (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((index , payload) , base_slot) | (base_slot + index , payload) }), input: CrossSingleton( Enumerate { is_static: false, input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , ()) , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -556,43 +682,20 @@ expression: built.ir() ), to_key: None, serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& data) . unwrap () . into ()) }, + | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > (& data) . unwrap () . into ()) }, ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > (& b) . unwrap ()) }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; move | ((key , value) , leader_id) | (leader_id , KvPayload { key , value : (CLUSTER_SELF_ID__free , value) }) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; move | (payload , leader_id) | (leader_id , payload) }), input: CrossSingleton( - CycleSource { - ident: Ident { - sym: cycle_1, - }, - location_kind: Cluster( - 2, - ), + Tee { + inner: , }, Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot : Ballot | ballot . proposer_id }), - input: Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), - input: CycleSource { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - }, - }, - ), - }, - }, + inner: , }, ), }, @@ -608,27 +711,27 @@ expression: built.ir() }, }, Tee { - inner: : Chain( + inner: : Chain( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), input: Tee { - inner: : Reduce { + inner: : Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >)) , (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , (count , entry)) | (slot , (count , entry . unwrap ())) }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)) , (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , (count , entry)) | (slot , (count , entry . unwrap ())) }), input: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , None) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { if let Some (curr_entry_payload) = & mut curr_entry . 1 { let same_values = new_entry . value == curr_entry_payload . value ; let higher_ballot = new_entry . ballot > curr_entry_payload . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry_payload . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry_payload . value = new_entry . value ; } } } else { * curr_entry = (1 , Some (new_entry)) ; } } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , None) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { if let Some (curr_entry_payload) = & mut curr_entry . 1 { let same_values = new_entry . value == curr_entry_payload . value ; let higher_ballot = new_entry . ballot > curr_entry_payload . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry_payload . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry_payload . value = new_entry . value ; } } } else { * curr_entry = (1 , Some (new_entry)) ; } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use hydro_lang :: __staged :: stream :: * ; | d | d }), + f: stageleft :: runtime_support :: fn1_type_hint :: < std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use hydro_lang :: __staged :: stream :: * ; | d | d }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_checkpoint , log) | log }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_checkpoint , log) | log }), input: Tee { - inner: : FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > > ({ use hydro_lang :: __staged :: optional :: * ; | v | v }), + inner: : FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > ({ use hydro_lang :: __staged :: optional :: * ; | v | v }), input: Tee { inner: , }, @@ -649,7 +752,7 @@ expression: built.ir() sym: cycle_7, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -674,7 +777,7 @@ expression: built.ir() }, }, Tee { - inner: , + inner: , }, ), }, @@ -685,7 +788,7 @@ expression: built.ir() sym: cycle_9, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -693,27 +796,27 @@ expression: built.ir() input: DeferTick( Difference( Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)) , core :: option :: Option < (usize , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , usize) > ({ use hydro_std :: __staged :: quorum :: * ; move | | (0 , 0) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { - inner: : Chain( + inner: : Chain( CycleSource { ident: Ident { sym: cycle_8, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), ), }, Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -732,11 +835,11 @@ expression: built.ir() | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) > (& b) . unwrap ()) }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , ((p2a . slot , p2a . ballot) , if p2a . ballot == max_ballot { Ok (()) } else { Err (max_ballot) })) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , ((p2a . slot , p2a . ballot) , if p2a . ballot == max_ballot { Ok (()) } else { Err (max_ballot) })) }), input: CrossSingleton( Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -747,26 +850,26 @@ expression: built.ir() ), to_key: None, serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > (& data) . unwrap () . into ()) }, + | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > (& data) . unwrap () . into ()) }, ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > (& b) . unwrap ()) }, ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor >] > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor >] > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , ballot) , value) | P2a { ballot , slot , value } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , ballot) , value) | P2a { ballot , slot , value } }), input: Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , ()) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , ()) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( Chain( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , payload) , ballot) | ((slot , ballot) , Some (payload)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , payload) , ballot) | ((slot , ballot) , Some (payload)) }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { inner: , @@ -774,29 +877,29 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2a | ((p2a . slot , p2a . ballot) , p2a . value) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2a | ((p2a . slot , p2a . ballot) , p2a . value) }), input: Chain( FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < usize >) , core :: option :: Option < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | (((slot , (count , entry)) , ballot) , checkpoint) | { if count > f__free { return None ; } else if let Some (checkpoint) = checkpoint { if slot <= checkpoint { return None ; } } Some (P2a { ballot , slot , value : entry . value , }) } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < usize >) , core :: option :: Option < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | (((slot , (count , entry)) , ballot) , checkpoint) | { if count > f__free { return None ; } else if let Some (checkpoint) = checkpoint { if slot <= checkpoint { return None ; } } Some (P2a { ballot , slot , value : entry . value , }) } }), input: CrossSingleton( CrossSingleton( Tee { - inner: , + inner: , }, Tee { inner: , }, ), Tee { - inner: : Chain( + inner: : Chain( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < usize > > ({ use hydro_lang :: __staged :: optional :: * ; | v | Some (v) }), input: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (checkpoint , _log) | checkpoint }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (checkpoint , _log) | checkpoint }), input: Tee { - inner: , + inner: , }, }, }, @@ -816,24 +919,24 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , ballot) | P2a { ballot , slot , value : None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , ballot) | P2a { ballot , slot , value : None } }), input: CrossSingleton( Difference( FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < usize >) , std :: ops :: Range < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (max_slot , checkpoint) | { if let Some (checkpoint) = checkpoint { (checkpoint + 1) .. max_slot } else { 0 .. max_slot } } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { - inner: , + inner: , }, }, ), @@ -874,10 +977,10 @@ expression: built.ir() }, }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)) , core :: option :: Option < (usize , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; let max__free = 3usize ; move | (key , (success , error)) | if (success + error) >= max__free { Some (key) } else { None } }), input: Tee { - inner: , + inner: , }, }, }, @@ -889,7 +992,7 @@ expression: built.ir() sym: cycle_8, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -897,10 +1000,10 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), ), @@ -910,7 +1013,7 @@ expression: built.ir() sym: cycle_10, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -918,38 +1021,38 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Chain( + inner: : Chain( CycleSource { ident: Ident { sym: cycle_10, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), ), }, Tee { - inner: , + inner: , }, ), }, Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) , (usize , hydro_test :: cluster :: paxos :: Ballot) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , _) | key }), input: Tee { - inner: : Map { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) > ({ use crate :: __staged :: cluster :: paxos :: * ; | k | (k , ()) }), input: Difference( Tee { - inner: , + inner: , }, CycleSource { ident: Ident { sym: cycle_9, }, location_kind: Tick( - 2, + 1, Cluster( 0, ), @@ -967,21 +1070,21 @@ expression: built.ir() sym: cycle_0, }, location_kind: Tick( - 3, + 2, Cluster( 1, ), ), input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (None , HashMap :: new ()) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { if prev_checkpoint . map (| prev | new_checkpoint > prev) . unwrap_or (true) { for slot in (prev_checkpoint . unwrap_or (0)) .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = Some (new_checkpoint) ; } } CheckpointOrP2a :: P2a (p2a) => { if prev_checkpoint . map (| prev | p2a . slot > prev) . unwrap_or (true) && log . get (& p2a . slot) . map (| prev_p2a : & LogValue < _ > | p2a . ballot > prev_p2a . ballot) . unwrap_or (true) { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (None , HashMap :: new ()) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { if prev_checkpoint . map (| prev | new_checkpoint > prev) . unwrap_or (true) { for slot in (prev_checkpoint . unwrap_or (0)) .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = Some (new_checkpoint) ; } } CheckpointOrP2a :: P2a (p2a) => { if prev_checkpoint . map (| prev | p2a . slot > prev) . unwrap_or (true) && log . get (& p2a . slot) . map (| prev_p2a : & LogValue < _ > | p2a . ballot > prev_p2a . ballot) . unwrap_or (true) { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } } }), input: Persist( Chain( FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { inner: , @@ -989,17 +1092,17 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), input: Delta( Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , usize) , ()) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , usize) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , ()) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { - inner: : ReduceKeyed { + inner: : ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), input: Persist( Network { @@ -1016,7 +1119,7 @@ expression: built.ir() ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < usize > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: kv_replica :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < usize > (& b) . unwrap ()) }, ), input: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor >] > (__hydro_lang_cluster_ids_1) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), @@ -1039,9 +1142,9 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | num_received | if num_received == f__free + 1 { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , usize) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1068,7 +1171,7 @@ expression: built.ir() input: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < ((usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), input: Tee { - inner: , + inner: , }, }, }, @@ -1085,45 +1188,45 @@ expression: built.ir() ), input: DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), input: CrossSingleton( Tee { - inner: : Sort( + inner: : Sort( Chain( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - from_key: None, - to_location: Cluster( - 3, - ), - to_key: None, - serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) >) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& data) . unwrap () . into ()) }, - ), - instantiate_fn: , - deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > (& b) . unwrap ()) }, - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica >] > (__hydro_lang_cluster_ids_3) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (slot , kv) | SequencedKv { seq : slot , kv } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (slot , kv) | SequencedKv { seq : slot , kv } }), + input: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + input: Network { + from_location: Cluster( + 0, + ), + from_key: None, + to_location: Cluster( + 3, + ), + to_key: None, + serialize_fn: Some( + | (id , data) : (hydro_lang :: ClusterId < _ > , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > (& data) . unwrap () . into ()) }, + ), + instantiate_fn: , + deserialize_fn: Some( + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > (& b) . unwrap ()) }, + ), + input: FlatMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica >] > (__hydro_lang_cluster_ids_3) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ())) , (usize , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , _ballot) , (value , _)) | (slot , value) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , _ballot) , (value , _)) | (slot , value) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ())) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > , ())) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , (meta , resp)) | (key , (meta , resp)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , (meta , resp)) | (key , (meta , resp)) }), input: Join( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1147,14 +1250,14 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < usize > , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | v | v }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < usize > , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | v | v }), input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | | None }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | filled_slot , (sorted_payload , highest_seq) | { let expected_next_slot = std :: cmp :: max (filled_slot . map (| v | v + 1) . unwrap_or (0) , highest_seq . map (| v | v + 1) . unwrap_or (0) ,) ; if sorted_payload . seq == expected_next_slot { * filled_slot = Some (sorted_payload . seq) ; } } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | | None }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , () > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | filled_slot , (sorted_payload , highest_seq) | { let expected_next_slot = std :: cmp :: max (filled_slot . map (| v | v + 1) . unwrap_or (0) , highest_seq . map (| v | v + 1) . unwrap_or (0) ,) ; if sorted_payload . seq == expected_next_slot { * filled_slot = Some (sorted_payload . seq) ; } } }), input: CrossSingleton( Tee { - inner: , + inner: , }, Chain( Map { @@ -1203,23 +1306,23 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (_kv_store , highest_seq) | highest_seq }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | | (HashMap :: new () , None) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < usize >) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , () > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (kv_store , last_seq) , payload | { if let Some (kv) = payload . kv { kv_store . insert (kv . key , kv . value) ; } debug_assert ! (payload . seq == (last_seq . map (| s | s + 1) . unwrap_or (0)) , "Hole in log between seq {:?} and {}" , * last_seq , payload . seq) ; * last_seq = Some (payload . seq) ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | | (HashMap :: new () , None) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , () > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (kv_store , last_seq) , payload | { if let Some (kv) = payload . kv { kv_store . insert (kv . key , kv . value) ; } debug_assert ! (payload . seq == (last_seq . map (| s | s + 1) . unwrap_or (0)) , "Hole in log between seq {:?} and {}" , * last_seq , payload . seq) ; * last_seq = Some (payload . seq) ; } }), input: Persist( Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , _) | { sorted_payload } }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), input: CrossSingleton( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, @@ -1243,8 +1346,8 @@ expression: built.ir() ), input: DeferTick( Tee { - inner: : FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , usize) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; let checkpoint_frequency__free = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if max_checkpointed_seq . map (| m | new_highest_seq - m >= checkpoint_frequency__free) . unwrap_or (true) { Some (new_highest_seq) } else { None } }), + inner: : FilterMap { + f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , usize) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; let checkpoint_frequency__free = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if max_checkpointed_seq . map (| m | new_highest_seq - m >= checkpoint_frequency__free) . unwrap_or (true) { Some (new_highest_seq) } else { None } }), input: CrossSingleton( Chain( Map { @@ -1278,7 +1381,7 @@ expression: built.ir() ), ), Tee { - inner: , + inner: , }, ), }, @@ -1293,72 +1396,7 @@ expression: built.ir() 3, ), input: Tee { - inner: , - }, - }, - CycleSink { - ident: Ident { - sym: cycle_0, - }, - location_kind: Cluster( - 2, - ), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), - input: Network { - from_location: Cluster( - 0, - ), - from_key: None, - to_location: Cluster( - 2, - ), - to_key: None, - serialize_fn: Some( - | (id , data) : (hydro_lang :: ClusterId < _ > , hydro_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydro_lang :: runtime_support :: bincode :: serialize :: < hydro_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }, - ), - instantiate_fn: , - deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < hydro_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }, - ), - input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client >] > (__hydro_lang_cluster_ids_2) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: , - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: , - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), - input: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: optional :: * ; | c | * c == 0 }), - input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: DeferTick( - Tee { - inner: , - }, - ), - }, - }, - }, - ), - }, - }, - ), - }, - }, - }, + inner: , }, }, CycleSink { @@ -1366,7 +1404,7 @@ expression: built.ir() sym: cycle_2, }, location_kind: Tick( - 0, + 9, Cluster( 2, ), @@ -1374,21 +1412,21 @@ expression: built.ir() input: DeferTick( AntiJoin( Tee { - inner: : Chain( + inner: : Chain( CycleSource { ident: Ident { sym: cycle_2, }, location_kind: Tick( - 0, + 9, Cluster( 2, ), ), }, Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_kv :: Replica > , ((u32 , u32) , core :: result :: Result < () , () >)) , ((u32 , u32) , core :: result :: Result < () , () >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , ((u32 , u32) , core :: result :: Result < () , () >)) , ((u32 , u32) , core :: result :: Result < () , () >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 3, @@ -1403,14 +1441,14 @@ expression: built.ir() ), instantiate_fn: , deserialize_fn: Some( - | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_kv :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < ((u32 , u32) , core :: result :: Result < () , () >) > (& b) . unwrap ()) }, + | res | { let (id , b) = res . unwrap () ; (hydro_lang :: ClusterId :: < hydro_test :: cluster :: kv_replica :: Replica > :: from_raw (id) , hydro_lang :: runtime_support :: bincode :: deserialize :: < ((u32 , u32) , core :: result :: Result < () , () >) > (& b) . unwrap ()) }, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , ((u32 , u32) , core :: result :: Result < () , () >)) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . 0 , ((payload . key , payload . value . 1) , Ok (()))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , ((u32 , u32) , core :: result :: Result < () , () >)) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . 0 , ((payload . key , payload . value . 1) , Ok (()))) }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos_kv :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > , core :: option :: Option < hydro_test :: cluster :: paxos_kv :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos_bench :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos_kv :: * ; | payload | payload . kv }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | payload | payload . kv }), input: Tee { - inner: , + inner: , }, }, }, @@ -1420,14 +1458,14 @@ expression: built.ir() ), }, Tee { - inner: : FilterMap { + inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , (usize , usize)) , core :: option :: Option < (u32 , u32) > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { - inner: : FoldKeyed { + inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , usize) > ({ use hydro_std :: __staged :: quorum :: * ; move | | (0 , 0) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < () , () > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { - inner: , + inner: , }, }, }, @@ -1438,33 +1476,30 @@ expression: built.ir() }, CycleSink { ident: Ident { - sym: cycle_1, + sym: cycle_0, }, location_kind: Cluster( 2, ), input: Chain( FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; let num_clients_per_node__free = 1usize ; move | _ | (0 .. num_clients_per_node__free) . map (move | i | ((CLUSTER_SELF_ID__free . raw_id * (num_clients_per_node__free as u32)) + i as u32 , 0)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: bench_client :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; let num_clients_per_node__free = 1usize ; move | _ | (0 .. num_clients_per_node__free) . map (move | i | ((CLUSTER_SELF_ID__free . raw_id * (num_clients_per_node__free as u32)) + i as u32 , 0)) }), input: Tee { - inner: : Reduce { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < () , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | * curr = new }), - input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | () }), - input: Delta( - Tee { - inner: , - }, - ), - }, + inner: : Source { + source: Iter( + { use hydro_lang :: __staged :: location :: tick :: * ; let e__free = { use crate :: __staged :: cluster :: bench_client :: * ; () } ; [e__free] }, + ), + location_kind: Cluster( + 2, + ), }, }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . 0 , payload . 1 + 1) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (u32 , u32) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | payload | (payload . 0 , payload . 1 + 1) }), input: Tee { - inner: : Tee { - inner: , + inner: : Tee { + inner: , }, }, }, @@ -1475,23 +1510,23 @@ expression: built.ir() sym: cycle_3, }, location_kind: Tick( - 1, + 0, Cluster( 2, ), ), input: DeferTick( ReduceKeyed { - f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < tokio :: time :: Instant , tokio :: time :: Instant , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_time , new_time | { if new_time > * curr_time { * curr_time = new_time ; } } }), + f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < tokio :: time :: Instant , tokio :: time :: Instant , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; | curr_time , new_time | { if new_time > * curr_time { * curr_time = new_time ; } } }), input: Chain( Chain( Tee { - inner: : CycleSource { + inner: : CycleSource { ident: Ident { sym: cycle_3, }, location_kind: Tick( - 1, + 0, Cluster( 2, ), @@ -1499,20 +1534,20 @@ expression: built.ir() }, }, FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let num_clients_per_node__free = 1usize ; move | now | (0 .. num_clients_per_node__free) . map (move | virtual_id | (virtual_id , now)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: bench_client :: * ; let num_clients_per_node__free = 1usize ; move | now | (0 .. num_clients_per_node__free) . map (move | virtual_id | (virtual_id , now)) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < () , tokio :: time :: Instant > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | Instant :: now () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < () , tokio :: time :: Instant > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | Instant :: now () }), input: Tee { - inner: , + inner: , }, }, }, ), Tee { - inner: : Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (usize , tokio :: time :: Instant) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (key , _prev_count) | (key as usize , Instant :: now ()) }), + inner: : Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (usize , tokio :: time :: Instant) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (key , _prev_count) | (key as usize , Instant :: now ()) }), input: Tee { - inner: , + inner: , }, }, }, @@ -1521,38 +1556,38 @@ expression: built.ir() ), }, ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; move | (latencies , throughput) | { let mut latencies_mut = latencies . borrow_mut () ; if latencies_mut . len () > 0 { let middle_idx = latencies_mut . len () / 2 ; let (_ , median , _) = latencies_mut . select_nth_unstable (middle_idx) ; println ! ("Median latency: {}ms" , median . as_micros () as f64 / 1000.0) ; } println ! ("Throughput: {} requests/s" , throughput) ; } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; move | (latencies , throughput) | { let mut latencies_mut = latencies . borrow_mut () ; if latencies_mut . len () > 0 { let middle_idx = latencies_mut . len () / 2 ; let (_ , median , _) = latencies_mut . select_nth_unstable (middle_idx) ; println ! ("Median latency: {}ms" , median . as_micros () as f64 / 1000.0) ; } println ! ("Throughput: {} requests/s" , throughput) ; } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , ()) , (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( CrossSingleton( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (latencies , _) | latencies }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (latencies , _) | latencies }), input: Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let median_latency_window_size__free = 1usize ; move | | (Rc :: new (RefCell :: new (Vec :: < Duration > :: with_capacity (median_latency_window_size__free))) , 0usize ,) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , core :: time :: Duration , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let median_latency_window_size__free = 1usize ; move | (latencies , write_index) , latency | { let mut latencies_mut = latencies . borrow_mut () ; if * write_index < latencies_mut . len () { latencies_mut [* write_index] = latency ; } else { latencies_mut . push (latency) ; } * write_index = (* write_index + 1) % median_latency_window_size__free ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) > ({ use crate :: __staged :: cluster :: bench_client :: * ; let median_latency_window_size__free = 1usize ; move | | (Rc :: new (RefCell :: new (Vec :: < Duration > :: with_capacity (median_latency_window_size__free))) , 0usize ,) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , core :: time :: Duration , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; let median_latency_window_size__free = 1usize ; move | (latencies , write_index) , latency | { let mut latencies_mut = latencies . borrow_mut () ; if * write_index < latencies_mut . len () { latencies_mut [* write_index] = latency ; } else { latencies_mut . push (latency) ; } * write_index = (* write_index + 1) % median_latency_window_size__free ; } }), input: Persist( FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < core :: time :: Duration > , core :: option :: Option < core :: time :: Duration > > ({ use hydro_lang :: __staged :: stream :: * ; | d | d }), input: Chain( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (tokio :: time :: Instant , tokio :: time :: Instant)) , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (_virtual_id , (prev_time , curr_time)) | Some (curr_time . duration_since (prev_time)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (tokio :: time :: Instant , tokio :: time :: Instant)) , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (_virtual_id , (prev_time , curr_time)) | Some (curr_time . duration_since (prev_time)) }), input: Join( Tee { - inner: , + inner: , }, Tee { - inner: , + inner: , }, ), }, DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | None }), + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | None }), input: Tee { - inner: : Source { + inner: : Source { source: Stream( - { use hydro_lang :: __staged :: location :: * ; let interval__free = { use crate :: __staged :: cluster :: paxos_bench :: * ; Duration :: from_secs (1) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval (interval__free)) }, + { use hydro_lang :: __staged :: location :: * ; let interval__free = { use crate :: __staged :: cluster :: bench_client :: * ; Duration :: from_secs (1) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval (interval__free)) }, ), location_kind: Cluster( 2, @@ -1567,12 +1602,12 @@ expression: built.ir() }, }, Fold { - init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , bool) , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | total , (batch_size , reset) | { if reset { * total = 0 ; } else { * total += batch_size ; } } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: bench_client :: * ; | | 0 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , bool) , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; | total , (batch_size , reset) | { if reset { * total = 0 ; } else { * total += batch_size ; } } }), input: Persist( Chain( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < usize , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | batch_size | (batch_size , false) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < usize , (usize , bool) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | batch_size | (batch_size , false) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -1580,7 +1615,7 @@ expression: built.ir() init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (u32 , u32) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, Map { @@ -1591,7 +1626,7 @@ expression: built.ir() init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { - inner: , + inner: , }, }, }, @@ -1601,9 +1636,9 @@ expression: built.ir() }, DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , (usize , bool) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | _ | (0 , true) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , (usize , bool) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | (0 , true) }), input: Tee { - inner: , + inner: , }, }, ), @@ -1614,7 +1649,7 @@ expression: built.ir() Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { - inner: , + inner: , }, }, ),