From 74e602a2428690db964d5816c5636d6c74011888 Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 8 Oct 2024 12:57:36 -0700 Subject: [PATCH] feat(hydroflow_plus)!: strongly-typed runtime cluster IDs Instead of `u32`s everywhere, we now have a `ClusterId` type that ensures that cluster IDs are not misused. --- docs/docs/hydroflow_plus/clusters.mdx | 6 +- hydroflow_plus/src/builder/mod.rs | 53 +- hydroflow_plus/src/deploy/deploy_graph.rs | 3 +- hydroflow_plus/src/deploy/deploy_runtime.rs | 7 +- hydroflow_plus/src/deploy/macro_runtime.rs | 2 +- hydroflow_plus/src/lib.rs | 4 +- hydroflow_plus/src/location.rs | 134 ++- hydroflow_plus/src/staging_util.rs | 14 + hydroflow_plus/src/stream.rs | 49 +- .../src/cluster/many_to_many.rs | 2 +- hydroflow_plus_test/src/cluster/map_reduce.rs | 5 +- hydroflow_plus_test/src/cluster/paxos.rs | 80 +- .../src/cluster/paxos_bench.rs | 23 +- .../src/cluster/simple_cluster.rs | 14 +- ...ter__compute_pi__tests__compute_pi_ir.snap | 4 +- ..._tests__compute_pi_ir@surface_graph_1.snap | 4 +- ...er__many_to_many__tests__many_to_many.snap | 8 +- ...ter__map_reduce__tests__map_reduce_ir.snap | 8 +- ..._tests__map_reduce_ir@surface_graph_0.snap | 8 +- ...cluster__paxos_bench__tests__paxos_ir.snap | 780 +++++++++--------- ...simple_cluster__tests__simple_cluster.snap | 18 +- hydroflow_plus_test/src/cluster/two_pc.rs | 2 +- 22 files changed, 706 insertions(+), 522 deletions(-) create mode 100644 hydroflow_plus/src/staging_util.rs diff --git a/docs/docs/hydroflow_plus/clusters.mdx b/docs/docs/hydroflow_plus/clusters.mdx index 63380857ca5b..5d742324c673 100644 --- a/docs/docs/hydroflow_plus/clusters.mdx +++ b/docs/docs/hydroflow_plus/clusters.mdx @@ -32,7 +32,7 @@ stream.for_each(q!(|x| println!("{}", x))) ## Sending Data Because clusters represent a set of instances, adding networking requires us to specify _which_ instance(s) to send data to. Clusters provide different types depending on if the source or receiver is a cluster or a process. -Elements in a cluster are identified by a **cluster ID** (a `u32`). To get the IDs of all instances in a cluster, use the `ids` method on cluster, which returns a runtime expression of type `&Vec` (which can only be used inside `q!()` or as an argument to `source_iter`). All IDs always are ranging from 0 through the length of the IDs vector. +Elements in a cluster are identified by a **cluster ID** (a `ClusterId` where `C` is the typetag of the cluster). To get the IDs of all instances in a cluster, use the `members` method on cluster, which returns a runtime expression of type `&Vec>` (which can only be used inside `q!()` or as an argument to `source_iter`). All IDs always are ranging from 0 through the length of the IDs vector. This can then be passed into `source_iter` to load the IDs into the graph. ```rust @@ -40,7 +40,7 @@ let stream = process.source_iter(cluster.members()).cloned(); ``` ### One-to-Many -When sending data from a process to a cluster, the source must be a stream of tuples of the form `(u32, T)` and sends each `T` element to the instance with the matching `u32` ID. +When sending data from a process to a cluster, the source must be a stream of tuples of the form `(ClusterId<_>, T)` and sends each `T` element to the instance with the matching ID. This is useful for partitioning data across instances. For example, we can partition a stream of elements in a round-robin fashion by using `enumerate` to add a sequence number to each element, then using `send_bincode` to send each element to the instance with the matching sequence number: ```rust @@ -48,7 +48,7 @@ let cluster_ids = cluster.members(); let stream = process.source_iter(q!(vec![123, 456, 789])) .enumerate() .map(q!(|(i, x)| ( - i % cluster_ids.len() as u32, + ClusterId::from_raw(i % cluster_ids.len() as u32), x ))) .send_bincode(&cluster); diff --git a/hydroflow_plus/src/builder/mod.rs b/hydroflow_plus/src/builder/mod.rs index 3950a08a183e..c30f19883177 100644 --- a/hydroflow_plus/src/builder/mod.rs +++ b/hydroflow_plus/src/builder/mod.rs @@ -9,9 +9,10 @@ use quote::quote; use runtime_support::FreeVariable; use stageleft::*; +use super::staging_util::get_this_crate; use crate::ir::HfPlusLeaf; use crate::location::{Cluster, ExternalProcess, Process}; -use crate::RuntimeContext; +use crate::{ClusterId, RuntimeContext}; pub mod built; pub mod deploy; @@ -31,13 +32,20 @@ pub struct FlowStateInner { pub type FlowState = Rc>; -#[derive(Copy, Clone)] -pub struct ClusterIds<'a> { +pub struct ClusterIds<'a, C> { pub(crate) id: usize, - pub(crate) _phantom: PhantomData<&'a mut &'a Vec>, + pub(crate) _phantom: PhantomData<&'a mut &'a C>, } -impl<'a> FreeVariable<&'a Vec> for ClusterIds<'a> { +impl<'a, C> Clone for ClusterIds<'a, C> { + fn clone(&self) -> Self { + *self + } +} + +impl<'a, C> Copy for ClusterIds<'a, C> {} + +impl<'a, C> FreeVariable<&'a Vec>> for ClusterIds<'a, C> { fn to_tokens(self) -> (Option, Option) where Self: Sized, @@ -46,19 +54,33 @@ impl<'a> FreeVariable<&'a Vec> for ClusterIds<'a> { &format!("__hydroflow_plus_cluster_ids_{}", self.id), Span::call_site(), ); - (None, Some(quote! { #ident })) + let root = get_this_crate(); + let c_type = quote_type::(); + ( + None, + Some( + quote! { unsafe { ::std::mem::transmute::<_, &::std::vec::Vec<#root::ClusterId<#c_type>>>(#ident) } }, + ), + ) } } -impl<'a> Quoted<'a, &'a Vec> for ClusterIds<'a> {} +impl<'a, C> Quoted<'a, &'a Vec>> for ClusterIds<'a, C> {} -#[derive(Copy, Clone)] -pub(crate) struct ClusterSelfId<'a> { +pub(crate) struct ClusterSelfId<'a, C> { pub(crate) id: usize, - pub(crate) _phantom: PhantomData<&'a mut &'a u32>, + pub(crate) _phantom: PhantomData<&'a mut &'a C>, } -impl<'a> FreeVariable for ClusterSelfId<'a> { +impl<'a, C> Clone for ClusterSelfId<'a, C> { + fn clone(&self) -> Self { + *self + } +} + +impl<'a, C> Copy for ClusterSelfId<'a, C> {} + +impl<'a, C> FreeVariable> for ClusterSelfId<'a, C> { fn to_tokens(self) -> (Option, Option) where Self: Sized, @@ -67,11 +89,16 @@ impl<'a> FreeVariable for ClusterSelfId<'a> { &format!("__hydroflow_plus_cluster_self_id_{}", self.id), Span::call_site(), ); - (None, Some(quote! { #ident })) + let root = get_this_crate(); + let c_type: syn::Type = quote_type::(); + ( + None, + Some(quote! { #root::ClusterId::<#c_type>::from_raw(#ident) }), + ) } } -impl<'a> Quoted<'a, u32> for ClusterSelfId<'a> {} +impl<'a, C> Quoted<'a, ClusterId> for ClusterSelfId<'a, C> {} pub struct FlowBuilder<'a> { flow_state: FlowState, diff --git a/hydroflow_plus/src/deploy/deploy_graph.rs b/hydroflow_plus/src/deploy/deploy_graph.rs index e7fecc6e7167..5b9da95314ce 100644 --- a/hydroflow_plus/src/deploy/deploy_graph.rs +++ b/hydroflow_plus/src/deploy/deploy_graph.rs @@ -14,7 +14,7 @@ use hydro_deploy::hydroflow_crate::tracing_options::TracingOptions; use hydro_deploy::hydroflow_crate::HydroflowCrateService; use hydro_deploy::{CustomService, Deployment, Host, HydroflowCrate}; use hydroflow::futures::StreamExt; -use hydroflow::util::deploy::ConnectedSource; +use hydroflow::util::deploy::{ConnectedSink, ConnectedSource}; use nameof::name_of; use serde::de::DeserializeOwned; use serde::Serialize; @@ -29,7 +29,6 @@ use super::trybuild::{compile_graph_trybuild, create_trybuild}; use super::{ClusterSpec, Deploy, ExternalSpec, Node, ProcessSpec, RegisterPort}; use crate::futures::SinkExt; use crate::lang::graph::HydroflowGraph; -use crate::util::deploy::ConnectedSink; pub struct HydroDeploy {} diff --git a/hydroflow_plus/src/deploy/deploy_runtime.rs b/hydroflow_plus/src/deploy/deploy_runtime.rs index afbf9250e0f0..ad81b8b76d22 100644 --- a/hydroflow_plus/src/deploy/deploy_runtime.rs +++ b/hydroflow_plus/src/deploy/deploy_runtime.rs @@ -1,11 +1,10 @@ use std::collections::HashMap; -use serde::{Deserialize, Serialize}; -use stageleft::{q, Quoted, RuntimeData}; - -use crate::util::deploy::{ +use hydroflow::util::deploy::{ ConnectedDemux, ConnectedDirect, ConnectedSink, ConnectedSource, ConnectedTagged, DeployPorts, }; +use serde::{Deserialize, Serialize}; +use stageleft::{q, Quoted, RuntimeData}; #[derive(Default, Serialize, Deserialize)] pub struct HydroflowPlusMeta { diff --git a/hydroflow_plus/src/deploy/macro_runtime.rs b/hydroflow_plus/src/deploy/macro_runtime.rs index 4f4e395f45da..2310c5743dc1 100644 --- a/hydroflow_plus/src/deploy/macro_runtime.rs +++ b/hydroflow_plus/src/deploy/macro_runtime.rs @@ -2,12 +2,12 @@ use std::cell::RefCell; use std::pin::Pin; use std::rc::Rc; +use hydroflow::util::deploy::DeployPorts; use stageleft::{Quoted, RuntimeData}; use super::HydroflowPlusMeta; use crate::deploy::{ClusterSpec, Deploy, ExternalSpec, Node, ProcessSpec, RegisterPort}; use crate::lang::graph::HydroflowGraph; -use crate::util::deploy::DeployPorts; pub struct DeployRuntime {} diff --git a/hydroflow_plus/src/lib.rs b/hydroflow_plus/src/lib.rs index 7c986128bd8f..3f3c5fbb109c 100644 --- a/hydroflow_plus/src/lib.rs +++ b/hydroflow_plus/src/lib.rs @@ -25,7 +25,7 @@ pub mod singleton; pub use singleton::{Optional, Singleton}; pub mod location; -pub use location::{Cluster, Location, Process}; +pub use location::{Cluster, ClusterId, Location, Process}; pub mod deploy; pub use deploy::{ClusterSpec, Deploy, ProcessSpec}; @@ -43,6 +43,8 @@ pub mod profiler; pub mod properties; +mod staging_util; + #[derive(Clone)] pub struct RuntimeContext<'a> { _phantom: PhantomData<&'a mut &'a ()>, diff --git a/hydroflow_plus/src/location.rs b/hydroflow_plus/src/location.rs index b4a6b7a9ef5f..da9a5978dd8d 100644 --- a/hydroflow_plus/src/location.rs +++ b/hydroflow_plus/src/location.rs @@ -1,3 +1,5 @@ +use std::fmt::{Debug, Display}; +use std::hash::Hash; use std::marker::PhantomData; use std::time::Duration; @@ -6,8 +8,8 @@ use hydroflow::futures::stream::Stream as FuturesStream; use hydroflow::{tokio, tokio_stream}; use proc_macro2::Span; use serde::de::DeserializeOwned; -use serde::Serialize; -use stageleft::{q, Quoted}; +use serde::{Deserialize, Serialize}; +use stageleft::{q, quote_type, Quoted}; use super::builder::{ClusterIds, ClusterSelfId, FlowState}; use crate::cycle::{CycleCollection, CycleCollectionWithInitial}; @@ -366,7 +368,7 @@ impl<'a, P> ExternalProcess<'a, P> { to_key: None, serialize_pipeline: None, instantiate_fn: crate::ir::DebugInstantiate::Building(), - deserialize_pipeline: Some(crate::stream::deserialize_bincode::(false)), + deserialize_pipeline: Some(crate::stream::deserialize_bincode::(None)), input: Box::new(HfPlusNode::Source { source: HfPlusSource::ExternalNetwork(), location_kind: LocationId::ExternalProcess(self.id), @@ -403,6 +405,98 @@ impl<'a, P> Location<'a> for Process<'a, P> { } } +#[repr(transparent)] +pub struct ClusterId { + pub raw_id: u32, + pub(crate) _phantom: PhantomData, +} + +impl Debug for ClusterId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "ClusterId::<{}>({})", + std::any::type_name::(), + self.raw_id + ) + } +} + +impl Display for ClusterId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "ClusterId::<{}>({})", + std::any::type_name::(), + self.raw_id + ) + } +} + +impl Clone for ClusterId { + fn clone(&self) -> Self { + *self + } +} + +impl Copy for ClusterId {} + +impl Serialize for ClusterId { + fn serialize(&self, serializer: S) -> Result + where + S: serde::ser::Serializer, + { + self.raw_id.serialize(serializer) + } +} + +impl<'de, C> Deserialize<'de> for ClusterId { + fn deserialize(deserializer: D) -> Result + where + D: serde::de::Deserializer<'de>, + { + u32::deserialize(deserializer).map(|id| ClusterId { + raw_id: id, + _phantom: PhantomData, + }) + } +} + +impl PartialEq for ClusterId { + fn eq(&self, other: &Self) -> bool { + self.raw_id == other.raw_id + } +} + +impl Eq for ClusterId {} + +impl PartialOrd for ClusterId { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for ClusterId { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.raw_id.cmp(&other.raw_id) + } +} + +impl Hash for ClusterId { + fn hash(&self, state: &mut H) { + self.raw_id.hash(state) + } +} + +impl ClusterId { + pub fn from_raw(id: u32) -> Self { + ClusterId { + raw_id: id, + _phantom: PhantomData, + } + } +} + pub struct Cluster<'a, C> { pub(crate) id: usize, pub(crate) flow_state: FlowState, @@ -410,14 +504,14 @@ pub struct Cluster<'a, C> { } impl<'a, C> Cluster<'a, C> { - pub fn self_id(&self) -> impl Quoted<'a, u32> + Copy + 'a { + pub fn self_id(&self) -> impl Quoted<'a, ClusterId> + Copy + 'a { ClusterSelfId { id: self.id, _phantom: PhantomData, } } - pub fn members(&self) -> impl Quoted<'a, &'a Vec> + Copy + 'a { + pub fn members(&self) -> impl Quoted<'a, &'a Vec>> + Copy + 'a { ClusterIds { id: self.id, _phantom: PhantomData, @@ -450,7 +544,7 @@ pub trait CanSend<'a, To: Location<'a>>: Location<'a> { type Out; fn is_demux() -> bool; - fn is_tagged() -> bool; + fn tagged_type() -> Option; } impl<'a, P1, P2> CanSend<'a, Process<'a, P2>> for Process<'a, P1> { @@ -461,47 +555,47 @@ impl<'a, P1, P2> CanSend<'a, Process<'a, P2>> for Process<'a, P1> { false } - fn is_tagged() -> bool { - false + fn tagged_type() -> Option { + None } } impl<'a, P1, C2> CanSend<'a, Cluster<'a, C2>> for Process<'a, P1> { - type In = (u32, T); + type In = (ClusterId, T); type Out = T; fn is_demux() -> bool { true } - fn is_tagged() -> bool { - false + fn tagged_type() -> Option { + None } } impl<'a, C1, P2> CanSend<'a, Process<'a, P2>> for Cluster<'a, C1> { type In = T; - type Out = (u32, T); + type Out = (ClusterId, T); fn is_demux() -> bool { false } - fn is_tagged() -> bool { - true + fn tagged_type() -> Option { + Some(quote_type::()) } } impl<'a, C1, C2> CanSend<'a, Cluster<'a, C2>> for Cluster<'a, C1> { - type In = (u32, T); - type Out = (u32, T); + type In = (ClusterId, T); + type Out = (ClusterId, T); fn is_demux() -> bool { true } - fn is_tagged() -> bool { - true + fn tagged_type() -> Option { + Some(quote_type::()) } } @@ -513,7 +607,7 @@ impl<'a, P1, E2> CanSend<'a, ExternalProcess<'a, E2>> for Process<'a, P1> { false } - fn is_tagged() -> bool { - false + fn tagged_type() -> Option { + None } } diff --git a/hydroflow_plus/src/staging_util.rs b/hydroflow_plus/src/staging_util.rs new file mode 100644 index 000000000000..86e5088bcb24 --- /dev/null +++ b/hydroflow_plus/src/staging_util.rs @@ -0,0 +1,14 @@ +use proc_macro2::{Span, TokenStream}; +use quote::quote; + +pub fn get_this_crate() -> TokenStream { + let hydroflow_crate = proc_macro_crate::crate_name("hydroflow_plus") + .expect("hydroflow_plus should be present in `Cargo.toml`"); + match hydroflow_crate { + proc_macro_crate::FoundCrate::Itself => quote! { hydroflow_plus }, + proc_macro_crate::FoundCrate::Name(name) => { + let ident = syn::Ident::new(&name, Span::call_site()); + quote! { #ident } + } + } +} diff --git a/hydroflow_plus/src/stream.rs b/hydroflow_plus/src/stream.rs index f5904a486789..094667e1e3fd 100644 --- a/hydroflow_plus/src/stream.rs +++ b/hydroflow_plus/src/stream.rs @@ -7,20 +7,19 @@ use std::rc::Rc; use hydroflow::bytes::Bytes; use hydroflow::futures::Sink; use hydroflow_lang::parse::Pipeline; -use proc_macro2::{Span, TokenStream}; -use quote::quote; use serde::de::DeserializeOwned; use serde::Serialize; use stageleft::{q, IntoQuotedMut, Quoted}; use syn::parse_quote; +use super::staging_util::get_this_crate; use crate::builder::{self, FlowState}; use crate::cycle::{CycleCollection, CycleComplete}; use crate::ir::{DebugInstantiate, HfPlusLeaf, HfPlusNode, HfPlusSource}; use crate::location::{ CanSend, ExternalBincodeStream, ExternalBytesPort, ExternalProcess, Location, LocationId, }; -use crate::{Cluster, Optional, Process, Singleton}; +use crate::{Cluster, ClusterId, Optional, Process, Singleton}; /// Marks the stream as being unbounded, which means that it is not /// guaranteed to be complete in finite time. @@ -636,18 +635,6 @@ impl<'a, K: Eq + Hash, V, N: Location<'a>> Stream<(K, V), Bounded, Tick, N> { } } -fn get_this_crate() -> TokenStream { - let hydroflow_crate = proc_macro_crate::crate_name("hydroflow_plus") - .expect("hydroflow_plus should be present in `Cargo.toml`"); - match hydroflow_crate { - proc_macro_crate::FoundCrate::Itself => quote! { hydroflow_plus }, - proc_macro_crate::FoundCrate::Name(name) => { - let ident = syn::Ident::new(&name, Span::call_site()); - quote! { #ident } - } - } -} - fn serialize_bincode(is_demux: bool) -> Pipeline { let root = get_this_crate(); @@ -655,8 +642,8 @@ fn serialize_bincode(is_demux: bool) -> Pipeline { if is_demux { parse_quote! { - map(|(id, data)| { - (id, #root::runtime_support::bincode::serialize::<#t_type>(&data).unwrap().into()) + map(|(id, data): (#root::ClusterId<_>, #t_type)| { + (id.raw_id, #root::runtime_support::bincode::serialize::<#t_type>(&data).unwrap().into()) }) } } else { @@ -668,16 +655,16 @@ fn serialize_bincode(is_demux: bool) -> Pipeline { } } -pub(super) fn deserialize_bincode(tagged: bool) -> Pipeline { +pub(super) fn deserialize_bincode(tagged: Option) -> Pipeline { let root = get_this_crate(); let t_type: syn::Type = stageleft::quote_type::(); - if tagged { + if let Some(c_type) = tagged { parse_quote! { map(|res| { let (id, b) = res.unwrap(); - (id, #root::runtime_support::bincode::deserialize::<#t_type>(&b).unwrap()) + (#root::ClusterId::<#c_type>::from_raw(id), #root::runtime_support::bincode::deserialize::<#t_type>(&b).unwrap()) }) } } else { @@ -701,12 +688,12 @@ impl<'a, T, W, N: Location<'a>> Stream { self.send_bincode::, T>(other) } - pub fn decouple_cluster( + pub fn decouple_cluster( self, other: &Cluster<'a, C2>, ) -> Stream> where - N: CanSend<'a, Cluster<'a, C2>, In = (u32, T), Out = (u32, T)>, + N: CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, T)>, T: Clone + Serialize + DeserializeOwned, { let self_node_id = match self.location_kind { @@ -731,7 +718,7 @@ impl<'a, T, W, N: Location<'a>> Stream { { let serialize_pipeline = Some(serialize_bincode::(N::is_demux())); - let deserialize_pipeline = Some(deserialize_bincode::(N::is_tagged())); + let deserialize_pipeline = Some(deserialize_bincode::(N::tagged_type())); Stream::new( other.id(), @@ -797,6 +784,7 @@ impl<'a, T, W, N: Location<'a>> Stream { where N: CanSend<'a, N2, In = T>, { + let root = get_this_crate(); Stream::new( other.id(), self.flow_state, @@ -807,8 +795,10 @@ impl<'a, T, W, N: Location<'a>> Stream { to_key: None, serialize_pipeline: None, instantiate_fn: DebugInstantiate::Building(), - deserialize_pipeline: if N::is_tagged() { - Some(parse_quote!(map(|(id, b)| (id, b.unwrap().freeze())))) + deserialize_pipeline: if let Some(c_type) = N::tagged_type() { + Some( + parse_quote!(map(|(id, b)| (#root::ClusterId<#c_type>::from_raw(id), b.unwrap().freeze()))), + ) } else { Some(parse_quote!(map(|b| b.unwrap().freeze()))) }, @@ -875,7 +865,7 @@ impl<'a, T, W, N: Location<'a>> Stream { other: &Cluster<'a, C2>, ) -> Stream, Unbounded, NoTick, Cluster<'a, C2>> where - N: CanSend<'a, Cluster<'a, C2>, In = (u32, T)>, + N: CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T)>, T: Clone + Serialize + DeserializeOwned, { let ids = other.members(); @@ -892,7 +882,7 @@ impl<'a, T, W, N: Location<'a>> Stream { other: &Cluster<'a, C2>, ) -> Stream> where - N: CanSend<'a, Cluster<'a, C2>, In = (u32, T), Out = (Tag, T)> + 'a, + N: CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, T)> + 'a, T: Clone + Serialize + DeserializeOwned, { self.broadcast_bincode(other).map(q!(|(_, b)| b)) @@ -903,7 +893,7 @@ impl<'a, T, W, N: Location<'a>> Stream { other: &Cluster<'a, C2>, ) -> Stream, Unbounded, NoTick, Cluster<'a, C2>> where - N: CanSend<'a, Cluster<'a, C2>, In = (u32, T)> + 'a, + N: CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T)> + 'a, T: Clone, { let ids = other.members(); @@ -920,7 +910,8 @@ impl<'a, T, W, N: Location<'a>> Stream { other: &Cluster<'a, C2>, ) -> Stream> where - N: CanSend<'a, Cluster<'a, C2>, In = (u32, T), Out = (Tag, Bytes)> + 'a, + N: CanSend<'a, Cluster<'a, C2>, In = (ClusterId, T), Out = (Tag, Bytes)> + + 'a, T: Clone, { self.broadcast_bytes(other).map(q!(|(_, b)| b)) diff --git a/hydroflow_plus_test/src/cluster/many_to_many.rs b/hydroflow_plus_test/src/cluster/many_to_many.rs index e3aa96870a5e..90adbf03f09b 100644 --- a/hydroflow_plus_test/src/cluster/many_to_many.rs +++ b/hydroflow_plus_test/src/cluster/many_to_many.rs @@ -61,7 +61,7 @@ mod tests { for value in 0..2 { assert_eq!( node_outs.next().unwrap(), - format!("cluster received: ({}, {})", sender, value) + format!("cluster received: (ClusterId::<()>({}), {})", sender, value) ); } } diff --git a/hydroflow_plus_test/src/cluster/map_reduce.rs b/hydroflow_plus_test/src/cluster/map_reduce.rs index 1e76c1fc14eb..95696d5cbcf0 100644 --- a/hydroflow_plus_test/src/cluster/map_reduce.rs +++ b/hydroflow_plus_test/src/cluster/map_reduce.rs @@ -16,7 +16,10 @@ pub fn map_reduce<'a>(flow: &FlowBuilder<'a>) -> (Process<'a, Leader>, Cluster<' let words_partitioned = words .tick_batch() .enumerate() - .map(q!(|(i, w)| ((i % all_ids_vec.len()) as u32, w))) + .map(q!(|(i, w)| ( + ClusterId::from_raw((i % all_ids_vec.len()) as u32), + w + ))) .all_ticks(); words_partitioned diff --git a/hydroflow_plus_test/src/cluster/paxos.rs b/hydroflow_plus_test/src/cluster/paxos.rs index fd0c6e3f784b..488be19c01e4 100644 --- a/hydroflow_plus_test/src/cluster/paxos.rs +++ b/hydroflow_plus_test/src/cluster/paxos.rs @@ -21,11 +21,11 @@ pub trait PaxosPayload: pub struct Ballot { // Note: Important that num comes before id, since Ord is defined lexicographically pub num: u32, - pub id: u32, + pub id: ClusterId, } impl LeaderElected for Ballot { - fn leader_id(&self) -> u32 { + fn leader_id(&self) -> ClusterId { self.id } } @@ -64,12 +64,12 @@ struct P2b

{ } #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -pub fn paxos_core<'a, P: PaxosPayload>( +pub fn paxos_core<'a, P: PaxosPayload, R>( flow: &FlowBuilder<'a>, r_to_acceptors_checkpoint: impl FnOnce( &Cluster<'a, Acceptor>, ) -> Stream< - (u32, i32), + (ClusterId, i32), Unbounded, NoTick, Cluster<'a, Acceptor>, @@ -184,16 +184,21 @@ pub fn paxos_core<'a, P: PaxosPayload>( } #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] -fn acceptor<'a, P: PaxosPayload>( +fn acceptor<'a, P: PaxosPayload, R>( p_to_acceptors_p1a: Stream>, p_to_acceptors_p2a: Stream, Unbounded, NoTick, Cluster<'a, Acceptor>>, - r_to_acceptors_checkpoint: Stream<(u32, i32), Unbounded, NoTick, Cluster<'a, Acceptor>>, + r_to_acceptors_checkpoint: Stream< + (ClusterId, i32), + Unbounded, + NoTick, + Cluster<'a, Acceptor>, + >, proposers: &Cluster<'a, Proposer>, acceptors: &Cluster<'a, Acceptor>, f: usize, ) -> ( - Stream<(u32, P1b

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

), Unbounded, NoTick, Cluster<'a, Proposer>>, + Stream<(ClusterId, P1b

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

), Unbounded, NoTick, Cluster<'a, Proposer>>, ) { // Get the latest checkpoint sequence per replica let a_checkpoint_largest_seqs = @@ -222,7 +227,10 @@ fn acceptor<'a, P: PaxosPayload>( min_seq, P2a { // Create tuple with checkpoint number and dummy p2a - ballot: Ballot { num: 0, id: 0 }, + ballot: Ballot { + num: 0, + id: ClusterId::from_raw(0) + }, slot: -1, value: Default::default() } @@ -233,7 +241,10 @@ fn acceptor<'a, P: PaxosPayload>( .clone() .map(q!(|p1a| p1a.ballot)) .max() - .unwrap_or(acceptors.singleton(q!(Ballot { num: 0, id: 0 }))); + .unwrap_or(acceptors.singleton(q!(Ballot { + num: 0, + id: ClusterId::from_raw(0) + }))); let a_p2as_to_place_in_log = p_to_acceptors_p2a .clone() .tick_batch() @@ -318,9 +329,15 @@ fn acceptor<'a, P: PaxosPayload>( (a_to_proposers_p1b_new, a_to_proposers_p2b_new) } +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_p2b<'a, P: PaxosPayload>( proposers: &Cluster<'a, Proposer>, - a_to_proposers_p2b: Stream<(u32, P2b

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

), + Unbounded, + NoTick, + Cluster<'a, Proposer>, + >, f: usize, ) -> Stream<(i32, P), Unbounded, NoTick, Cluster<'a, Proposer>> { let (p_broadcasted_p2b_slots_complete_cycle, p_broadcasted_p2b_slots) = proposers.tick_cycle(); @@ -341,8 +358,14 @@ fn p_p2b<'a, P: PaxosPayload>( q!(|| ( 0, P2b { - ballot: Ballot { num: 0, id: 0 }, - max_ballot: Ballot { num: 0, id: 0 }, + ballot: Ballot { + num: 0, + id: ClusterId::from_raw(0) + }, + max_ballot: Ballot { + num: 0, + id: ClusterId::from_raw(0) + }, slot: 0, value: Default::default() } @@ -466,7 +489,12 @@ fn p_p2a<'a, P: PaxosPayload>( #[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_p1b<'a, P: PaxosPayload>( proposers: &Cluster<'a, Proposer>, - a_to_proposers_p1b: Stream<(u32, P1b

), Unbounded, NoTick, Cluster<'a, Proposer>>, + a_to_proposers_p1b: Stream< + (ClusterId, P1b

), + Unbounded, + NoTick, + Cluster<'a, Proposer>, + >, p_ballot_num: Singleton>, p_has_largest_ballot: Optional<(Ballot, u32), Bounded, Tick, Cluster<'a, Proposer>>, f: usize, @@ -501,7 +529,7 @@ fn p_p1b<'a, P: PaxosPayload>( let p_p1b_highest_entries_and_count = p_relevant_p1bs .clone() .flat_map(q!(|((_, p1b), _)| p1b.accepted.into_iter())) // Convert HashMap log back to stream - .fold_keyed(q!(|| (0, LogValue { ballot: Ballot { num: 0, id: 0 }, value: Default::default() })), q!(|curr_entry, new_entry| { + .fold_keyed(q!(|| (0, LogValue { ballot: Ballot { num: 0, id: ClusterId::from_raw(0) }, value: Default::default() })), q!(|curr_entry, new_entry| { let same_values = new_entry.value == curr_entry.1.value; let higher_ballot = new_entry.ballot > curr_entry.1.ballot; // Increment count if the values are the same @@ -559,10 +587,21 @@ fn p_p1b<'a, P: PaxosPayload>( } // Proposer logic to calculate the largest ballot received so far. +#[expect(clippy::type_complexity, reason = "internal paxos code // TODO")] fn p_max_ballot<'a, P: PaxosPayload>( proposers: &Cluster<'a, Proposer>, - a_to_proposers_p1b: Stream<(u32, P1b

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

), Unbounded, NoTick, Cluster<'a, Proposer>>, + a_to_proposers_p1b: Stream< + (ClusterId, P1b

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

), + Unbounded, + NoTick, + Cluster<'a, Proposer>, + >, p_to_proposers_i_am_leader: Stream>, ) -> Singleton> { let p_received_p1b_ballots = a_to_proposers_p1b @@ -575,7 +614,10 @@ fn p_max_ballot<'a, P: PaxosPayload>( .union(p_received_p2b_ballots) .union(p_to_proposers_i_am_leader) .max() - .unwrap_or(proposers.singleton(q!(Ballot { num: 0, id: 0 }))) + .unwrap_or(proposers.singleton(q!(Ballot { + num: 0, + id: ClusterId::from_raw(0) + }))) } // Proposer logic to calculate the next ballot number. Expects p_received_max_ballot, the largest ballot received so far. Outputs streams: ballot_num, and has_largest_ballot, which only contains a value if we have the largest ballot. @@ -666,7 +708,7 @@ fn p_p1a<'a>( }), ); // Add random delay depending on node ID so not everyone sends p1a at the same time - let p_leader_expired = proposers.source_interval_delayed(q!(Duration::from_secs((p_id * i_am_leader_check_timeout_delay_multiplier as u32).into())), q!(Duration::from_secs(i_am_leader_check_timeout))) + let p_leader_expired = proposers.source_interval_delayed(q!(Duration::from_secs((p_id.raw_id * i_am_leader_check_timeout_delay_multiplier as u32).into())), q!(Duration::from_secs(i_am_leader_check_timeout))) .cross_singleton(p_latest_received_i_am_leader.clone()) .latest_tick() // .inspect(q!(|v| println!("Proposer checking if leader expired"))) diff --git a/hydroflow_plus_test/src/cluster/paxos_bench.rs b/hydroflow_plus_test/src/cluster/paxos_bench.rs index fdf76683b706..932772021a22 100644 --- a/hydroflow_plus_test/src/cluster/paxos_bench.rs +++ b/hydroflow_plus_test/src/cluster/paxos_bench.rs @@ -10,7 +10,7 @@ use stageleft::*; use super::paxos::{paxos_core, Acceptor, Ballot, PaxosPayload, Proposer}; pub trait LeaderElected: Ord + Clone { - fn leader_id(&self) -> u32; + fn leader_id(&self) -> ClusterId; } pub struct Replica {} @@ -97,7 +97,12 @@ pub fn paxos_bench<'a>( fn paxos_with_replica<'a>( flow: &FlowBuilder<'a>, replicas: &Cluster<'a, Replica>, - c_to_proposers: Stream<(u32, ClientPayload), Unbounded, NoTick, Cluster<'a, Client>>, + c_to_proposers: Stream< + (ClusterId, ClientPayload), + Unbounded, + NoTick, + Cluster<'a, Client>, + >, f: usize, i_am_leader_send_timeout: u64, i_am_leader_check_timeout: u64, @@ -107,7 +112,7 @@ fn paxos_with_replica<'a>( Cluster<'a, Proposer>, Cluster<'a, Acceptor>, Stream>, - Stream<(u32, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Replica>>, + Stream<(ClusterId, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Replica>>, ) { let (r_to_acceptors_checkpoint_complete_cycle, r_to_acceptors_checkpoint) = replicas.cycle::>(); @@ -152,7 +157,7 @@ pub fn replica<'a>( checkpoint_frequency: usize, ) -> ( Stream>, - Stream<(u32, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Replica>>, + Stream<(ClusterId, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Replica>>, ) { let (r_buffered_payloads_complete_cycle, r_buffered_payloads) = replicas.tick_cycle(); // p_to_replicas.inspect(q!(|payload: ReplicaPayload| println!("Replica received payload: {:?}", payload))); @@ -239,7 +244,7 @@ pub fn replica<'a>( // Tell clients that the payload has been committed. All ReplicaPayloads contain the client's machine ID (to string) as value. let r_to_clients = p_to_replicas.map(q!(|payload| ( - payload.value.parse::().unwrap(), + ClusterId::from_raw(payload.value.parse::().unwrap()), payload ))); (r_checkpoint_seq_new.all_ticks(), r_to_clients) @@ -250,7 +255,7 @@ fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( clients: &Cluster<'a, Client>, p_to_clients_leader_elected: Stream>, r_to_clients_payload_applied: Stream< - (u32, ReplicaPayload), + (ClusterId, ReplicaPayload), Unbounded, NoTick, Cluster<'a, Client>, @@ -258,7 +263,7 @@ fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( num_clients_per_node: usize, median_latency_window_size: usize, f: usize, -) -> Stream<(u32, ClientPayload), Unbounded, NoTick, Cluster<'a, Client>> { +) -> Stream<(ClusterId, ClientPayload), Unbounded, NoTick, Cluster<'a, Client>> { let c_id = clients.self_id(); // r_to_clients_payload_applied.clone().inspect(q!(|payload: &(u32, ReplicaPayload)| println!("Client received payload: {:?}", payload))); // Only keep the latest leader @@ -278,7 +283,7 @@ fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( leader_ballot.leader_id(), ClientPayload { key: i as u32, - value: c_id.to_string() + value: c_id.raw_id.to_string() } ) ))); @@ -318,7 +323,7 @@ fn bench_client<'a, B: LeaderElected + std::fmt::Debug>( leader_ballot.leader_id(), ClientPayload { key, - value: c_id.to_string() + value: c_id.raw_id.to_string() } ))); let c_to_proposers = c_new_payloads_when_leader_elected diff --git a/hydroflow_plus_test/src/cluster/simple_cluster.rs b/hydroflow_plus_test/src/cluster/simple_cluster.rs index 68a2c63cd8cb..771a7ed1c97b 100644 --- a/hydroflow_plus_test/src/cluster/simple_cluster.rs +++ b/hydroflow_plus_test/src/cluster/simple_cluster.rs @@ -95,7 +95,7 @@ mod tests { for j in 0..5 { assert_eq!( stdout.recv().await.unwrap(), - format!("cluster received: ({}, {}) (self cluster id: {})", i, j, i) + format!("cluster received: (ClusterId::<()>({}), {}) (self cluster id: ClusterId::<()>({}))", i, j, i) ); } } @@ -109,7 +109,12 @@ mod tests { for (i, n) in node_outs.into_iter().enumerate() { assert_eq!( n, - format!("node received: ({}, ({}, {}))", i / 5, i / 5, i % 5) + format!( + "node received: (ClusterId::<()>({}), (ClusterId::<()>({}), {}))", + i / 5, + i / 5, + i % 5 + ) ); } } @@ -174,7 +179,10 @@ mod tests { for (i, mut stdout) in cluster2_stdouts.into_iter().enumerate() { for _j in 0..1 { - let expected_message = format!("My self id is {}, my message is {}", i, i); + let expected_message = format!( + "My self id is ClusterId::<()>({}), my message is ClusterId::<()>({})", + i, i + ); assert_eq!(stdout.recv().await.unwrap(), expected_message); } } diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap index c8bbf0a947ec..70a8dd42e91b 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir.snap @@ -12,7 +12,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u64 , u64) , (u64 , u64) , () > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | (inside , total) , (inside_batch , total_batch) | { * inside += inside_batch ; * total += total_batch ; } }), input: Persist( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , (u64 , u64)) , (u64 , u64) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: compute_pi :: Worker > , (u64 , u64)) , (u64 , u64) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -38,7 +38,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (u64 , u64) > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: compute_pi :: Worker > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (u64 , u64) > (& b) . unwrap ()) }", ], }, ), diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_1.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_1.snap index ca6d96f060dc..c73fdb714a8e 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_1.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__compute_pi__tests__compute_pi_ir@surface_graph_1.snap @@ -3,8 +3,8 @@ source: hydroflow_plus_test/src/cluster/compute_pi.rs expression: ir.surface_syntax_string() --- 1v1 = source_stream ({ use hydroflow_plus :: __staged :: deploy :: deploy_runtime :: * ; let env = FAKE ; let p2_port = "port_0" ; { env . port (p2_port) . connect_local_blocking :: < ConnectedTagged < ConnectedDirect > > () . into_source () } }); -2v1 = map (| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (u64 , u64) > (& b) . unwrap ()) }); -3v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (u32 , (u64 , u64)) , (u64 , u64) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b })); +2v1 = map (| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: compute_pi :: Worker > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (u64 , u64) > (& b) . unwrap ()) }); +3v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: compute_pi :: Worker > , (u64 , u64)) , (u64 , u64) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b })); 4v1 = reduce :: < 'static > (stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u64 , u64) , (u64 , u64) , () > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | (inside , total) , (inside_batch , total_batch) | { * inside += inside_batch ; * total += total_batch ; } })); 5v1 = source_interval (stageleft :: runtime_support :: type_hint :: < core :: time :: Duration > ({ use crate :: __staged :: cluster :: compute_pi :: * ; Duration :: from_secs (1) })); 6v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydroflow_plus :: __staged :: singleton :: * ; | _u | () })); diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__many_to_many__tests__many_to_many.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__many_to_many__tests__many_to_many.snap index 1a04dda0fb11..d58ad11b7168 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__many_to_many__tests__many_to_many.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__many_to_many__tests__many_to_many.snap @@ -4,7 +4,7 @@ expression: built.ir() --- [ ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , i32) , () > ({ use crate :: __staged :: cluster :: many_to_many :: * ; | n | println ! ("cluster received: {:?}" , n) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < () > , i32) , () > ({ use crate :: __staged :: cluster :: many_to_many :: * ; | n | println ! ("cluster received: {:?}" , n) }), input: Network { from_location: Cluster( 0, @@ -19,7 +19,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , i32) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", ], }, ), @@ -30,13 +30,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < () > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < () > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < () > > > (__hydroflow_plus_cluster_ids_0) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Source { source: Iter( { use crate :: __staged :: cluster :: many_to_many :: * ; 0 .. 2 }, diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir.snap index c33d9b128113..c895c3413a7c 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir.snap @@ -9,7 +9,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | total , count | * total += count }), input: Persist( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , (std :: string :: String , i32)) , (std :: string :: String , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: map_reduce :: Worker > , (std :: string :: String , i32)) , (std :: string :: String , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 1, @@ -35,7 +35,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (std :: string :: String , i32) > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: map_reduce :: Worker > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (std :: string :: String , i32) > (& b) . unwrap ()) }", ], }, ), @@ -61,7 +61,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < std :: string :: String > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , std :: string :: String) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < std :: string :: String > (& data) . unwrap () . into ()) }", ], }, ), @@ -78,7 +78,7 @@ expression: built.ir() ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (u32 , std :: string :: String) > ({ use crate :: __staged :: cluster :: map_reduce :: * ; let all_ids_vec = __hydroflow_plus_cluster_ids_1 ; | (i , w) | ((i % all_ids_vec . len ()) as u32 , w) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: map_reduce :: Worker > , std :: string :: String) > ({ use crate :: __staged :: cluster :: map_reduce :: * ; let all_ids_vec = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: map_reduce :: Worker > > > (__hydroflow_plus_cluster_ids_1) } ; | (i , w) | (ClusterId :: from_raw ((i % all_ids_vec . len ()) as u32) , w) }), input: Enumerate( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < & str , std :: string :: String > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | s | s . to_string () }), diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap index de23c7f7e01c..06befe0c8323 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__map_reduce__tests__map_reduce_ir@surface_graph_0.snap @@ -5,12 +5,12 @@ expression: ir.surface_syntax_string() 1v1 = source_iter ({ use crate :: __staged :: cluster :: map_reduce :: * ; vec ! ["abc" , "abc" , "xyz" , "abc"] }); 2v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < & str , std :: string :: String > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | s | s . to_string () })); 3v1 = enumerate (); -4v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (u32 , std :: string :: String) > ({ use crate :: __staged :: cluster :: map_reduce :: * ; let all_ids_vec = __hydroflow_plus_cluster_ids_1 ; | (i , w) | ((i % all_ids_vec . len ()) as u32 , w) })); -5v1 = map (| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < std :: string :: String > (& data) . unwrap () . into ()) }); +4v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (usize , std :: string :: String) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: map_reduce :: Worker > , std :: string :: String) > ({ use crate :: __staged :: cluster :: map_reduce :: * ; let all_ids_vec = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: map_reduce :: Worker > > > (__hydroflow_plus_cluster_ids_1) } ; | (i , w) | (ClusterId :: from_raw ((i % all_ids_vec . len ()) as u32) , w) })); +5v1 = map (| (id , data) : (hydroflow_plus :: ClusterId < _ > , std :: string :: String) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < std :: string :: String > (& data) . unwrap () . into ()) }); 6v1 = dest_sink ({ use hydroflow_plus :: __staged :: deploy :: deploy_runtime :: * ; let env = FAKE ; let p1_port = "port_0" ; { env . port (p1_port) . connect_local_blocking :: < ConnectedDemux < ConnectedDirect > > () . into_sink () } }); 7v1 = source_stream ({ use hydroflow_plus :: __staged :: deploy :: deploy_runtime :: * ; let env = FAKE ; let p2_port = "port_1" ; { env . port (p2_port) . connect_local_blocking :: < ConnectedTagged < ConnectedDirect > > () . into_source () } }); -8v1 = map (| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (std :: string :: String , i32) > (& b) . unwrap ()) }); -9v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (u32 , (std :: string :: String , i32)) , (std :: string :: String , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b })); +8v1 = map (| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: map_reduce :: Worker > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (std :: string :: String , i32) > (& b) . unwrap ()) }); +9v1 = map (stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: map_reduce :: Worker > , (std :: string :: String , i32)) , (std :: string :: String , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b })); 10v1 = reduce_keyed :: < 'static > (stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | total , count | * total += count })); 11v1 = for_each (stageleft :: runtime_support :: fn1_type_hint :: < (std :: string :: String , i32) , () > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | (string , count) | println ! ("{}: {}" , string , count) })); diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap index d5aba2eee9ab..aa260c31ecb3 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__paxos_bench__tests__paxos_ir.snap @@ -22,7 +22,7 @@ expression: built.ir() 2, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , id : p_id , }) { received_max_ballot . num + 1 } else { ballot_num } } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , id : p_id , }) { received_max_ballot . num + 1 } else { ballot_num } } }), input: CrossSingleton( Tee { inner: RefCell { @@ -33,7 +33,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -48,7 +48,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -81,7 +81,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -130,7 +130,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -175,20 +175,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -242,7 +242,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -253,7 +253,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -268,7 +268,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -301,7 +301,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -361,7 +361,7 @@ expression: built.ir() 2, ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -376,7 +376,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", ], }, ), @@ -387,15 +387,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_2 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > > > (__hydroflow_plus_cluster_ids_2) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ballot_num | Ballot { num : ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | Ballot { num : ballot_num , id : p_id } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -450,20 +450,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -517,7 +517,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -528,7 +528,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -543,7 +543,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -576,7 +576,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -658,20 +658,20 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -776,7 +776,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -791,7 +791,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -802,7 +802,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -836,7 +836,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -851,7 +851,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -862,7 +862,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -939,7 +939,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -954,7 +954,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -965,7 +965,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -1005,20 +1005,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -1072,7 +1072,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -1083,7 +1083,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -1098,7 +1098,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -1131,7 +1131,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -1193,20 +1193,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -1260,7 +1260,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -1271,7 +1271,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -1286,7 +1286,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -1319,7 +1319,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -1391,20 +1391,20 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -1509,7 +1509,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -1524,7 +1524,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -1535,7 +1535,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -1569,7 +1569,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -1584,7 +1584,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -1595,7 +1595,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -1672,7 +1672,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -1687,7 +1687,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -1698,7 +1698,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -1738,20 +1738,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -1805,7 +1805,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -1816,7 +1816,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -1831,7 +1831,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -1864,7 +1864,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -1939,10 +1939,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -1984,10 +1984,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -2033,10 +2033,10 @@ expression: built.ir() ), input: DeferTick( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (sender , p2b)) | (sender , p2b) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_slot , (sender , p2b)) | (sender , p2b) }), input: AntiJoin( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | (p2b . slot , (sender , p2b)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | (p2b . slot , (sender , p2b)) }), input: Tee { inner: RefCell { value: Union( @@ -2071,10 +2071,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -2144,7 +2144,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", ], }, ), @@ -2155,13 +2155,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , log)) | (p1a . ballot . id , P1b { ballot : p1a . ballot , max_ballot , accepted : log }) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus_test :: cluster :: paxos :: P1a , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (i32 , std :: collections :: hash_map :: HashMap < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > >)) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((p1a , max_ballot) , (_prev_checkpoint , log)) | (p1a . ballot . id , P1b { ballot : p1a . ballot , max_ballot , accepted : log }) }), input: CrossSingleton( CrossSingleton( Tee { @@ -2169,7 +2169,7 @@ expression: built.ir() value: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -2184,7 +2184,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1a) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", ], }, ), @@ -2195,15 +2195,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -2243,7 +2243,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -2288,20 +2288,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -2355,7 +2355,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -2366,7 +2366,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -2381,7 +2381,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -2414,7 +2414,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -2488,7 +2488,7 @@ expression: built.ir() value: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -2503,7 +2503,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1a) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", ], }, ), @@ -2514,15 +2514,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -2562,7 +2562,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -2607,20 +2607,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -2674,7 +2674,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -2685,7 +2685,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -2700,7 +2700,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -2733,7 +2733,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -2800,7 +2800,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 3, @@ -2822,7 +2822,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -2837,7 +2837,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", ], }, ), @@ -2848,13 +2848,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -2864,25 +2864,25 @@ expression: built.ir() input: CrossSingleton( Union( FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , id : p_id , } , slot , value : entry . value , }) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , id : p_id , } , slot , value : entry . value , }) } else { None } }), input: CrossSingleton( Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -2958,7 +2958,7 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : Default :: default () } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : Default :: default () } }), input: CrossSingleton( Difference( FlatMap { @@ -2972,20 +2972,20 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -3044,20 +3044,20 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -3160,14 +3160,14 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }), input: CrossSingleton( CrossSingleton( Enumerate( Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -3182,7 +3182,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -3193,7 +3193,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -3262,20 +3262,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -3329,7 +3329,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -3340,7 +3340,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -3355,7 +3355,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -3388,7 +3388,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -3452,7 +3452,7 @@ expression: built.ir() value: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -3467,7 +3467,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1a) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", ], }, ), @@ -3478,15 +3478,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -3526,7 +3526,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -3571,20 +3571,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -3638,7 +3638,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -3649,7 +3649,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -3664,7 +3664,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -3697,7 +3697,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -3764,7 +3764,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 3, @@ -3777,15 +3777,15 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | (min_seq , P2a { ballot : Ballot { num : 0 , id : 0 } , slot : - 1 , value : Default :: default () }) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , (i32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | (min_seq , P2a { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : - 1 , value : Default :: default () }) }), input: Delta( Union( Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , i32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , i32) , ()) , (u32 , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , ()) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( Tee { inner: RefCell { @@ -3806,7 +3806,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , i32) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", ], }, ), @@ -3817,13 +3817,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: CycleSource { ident: Ident { sym: cycle_0, @@ -3844,7 +3844,7 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received == f + 1 { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (u32 , i32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , i32) , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: RefCell { value: ReduceKeyed { @@ -3864,7 +3864,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , i32) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < i32 > (& data) . unwrap () . into ()) }", ], }, ), @@ -3875,13 +3875,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < i32 > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: CycleSource { ident: Ident { sym: cycle_0, @@ -3944,7 +3944,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", ], }, ), @@ -3955,18 +3955,18 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Acceptor > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . id , P2b { ballot : p2a . ballot , max_ballot , slot : p2a . slot , value : p2a . value }) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . id , P2b { ballot : p2a . ballot , max_ballot , slot : p2a . slot , value : p2a . value }) }), input: CrossSingleton( Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -3981,7 +3981,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& data) . unwrap () . into ()) }", ], }, ), @@ -3992,13 +3992,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , ()) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use hydroflow_plus :: __staged :: stream :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -4008,25 +4008,25 @@ expression: built.ir() input: CrossSingleton( Union( FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , id : p_id , } , slot , value : entry . value , }) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) , u32) , core :: option :: Option < hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((slot , (count , entry)) , ballot_num) | if count <= f as u32 { Some (P2a { ballot : Ballot { num : ballot_num , id : p_id , } , slot , value : entry . value , }) } else { None } }), input: CrossSingleton( Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -4102,7 +4102,7 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : Default :: default () } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (slot , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot , value : Default :: default () } }), input: CrossSingleton( Difference( FlatMap { @@ -4116,20 +4116,20 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -4188,20 +4188,20 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : 0 } , value : Default :: default () }) }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , LogValue { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , value : Default :: default () }) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_entry , new_entry | { let same_values = new_entry . value == curr_entry . 1 . value ; let higher_ballot = new_entry . ballot > curr_entry . 1 . ballot ; if same_values { curr_entry . 0 += 1 ; } if higher_ballot { curr_entry . 1 . ballot = new_entry . ballot ; if ! same_values { curr_entry . 0 = 1 ; curr_entry . 1 . value = new_entry . value ; } } } }), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , std :: collections :: hash_map :: IntoIter < i32 , hydroflow_plus_test :: cluster :: paxos :: LogValue < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((_ , p1b) , _) | p1b . accepted . into_iter () }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -4304,14 +4304,14 @@ expression: built.ir() ), }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , i32) , u32) , hydroflow_plus_test :: cluster :: paxos :: P2a < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (((index , payload) , next_slot) , ballot_num) | P2a { ballot : Ballot { num : ballot_num , id : p_id } , slot : next_slot + index as i32 , value : payload } }), input: CrossSingleton( CrossSingleton( Enumerate( Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 0, @@ -4326,7 +4326,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -4337,7 +4337,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload > (& b) . unwrap ()) }", ], }, ), @@ -4406,20 +4406,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -4473,7 +4473,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -4484,7 +4484,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -4499,7 +4499,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -4532,7 +4532,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -4596,7 +4596,7 @@ expression: built.ir() value: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: P1a) , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -4611,7 +4611,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: P1a) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& data) . unwrap () . into ()) }", ], }, ), @@ -4622,15 +4622,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: P1a > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_3 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: P1a , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > > (__hydroflow_plus_cluster_ids_3) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydroflow_plus_test :: cluster :: paxos :: P1a > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ballot_num | P1a { ballot : Ballot { num : ballot_num , id : p_id } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , ()) , u32 > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), input: CrossSingleton( @@ -4670,7 +4670,7 @@ expression: built.ir() CrossSingleton( Source { source: Stream( - { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = __hydroflow_plus_cluster_self_id_2 ; Duration :: from_secs ((p_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, + { use hydroflow_plus :: __staged :: location :: * ; let delay = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout_delay_multiplier = 1usize ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; Duration :: from_secs ((p_id . raw_id * i_am_leader_check_timeout_delay_multiplier as u32) . into ()) } ; let interval = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout) } ; tokio_stream :: wrappers :: IntervalStream :: new (tokio :: time :: interval_at (tokio :: time :: Instant :: now () + delay , interval)) }, ), location_kind: Cluster( 2, @@ -4715,20 +4715,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -4782,7 +4782,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -4793,7 +4793,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -4808,7 +4808,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -4841,7 +4841,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -4908,7 +4908,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 3, @@ -4942,7 +4942,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -4957,7 +4957,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -4968,13 +4968,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -4987,10 +4987,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -5065,7 +5065,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -5080,7 +5080,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -5091,13 +5091,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -5110,10 +5110,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -5232,7 +5232,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -5247,7 +5247,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -5258,13 +5258,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -5277,10 +5277,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -5355,7 +5355,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -5370,7 +5370,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -5381,13 +5381,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -5400,10 +5400,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -5559,7 +5559,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -5574,7 +5574,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -5585,13 +5585,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -5604,10 +5604,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -5682,7 +5682,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -5697,7 +5697,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -5708,13 +5708,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -5727,10 +5727,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -5889,7 +5889,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -5904,7 +5904,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -5915,13 +5915,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -5934,10 +5934,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -6012,7 +6012,7 @@ expression: built.ir() Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -6027,7 +6027,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6038,13 +6038,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -6057,10 +6057,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -6172,7 +6172,7 @@ expression: built.ir() inner: RefCell { value: Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { from_location: Cluster( 1, @@ -6187,7 +6187,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6198,17 +6198,17 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -6223,7 +6223,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6234,13 +6234,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -6253,10 +6253,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -6326,12 +6326,12 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 1 { Some (key) } else { None } } }), input: FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { inner: RefCell { value: Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { from_location: Cluster( 1, @@ -6346,7 +6346,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6357,17 +6357,17 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -6382,7 +6382,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6393,13 +6393,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -6412,10 +6412,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -6525,7 +6525,7 @@ expression: built.ir() Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -6540,7 +6540,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", ], }, ), @@ -6551,15 +6551,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (__hydroflow_plus_cluster_ids_0) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , id : p_id } }), input: CrossSingleton( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), @@ -6573,20 +6573,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -6640,7 +6640,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -6651,7 +6651,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -6666,7 +6666,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -6699,7 +6699,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -6816,12 +6816,12 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 1 { Some (key) } else { None } } }), input: FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { inner: RefCell { value: Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { from_location: Cluster( 1, @@ -6836,7 +6836,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6847,17 +6847,17 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -6872,7 +6872,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -6883,13 +6883,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -6902,10 +6902,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -7026,12 +7026,12 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 1 { Some (key) } else { None } } }), input: FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { inner: RefCell { value: Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { from_location: Cluster( 1, @@ -7046,7 +7046,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -7057,17 +7057,17 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -7082,7 +7082,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -7093,13 +7093,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -7112,10 +7112,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -7228,12 +7228,12 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 1 { Some (key) } else { None } } }), input: FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { inner: RefCell { value: Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { from_location: Cluster( 1, @@ -7248,7 +7248,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -7259,17 +7259,17 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -7284,7 +7284,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -7295,13 +7295,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -7314,10 +7314,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -7442,7 +7442,7 @@ expression: built.ir() ), input: Union( FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = __hydroflow_plus_cluster_self_id_0 ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . leader_id () , ClientPayload { key : i as u32 , value : c_id . to_string () })) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; let num_clients_per_node = 1usize ; move | leader_ballot | (0 .. num_clients_per_node) . map (move | i | (leader_ballot . leader_id () , ClientPayload { key : i as u32 , value : c_id . raw_id . to_string () })) }), input: Tee { inner: RefCell { value: Delta( @@ -7454,7 +7454,7 @@ expression: built.ir() Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -7469,7 +7469,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", ], }, ), @@ -7480,15 +7480,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (__hydroflow_plus_cluster_ids_0) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , id : p_id } }), input: CrossSingleton( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), @@ -7502,20 +7502,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -7569,7 +7569,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -7580,7 +7580,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -7595,7 +7595,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -7628,7 +7628,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, @@ -7734,7 +7734,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = __hydroflow_plus_cluster_self_id_0 ; move | (key , leader_ballot) | (leader_ballot . leader_id () , ClientPayload { key , value : c_id . to_string () }) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let c_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Client > :: from_raw (__hydroflow_plus_cluster_self_id_0) ; move | (key , leader_ballot) | (leader_ballot . leader_id () , ClientPayload { key , value : c_id . raw_id . to_string () }) }), input: CrossSingleton( Tee { inner: RefCell { @@ -7742,12 +7742,12 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , usize) , core :: option :: Option < u32 > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let f = 1usize ; move | (key , count) | { if count == f + 1 { Some (key) } else { None } } }), input: FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | | 0 }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | curr_count , _sender | { * curr_count += 1 ; } }), input: Tee { inner: RefCell { value: Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , u32) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , (u32 , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica >) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (sender , replica_payload) | (replica_payload . key , sender) }), input: Network { from_location: Cluster( 1, @@ -7762,7 +7762,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -7773,17 +7773,17 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (payload . value . parse :: < u32 > () . unwrap () , payload) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | payload | (ClusterId :: from_raw (payload . value . parse :: < u32 > () . unwrap ()) , payload) }), input: Tee { inner: RefCell { value: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -7798,7 +7798,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& data) . unwrap () . into ()) }", ], }, ), @@ -7809,13 +7809,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_1 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Replica > > > (__hydroflow_plus_cluster_ids_1) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (i32 , hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload) , hydroflow_plus_test :: cluster :: paxos_bench :: ReplicaPayload > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | (slot , data) | ReplicaPayload { seq : slot , key : data . key , value : data . value } }), input: Map { @@ -7828,10 +7828,10 @@ expression: built.ir() input: Tee { inner: RefCell { value: FoldKeyed { - init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : 0 } , max_ballot : Ballot { num : 0 , id : 0 } , slot : 0 , value : Default :: default () }) }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), + init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (0 , P2b { ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , max_ballot : Ballot { num : 0 , id : ClusterId :: from_raw (0) } , slot : 0 , value : Default :: default () }) }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | accum , (_sender , p2b) | { accum . 0 += 1 ; accum . 1 = p2b ; } }), input: FilterMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , core :: option :: Option < (i32 , (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >)) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (sender , p2b) | if p2b . ballot == p2b . max_ballot { Some ((p2b . slot , (sender , p2b))) } else { None } }), input: Tee { inner: RefCell { value: Union( @@ -7907,7 +7907,7 @@ expression: built.ir() Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Proposer > , hydroflow_plus_test :: cluster :: paxos :: Ballot) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use hydroflow_plus :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( 2, @@ -7922,7 +7922,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , hydroflow_plus_test :: cluster :: paxos :: Ballot) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& data) . unwrap () . into ()) }", ], }, ), @@ -7933,15 +7933,15 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < hydroflow_plus_test :: cluster :: paxos :: Ballot > (& b) . unwrap ()) }", ], }, ), ), input: FlatMap { - f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < u32 > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = __hydroflow_plus_cluster_ids_0 ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < hydroflow_plus_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > , _ > > ({ use hydroflow_plus :: __staged :: stream :: * ; let ids = unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < hydroflow_plus_test :: cluster :: paxos_bench :: Client > > > (__hydroflow_plus_cluster_ids_0) } ; | b | ids . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , u32) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (_is_leader , ballot_num) | Ballot { num : ballot_num , id : p_id } }), input: CrossSingleton( Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (bool , ()) , bool > ({ use hydroflow_plus :: __staged :: singleton :: * ; | (d , _signal) | d }), @@ -7955,20 +7955,20 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f = 1usize ; move | num_received | if num_received > f { Some (true) } else { None } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydroflow_plus :: __staged :: stream :: * ; | | 0usize }), - acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , u32 , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), + acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , () > ({ use hydroflow_plus :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Unique( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), + f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((sender , _p1b) , _ballot_num) | { sender } }), input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < ((hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | ((_sender , p1b) , ballot_num) | p1b . ballot == Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Persist( Tee { inner: RefCell { value: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Tee { inner: RefCell { value: CycleSource { @@ -8022,7 +8022,7 @@ expression: built.ir() input: Tee { inner: RefCell { value: Filter { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = __hydroflow_plus_cluster_self_id_2 ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus_test :: cluster :: paxos :: Ballot , u32) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; let p_id = hydroflow_plus :: ClusterId :: < hydroflow_plus_test :: cluster :: paxos :: Proposer > :: from_raw (__hydroflow_plus_cluster_self_id_2) ; move | (received_max_ballot , ballot_num) | * received_max_ballot <= Ballot { num : * ballot_num , id : p_id } }), input: CrossSingleton( Tee { inner: RefCell { @@ -8033,7 +8033,7 @@ expression: built.ir() Union( Union( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P1b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p1b) | p1b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -8048,7 +8048,7 @@ expression: built.ir() }, }, Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < hydroflow_plus_test :: cluster :: paxos :: Acceptor > , hydroflow_plus_test :: cluster :: paxos :: P2b < hydroflow_plus_test :: cluster :: paxos_bench :: ClientPayload >) , hydroflow_plus_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_ , p2b) | p2b . max_ballot }), input: Tee { inner: RefCell { value: CycleSource { @@ -8081,7 +8081,7 @@ expression: built.ir() Persist( Source { source: Iter( - { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : 0 } } ; [e] }, + { use hydroflow_plus :: __staged :: location :: * ; let e = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , id : ClusterId :: from_raw (0) } } ; [e] }, ), location_kind: Cluster( 2, diff --git a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap index 1889a09ef06e..8fd05b1df972 100644 --- a/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap +++ b/hydroflow_plus_test/src/cluster/snapshots/hydroflow_plus_test__cluster__simple_cluster__tests__simple_cluster.snap @@ -4,7 +4,7 @@ expression: built.ir() --- [ ForEach { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , (u32 , i32)) , () > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | (id , d) | println ! ("node received: ({}, {:?})" , id , d) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < () > , (hydroflow_plus :: location :: ClusterId < () > , i32)) , () > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | (id , d) | println ! ("node received: ({}, {:?})" , id , d) }), input: Network { from_location: Cluster( 1, @@ -19,7 +19,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| data | { hydroflow_plus :: runtime_support :: bincode :: serialize :: < (u32 , i32) > (& data) . unwrap () . into () }", + "| data | { hydroflow_plus :: runtime_support :: bincode :: serialize :: < (hydroflow_plus :: location :: ClusterId < () > , i32) > (& data) . unwrap () . into () }", ], }, ), @@ -30,13 +30,13 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { let (id , b) = res . unwrap () ; (id , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (u32 , i32) > (& b) . unwrap ()) }", + "| res | { let (id , b) = res . unwrap () ; (hydroflow_plus :: ClusterId :: < () > :: from_raw (id) , hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (hydroflow_plus :: location :: ClusterId < () > , i32) > (& b) . unwrap ()) }", ], }, ), ), input: Inspect { - f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (u32 , i32) , () > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; let cluster_self_id = __hydroflow_plus_cluster_self_id_1 ; move | n | println ! ("cluster received: {:?} (self cluster id: {})" , n , cluster_self_id) }), + f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydroflow_plus :: location :: ClusterId < () > , i32) , () > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; let cluster_self_id = hydroflow_plus :: ClusterId :: < () > :: from_raw (__hydroflow_plus_cluster_self_id_1) ; move | n | println ! ("cluster received: {:?} (self cluster id: {})" , n , cluster_self_id) }), input: Network { from_location: Process( 0, @@ -51,7 +51,7 @@ expression: built.ir() Operator { path: "map", args: [ - "| (id , data) | { (id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < (u32 , i32) > (& data) . unwrap () . into ()) }", + "| (id , data) : (hydroflow_plus :: ClusterId < _ > , (hydroflow_plus :: location :: ClusterId < () > , i32)) | { (id . raw_id , hydroflow_plus :: runtime_support :: bincode :: serialize :: < (hydroflow_plus :: location :: ClusterId < () > , i32) > (& data) . unwrap () . into ()) }", ], }, ), @@ -62,21 +62,21 @@ expression: built.ir() Operator { path: "map", args: [ - "| res | { hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (u32 , i32) > (& res . unwrap ()) . unwrap () }", + "| res | { hydroflow_plus :: runtime_support :: bincode :: deserialize :: < (hydroflow_plus :: location :: ClusterId < () > , i32) > (& res . unwrap ()) . unwrap () }", ], }, ), ), input: Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , i32) , (u32 , (u32 , i32)) > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | (id , n) | (id , (id , n)) }), + f: stageleft :: runtime_support :: fn1_type_hint :: < (hydroflow_plus :: location :: ClusterId < () > , i32) , (hydroflow_plus :: location :: ClusterId < () > , (hydroflow_plus :: location :: ClusterId < () > , i32)) > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | (id , n) | (id , (id , n)) }), input: Delta( CrossProduct( Persist( Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < & u32 , u32 > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | & id | id }), + f: stageleft :: runtime_support :: fn1_type_hint :: < & hydroflow_plus :: location :: ClusterId < () > , hydroflow_plus :: location :: ClusterId < () > > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | & id | id }), input: Source { source: Iter( - __hydroflow_plus_cluster_ids_1, + unsafe { :: std :: mem :: transmute :: < _ , & :: std :: vec :: Vec < hydroflow_plus :: ClusterId < () > > > (__hydroflow_plus_cluster_ids_1) }, ), location_kind: Process( 0, diff --git a/hydroflow_plus_test/src/cluster/two_pc.rs b/hydroflow_plus_test/src/cluster/two_pc.rs index f93516970cf9..42f944cb419f 100644 --- a/hydroflow_plus_test/src/cluster/two_pc.rs +++ b/hydroflow_plus_test/src/cluster/two_pc.rs @@ -69,7 +69,7 @@ pub fn two_pc<'a>( .map(q!(|(id, (t, _reply))| (t, id))) // fold_keyed: 1 input stream of type (K, V1), 1 output stream of type (K, V2). // The output will have one tuple for each distinct K, with an accumulated value of type V2. - .tick_batch().fold_keyed(q!(|| 0), q!(|old: &mut u32, _: u32| *old += 1)).filter_map(q!(move |(t, count)| { + .tick_batch().fold_keyed(q!(|| 0), q!(|old: &mut u32, _| *old += 1)).filter_map(q!(move |(t, count)| { // here I set the participant to 3. If want more or less participant, fix line 26 of examples/broadcast.rs if count == num_participants { Some(t)