From eee28d3a17ea542c69a2d7e535c38333f42d4398 Mon Sep 17 00:00:00 2001 From: David Chu Date: Wed, 29 Jan 2025 14:45:02 +0800 Subject: [PATCH] feat(hydro_lang): Add metadata field to HydroNode (#1632) --- hydro_lang/src/builder/mod.rs | 26 +- hydro_lang/src/ir.rs | 360 +- hydro_lang/src/location/external_process.rs | 58 +- hydro_lang/src/location/mod.rs | 68 +- hydro_lang/src/location/tick.rs | 11 +- hydro_lang/src/optional.rs | 199 +- hydro_lang/src/rewrites/persist_pullup.rs | 198 +- hydro_lang/src/rewrites/profiler.rs | 2 + ...p__tests__persist_pullup_behind_tee-2.snap | 76 +- ...lup__tests__persist_pullup_behind_tee.snap | 176 +- ...__tests__persist_pullup_through_map-2.snap | 16 + ...up__tests__persist_pullup_through_map.snap | 42 +- ...ts__profiler_wrapping_all_operators-2.snap | 32 + ...ests__profiler_wrapping_all_operators.snap | 42 +- ...properties__tests__property_optimized.snap | 24 + hydro_lang/src/singleton.rs | 142 +- hydro_lang/src/stream.rs | 252 +- hydro_std/Cargo.toml | 4 +- ...ter__compute_pi__tests__compute_pi_ir.snap | 126 +- ...er__many_to_many__tests__many_to_many.snap | 24 + ...ter__map_reduce__tests__map_reduce_ir.snap | 102 +- ...cluster__paxos_bench__tests__paxos_ir.snap | 3949 +++++++++++++++-- ...simple_cluster__tests__simple_cluster.snap | 102 +- ...rst_ten__tests__first_ten_distributed.snap | 40 + 24 files changed, 5310 insertions(+), 761 deletions(-) diff --git a/hydro_lang/src/builder/mod.rs b/hydro_lang/src/builder/mod.rs index 6aa6859cd86..78895e22bde 100644 --- a/hydro_lang/src/builder/mod.rs +++ b/hydro_lang/src/builder/mod.rs @@ -37,6 +37,9 @@ pub struct FlowStateInner { /// Counters for clock IDs. pub(crate) next_clock_id: usize, + + /// Counter for unique HydroNode IDs. + pub(crate) next_node_id: usize, } pub type FlowState = Rc>; @@ -49,7 +52,7 @@ pub struct FlowBuilder<'a> { clusters: RefCell>, externals: RefCell>, - next_node_id: RefCell, + next_location_id: RefCell, /// Tracks whether this flow has been finalized; it is an error to /// drop without finalizing. @@ -88,11 +91,12 @@ impl<'a> FlowBuilder<'a> { next_external_out: 0, cycle_counts: HashMap::new(), next_clock_id: 0, + next_node_id: 0, })), processes: RefCell::new(vec![]), clusters: RefCell::new(vec![]), externals: RefCell::new(vec![]), - next_node_id: RefCell::new(0), + next_location_id: RefCell::new(0), finalized: false, _phantom: PhantomData, } @@ -130,9 +134,9 @@ impl<'a> FlowBuilder<'a> { } pub fn process

(&self) -> Process<'a, P> { - let mut next_node_id = self.next_node_id.borrow_mut(); - let id = *next_node_id; - *next_node_id += 1; + let mut next_location_id = self.next_location_id.borrow_mut(); + let id = *next_location_id; + *next_location_id += 1; self.processes .borrow_mut() @@ -146,9 +150,9 @@ impl<'a> FlowBuilder<'a> { } pub fn external_process

(&self) -> ExternalProcess<'a, P> { - let mut next_node_id = self.next_node_id.borrow_mut(); - let id = *next_node_id; - *next_node_id += 1; + let mut next_location_id = self.next_location_id.borrow_mut(); + let id = *next_location_id; + *next_location_id += 1; self.externals .borrow_mut() @@ -162,9 +166,9 @@ impl<'a> FlowBuilder<'a> { } pub fn cluster(&self) -> Cluster<'a, C> { - let mut next_node_id = self.next_node_id.borrow_mut(); - let id = *next_node_id; - *next_node_id += 1; + let mut next_location_id = self.next_location_id.borrow_mut(); + let id = *next_location_id; + *next_location_id += 1; self.clusters .borrow_mut() diff --git a/hydro_lang/src/ir.rs b/hydro_lang/src/ir.rs index 2ffd432d0f2..adada879126 100644 --- a/hydro_lang/src/ir.rs +++ b/hydro_lang/src/ir.rs @@ -4,6 +4,7 @@ use std::cell::RefCell; use std::collections::BTreeMap; use std::collections::HashMap; use std::fmt::Debug; +use std::hash::{Hash, Hasher}; use std::ops::Deref; use std::rc::Rc; @@ -20,7 +21,7 @@ use syn::parse_quote; use crate::deploy::{Deploy, RegisterPort}; use crate::location::LocationId; -#[derive(Clone)] +#[derive(Clone, Hash)] pub struct DebugExpr(pub syn::Expr); impl From for DebugExpr { @@ -49,6 +50,35 @@ impl Debug for DebugExpr { } } +#[derive(Clone, Hash)] +pub struct DebugType(pub syn::Type); + +impl From for DebugType { + fn from(t: syn::Type) -> DebugType { + DebugType(t) + } +} + +impl Deref for DebugType { + type Target = syn::Type; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl ToTokens for DebugType { + fn to_tokens(&self, tokens: &mut TokenStream) { + self.0.to_tokens(tokens); + } +} + +impl Debug for DebugType { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.to_token_stream()) + } +} + pub enum DebugInstantiate { Building(), Finalized(syn::Expr, syn::Expr, Option>), @@ -60,8 +90,14 @@ impl Debug for DebugInstantiate { } } +impl Hash for DebugInstantiate { + fn hash(&self, _state: &mut H) { + // Do nothing + } +} + /// A source in a Hydro graph, where data enters the graph. -#[derive(Debug)] +#[derive(Debug, Hash)] pub enum HydroSource { Stream(DebugExpr), ExternalNetwork(), @@ -72,7 +108,7 @@ pub enum HydroSource { /// An leaf in a Hydro graph, which is an pipeline that doesn't emit /// any downstream values. Traversals over the dataflow graph and /// generating Hydroflow IR start from leaves. -#[derive(Debug)] +#[derive(Debug, Hash)] pub enum HydroLeaf { ForEach { f: DebugExpr, @@ -261,85 +297,159 @@ impl Debug for TeeNode { } } +impl Hash for TeeNode { + fn hash(&self, state: &mut H) { + self.0.borrow_mut().hash(state); + } +} + +#[derive(Debug, Clone, Hash)] +pub struct HydroNodeMetadata { + pub location_kind: LocationId, + pub output_type: Option, +} + /// An intermediate node in a Hydro graph, which consumes data /// from upstream nodes and emits data to downstream nodes. -#[derive(Debug)] +#[derive(Debug, Hash)] pub enum HydroNode { Placeholder, Source { source: HydroSource, location_kind: LocationId, + metadata: HydroNodeMetadata, }, CycleSource { ident: syn::Ident, location_kind: LocationId, + metadata: HydroNodeMetadata, }, Tee { inner: TeeNode, + metadata: HydroNodeMetadata, + }, + + Persist { + inner: Box, + metadata: HydroNodeMetadata, + }, + + Unpersist { + inner: Box, + metadata: HydroNodeMetadata, + }, + + Delta { + inner: Box, + metadata: HydroNodeMetadata, + }, + + Chain { + first: Box, + second: Box, + metadata: HydroNodeMetadata, }, - Persist(Box), - Unpersist(Box), - Delta(Box), + CrossProduct { + left: Box, + right: Box, + metadata: HydroNodeMetadata, + }, + + CrossSingleton { + left: Box, + right: Box, + metadata: HydroNodeMetadata, + }, + + Join { + left: Box, + right: Box, + metadata: HydroNodeMetadata, + }, + + Difference { + pos: Box, + neg: Box, + metadata: HydroNodeMetadata, + }, - Chain(Box, Box), - CrossProduct(Box, Box), - CrossSingleton(Box, Box), - Join(Box, Box), - Difference(Box, Box), - AntiJoin(Box, Box), + AntiJoin { + pos: Box, + neg: Box, + metadata: HydroNodeMetadata, + }, Map { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, FlatMap { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, Filter { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, FilterMap { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, - DeferTick(Box), + DeferTick { + input: Box, + metadata: HydroNodeMetadata, + }, Enumerate { is_static: bool, input: Box, + metadata: HydroNodeMetadata, }, Inspect { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, - Unique(Box), + Unique { + input: Box, + metadata: HydroNodeMetadata, + }, - Sort(Box), + Sort { + input: Box, + metadata: HydroNodeMetadata, + }, Fold { init: DebugExpr, acc: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, FoldKeyed { init: DebugExpr, acc: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, Reduce { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, ReduceKeyed { f: DebugExpr, input: Box, + metadata: HydroNodeMetadata, }, Network { @@ -351,6 +461,7 @@ pub enum HydroNode { instantiate_fn: DebugInstantiate, deserialize_fn: Option, input: Box, + metadata: HydroNodeMetadata, }, } @@ -438,7 +549,7 @@ impl<'a> HydroNode { HydroNode::CycleSource { .. } => {} - HydroNode::Tee { inner } => { + HydroNode::Tee { inner, .. } => { if let Some(transformed) = seen_tees.get(&(inner.0.as_ref() as *const RefCell)) { @@ -456,33 +567,33 @@ impl<'a> HydroNode { } } - HydroNode::Persist(inner) => transform(inner.as_mut(), seen_tees), - HydroNode::Unpersist(inner) => transform(inner.as_mut(), seen_tees), - HydroNode::Delta(inner) => transform(inner.as_mut(), seen_tees), + HydroNode::Persist { inner, .. } => transform(inner.as_mut(), seen_tees), + HydroNode::Unpersist { inner, .. } => transform(inner.as_mut(), seen_tees), + HydroNode::Delta { inner, .. } => transform(inner.as_mut(), seen_tees), - HydroNode::Chain(left, right) => { - transform(left.as_mut(), seen_tees); - transform(right.as_mut(), seen_tees); + HydroNode::Chain { first, second, .. } => { + transform(first.as_mut(), seen_tees); + transform(second.as_mut(), seen_tees); } - HydroNode::CrossProduct(left, right) => { + HydroNode::CrossProduct { left, right, .. } => { transform(left.as_mut(), seen_tees); transform(right.as_mut(), seen_tees); } - HydroNode::CrossSingleton(left, right) => { + HydroNode::CrossSingleton { left, right, .. } => { transform(left.as_mut(), seen_tees); transform(right.as_mut(), seen_tees); } - HydroNode::Join(left, right) => { + HydroNode::Join { left, right, .. } => { transform(left.as_mut(), seen_tees); transform(right.as_mut(), seen_tees); } - HydroNode::Difference(left, right) => { - transform(left.as_mut(), seen_tees); - transform(right.as_mut(), seen_tees); + HydroNode::Difference { pos, neg, .. } => { + transform(pos.as_mut(), seen_tees); + transform(neg.as_mut(), seen_tees); } - HydroNode::AntiJoin(left, right) => { - transform(left.as_mut(), seen_tees); - transform(right.as_mut(), seen_tees); + HydroNode::AntiJoin { pos, neg, .. } => { + transform(pos.as_mut(), seen_tees); + transform(neg.as_mut(), seen_tees); } HydroNode::Map { input, .. } => { @@ -497,10 +608,10 @@ impl<'a> HydroNode { HydroNode::FilterMap { input, .. } => { transform(input.as_mut(), seen_tees); } - HydroNode::Sort(input) => { + HydroNode::Sort { input, .. } => { transform(input.as_mut(), seen_tees); } - HydroNode::DeferTick(input) => { + HydroNode::DeferTick { input, .. } => { transform(input.as_mut(), seen_tees); } HydroNode::Enumerate { input, .. } => { @@ -510,7 +621,7 @@ impl<'a> HydroNode { transform(input.as_mut(), seen_tees); } - HydroNode::Unique(input) => { + HydroNode::Unique { input, .. } => { transform(input.as_mut(), seen_tees); } @@ -546,7 +657,7 @@ impl<'a> HydroNode { panic!() } - HydroNode::Persist(inner) => { + HydroNode::Persist { inner, .. } => { let (inner_ident, location) = inner.emit(graph_builders, built_tees, next_stmt_id); let persist_id = *next_stmt_id; @@ -563,11 +674,11 @@ impl<'a> HydroNode { (persist_ident, location) } - HydroNode::Unpersist(_) => { + HydroNode::Unpersist { .. } => { panic!("Unpersist is a marker node and should have been optimized away. This is likely a compiler bug.") } - HydroNode::Delta(inner) => { + HydroNode::Delta { inner, .. } => { let (inner_ident, location) = inner.emit(graph_builders, built_tees, next_stmt_id); let delta_id = *next_stmt_id; @@ -587,6 +698,7 @@ impl<'a> HydroNode { HydroNode::Source { source, location_kind, + .. } => { let location_id = match location_kind { LocationId::Process(id) => id, @@ -640,6 +752,7 @@ impl<'a> HydroNode { HydroNode::CycleSource { ident, location_kind, + .. } => { let location_id = match location_kind.root() { LocationId::Process(id) => id, @@ -651,7 +764,7 @@ impl<'a> HydroNode { (ident.clone(), *location_id) } - HydroNode::Tee { inner } => { + HydroNode::Tee { inner, .. } => { if let Some(ret) = built_tees.get(&(inner.0.as_ref() as *const RefCell)) { ret.clone() @@ -682,14 +795,14 @@ impl<'a> HydroNode { } } - HydroNode::Chain(left, right) => { - let (left_ident, left_location_id) = - left.emit(graph_builders, built_tees, next_stmt_id); - let (right_ident, right_location_id) = - right.emit(graph_builders, built_tees, next_stmt_id); + HydroNode::Chain { first, second, .. } => { + let (first_ident, first_location_id) = + first.emit(graph_builders, built_tees, next_stmt_id); + let (second_ident, second_location_id) = + second.emit(graph_builders, built_tees, next_stmt_id); assert_eq!( - left_location_id, right_location_id, + first_location_id, second_location_id, "chain inputs must be in the same location" ); @@ -699,23 +812,23 @@ impl<'a> HydroNode { let chain_ident = syn::Ident::new(&format!("stream_{}", union_id), Span::call_site()); - let builder = graph_builders.entry(left_location_id).or_default(); + let builder = graph_builders.entry(first_location_id).or_default(); builder.add_statement(parse_quote! { #chain_ident = chain(); }); builder.add_statement(parse_quote! { - #left_ident -> [0]#chain_ident; + #first_ident -> [0]#chain_ident; }); builder.add_statement(parse_quote! { - #right_ident -> [1]#chain_ident; + #second_ident -> [1]#chain_ident; }); - (chain_ident, left_location_id) + (chain_ident, first_location_id) } - HydroNode::CrossSingleton(left, right) => { + HydroNode::CrossSingleton { left, right, .. } => { let (left_ident, left_location_id) = left.emit(graph_builders, built_tees, next_stmt_id); let (right_ident, right_location_id) = @@ -748,27 +861,28 @@ impl<'a> HydroNode { (cross_ident, left_location_id) } - HydroNode::CrossProduct(..) | HydroNode::Join(..) => { - let operator: syn::Ident = if matches!(self, HydroNode::CrossProduct(..)) { + HydroNode::CrossProduct { .. } | HydroNode::Join { .. } => { + let operator: syn::Ident = if matches!(self, HydroNode::CrossProduct { .. }) { parse_quote!(cross_join_multiset) } else { parse_quote!(join_multiset) }; - let (HydroNode::CrossProduct(left, right) | HydroNode::Join(left, right)) = self + let (HydroNode::CrossProduct { left, right, .. } + | HydroNode::Join { left, right, .. }) = self else { unreachable!() }; - let (left_inner, left_was_persist) = if let HydroNode::Persist(left) = left.as_ref() - { - (left, true) - } else { - (left, false) - }; + let (left_inner, left_was_persist) = + if let HydroNode::Persist { inner: left, .. } = left.as_ref() { + (left, true) + } else { + (left, false) + }; let (right_inner, right_was_persist) = - if let HydroNode::Persist(right) = right.as_ref() { + if let HydroNode::Persist { inner: right, .. } = right.as_ref() { (right, true) } else { (right, false) @@ -826,31 +940,33 @@ impl<'a> HydroNode { (stream_ident, left_location_id) } - HydroNode::Difference(..) | HydroNode::AntiJoin(..) => { - let operator: syn::Ident = if matches!(self, HydroNode::Difference(..)) { + HydroNode::Difference { .. } | HydroNode::AntiJoin { .. } => { + let operator: syn::Ident = if matches!(self, HydroNode::Difference { .. }) { parse_quote!(difference_multiset) } else { parse_quote!(anti_join_multiset) }; - let (HydroNode::Difference(left, right) | HydroNode::AntiJoin(left, right)) = self + let (HydroNode::Difference { pos, neg, .. } | HydroNode::AntiJoin { pos, neg, .. }) = + self else { unreachable!() }; - let (right, right_was_persist) = if let HydroNode::Persist(right) = right.as_ref() { - (right, true) - } else { - (right, false) - }; + let (neg, neg_was_persist) = + if let HydroNode::Persist { inner: neg, .. } = neg.as_ref() { + (neg, true) + } else { + (neg, false) + }; - let (left_ident, left_location_id) = - left.emit(graph_builders, built_tees, next_stmt_id); - let (right_ident, right_location_id) = - right.emit(graph_builders, built_tees, next_stmt_id); + let (pos_ident, pos_location_id) = + pos.emit(graph_builders, built_tees, next_stmt_id); + let (neg_ident, neg_location_id) = + neg.emit(graph_builders, built_tees, next_stmt_id); assert_eq!( - left_location_id, right_location_id, + pos_location_id, neg_location_id, "difference / anti join inputs must be in the same location" ); @@ -860,9 +976,9 @@ impl<'a> HydroNode { let stream_ident = syn::Ident::new(&format!("stream_{}", stream_id), Span::call_site()); - let builder = graph_builders.entry(left_location_id).or_default(); + let builder = graph_builders.entry(pos_location_id).or_default(); - if right_was_persist { + if neg_was_persist { builder.add_statement(parse_quote! { #stream_ident = #operator::<'tick, 'static>(); }); @@ -873,17 +989,17 @@ impl<'a> HydroNode { } builder.add_statement(parse_quote! { - #left_ident -> [pos]#stream_ident; + #pos_ident -> [pos]#stream_ident; }); builder.add_statement(parse_quote! { - #right_ident -> [neg]#stream_ident; + #neg_ident -> [neg]#stream_ident; }); - (stream_ident, left_location_id) + (stream_ident, pos_location_id) } - HydroNode::Map { f, input } => { + HydroNode::Map { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -900,7 +1016,7 @@ impl<'a> HydroNode { (map_ident, input_location_id) } - HydroNode::FlatMap { f, input } => { + HydroNode::FlatMap { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -918,7 +1034,7 @@ impl<'a> HydroNode { (flat_map_ident, input_location_id) } - HydroNode::Filter { f, input } => { + HydroNode::Filter { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -936,7 +1052,7 @@ impl<'a> HydroNode { (filter_ident, input_location_id) } - HydroNode::FilterMap { f, input } => { + HydroNode::FilterMap { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -954,7 +1070,7 @@ impl<'a> HydroNode { (filter_map_ident, input_location_id) } - HydroNode::Sort(input) => { + HydroNode::Sort { input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -971,7 +1087,7 @@ impl<'a> HydroNode { (sort_ident, input_location_id) } - HydroNode::DeferTick(input) => { + HydroNode::DeferTick { input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -989,7 +1105,9 @@ impl<'a> HydroNode { (defer_tick_ident, input_location_id) } - HydroNode::Enumerate { is_static, input } => { + HydroNode::Enumerate { + is_static, input, .. + } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1014,7 +1132,7 @@ impl<'a> HydroNode { (enumerate_ident, input_location_id) } - HydroNode::Inspect { f, input } => { + HydroNode::Inspect { f, input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1032,7 +1150,7 @@ impl<'a> HydroNode { (inspect_ident, input_location_id) } - HydroNode::Unique(input) => { + HydroNode::Unique { input, .. } => { let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1057,17 +1175,22 @@ impl<'a> HydroNode { parse_quote!(fold_keyed) }; - let (HydroNode::Fold { init, acc, input } - | HydroNode::FoldKeyed { init, acc, input }) = self + let (HydroNode::Fold { + init, acc, input, .. + } + | HydroNode::FoldKeyed { + init, acc, input, .. + }) = self else { unreachable!() }; - let (input, input_was_persist) = if let HydroNode::Persist(input) = input.as_ref() { - (input, true) - } else { - (input, false) - }; + let (input, input_was_persist) = + if let HydroNode::Persist { inner: input, .. } = input.as_ref() { + (input, true) + } else { + (input, false) + }; let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1099,16 +1222,18 @@ impl<'a> HydroNode { parse_quote!(reduce_keyed) }; - let (HydroNode::Reduce { f, input } | HydroNode::ReduceKeyed { f, input }) = self + let (HydroNode::Reduce { f, input, .. } | HydroNode::ReduceKeyed { f, input, .. }) = + self else { unreachable!() }; - let (input, input_was_persist) = if let HydroNode::Persist(input) = input.as_ref() { - (input, true) - } else { - (input, false) - }; + let (input, input_was_persist) = + if let HydroNode::Persist { inner: input, .. } = input.as_ref() { + (input, true) + } else { + (input, false) + }; let (input_ident, input_location_id) = input.emit(graph_builders, built_tees, next_stmt_id); @@ -1142,6 +1267,7 @@ impl<'a> HydroNode { instantiate_fn, deserialize_fn: deserialize_pipeline, input, + .. } => { let (sink_expr, source_expr, _connect_fn) = match instantiate_fn { DebugInstantiate::Building() => { @@ -1196,6 +1322,40 @@ impl<'a> HydroNode { } } } + + pub fn metadata(&self) -> &HydroNodeMetadata { + match self { + HydroNode::Placeholder => { + panic!() + } + HydroNode::Source { metadata, .. } => metadata, + HydroNode::CycleSource { metadata, .. } => metadata, + HydroNode::Tee { metadata, .. } => metadata, + HydroNode::Persist { metadata, .. } => metadata, + HydroNode::Unpersist { metadata, .. } => metadata, + HydroNode::Delta { metadata, .. } => metadata, + HydroNode::Chain { metadata, .. } => metadata, + HydroNode::CrossProduct { metadata, .. } => metadata, + HydroNode::CrossSingleton { metadata, .. } => metadata, + HydroNode::Join { metadata, .. } => metadata, + HydroNode::Difference { metadata, .. } => metadata, + HydroNode::AntiJoin { metadata, .. } => metadata, + HydroNode::Map { metadata, .. } => metadata, + HydroNode::FlatMap { metadata, .. } => metadata, + HydroNode::Filter { metadata, .. } => metadata, + HydroNode::FilterMap { metadata, .. } => metadata, + HydroNode::DeferTick { metadata, .. } => metadata, + HydroNode::Enumerate { metadata, .. } => metadata, + HydroNode::Inspect { metadata, .. } => metadata, + HydroNode::Unique { metadata, .. } => metadata, + HydroNode::Sort { metadata, .. } => metadata, + HydroNode::Fold { metadata, .. } => metadata, + HydroNode::FoldKeyed { metadata, .. } => metadata, + HydroNode::Reduce { metadata, .. } => metadata, + HydroNode::ReduceKeyed { metadata, .. } => metadata, + HydroNode::Network { metadata, .. } => metadata, + } + } } #[cfg(feature = "build")] diff --git a/hydro_lang/src/location/external_process.rs b/hydro_lang/src/location/external_process.rs index 926e38de670..e18c9e2f0b2 100644 --- a/hydro_lang/src/location/external_process.rs +++ b/hydro_lang/src/location/external_process.rs @@ -110,19 +110,24 @@ impl<'a, P> ExternalProcess<'a, P> { }, Stream::new( to.clone(), - HydroNode::Persist(Box::new(HydroNode::Network { - from_location: LocationId::ExternalProcess(self.id), - from_key: Some(next_external_port_id), - to_location: to.id(), - to_key: None, - serialize_fn: None, - instantiate_fn: crate::ir::DebugInstantiate::Building(), - deserialize_fn: Some(deser_expr.into()), - input: Box::new(HydroNode::Source { - source: HydroSource::ExternalNetwork(), - location_kind: LocationId::ExternalProcess(self.id), + HydroNode::Persist { + inner: Box::new(HydroNode::Network { + from_location: LocationId::ExternalProcess(self.id), + from_key: Some(next_external_port_id), + to_location: to.id(), + to_key: None, + serialize_fn: None, + instantiate_fn: crate::ir::DebugInstantiate::Building(), + deserialize_fn: Some(deser_expr.into()), + input: Box::new(HydroNode::Source { + source: HydroSource::ExternalNetwork(), + location_kind: LocationId::ExternalProcess(self.id), + metadata: self.new_node_metadata::(), + }), + metadata: to.new_node_metadata::(), }), - })), + metadata: to.new_node_metadata::(), + }, ), ) } @@ -146,19 +151,24 @@ impl<'a, P> ExternalProcess<'a, P> { }, Stream::new( to.clone(), - HydroNode::Persist(Box::new(HydroNode::Network { - from_location: LocationId::ExternalProcess(self.id), - from_key: Some(next_external_port_id), - to_location: to.id(), - to_key: None, - serialize_fn: None, - instantiate_fn: crate::ir::DebugInstantiate::Building(), - deserialize_fn: Some(crate::stream::deserialize_bincode::(None).into()), - input: Box::new(HydroNode::Source { - source: HydroSource::ExternalNetwork(), - location_kind: LocationId::ExternalProcess(self.id), + HydroNode::Persist { + inner: Box::new(HydroNode::Network { + from_location: LocationId::ExternalProcess(self.id), + from_key: Some(next_external_port_id), + to_location: to.id(), + to_key: None, + serialize_fn: None, + instantiate_fn: crate::ir::DebugInstantiate::Building(), + deserialize_fn: Some(crate::stream::deserialize_bincode::(None).into()), + input: Box::new(HydroNode::Source { + source: HydroSource::ExternalNetwork(), + location_kind: LocationId::ExternalProcess(self.id), + metadata: self.new_node_metadata::(), + }), + metadata: to.new_node_metadata::(), }), - })), + metadata: to.new_node_metadata::(), + }, ), ) } diff --git a/hydro_lang/src/location/mod.rs b/hydro_lang/src/location/mod.rs index 8b91d1d81e1..f8902476ed3 100644 --- a/hydro_lang/src/location/mod.rs +++ b/hydro_lang/src/location/mod.rs @@ -9,7 +9,7 @@ use stageleft::{q, QuotedWithContext}; use super::builder::FlowState; use crate::cycle::{CycleCollection, ForwardRef, ForwardRefMarker}; -use crate::ir::{HydroNode, HydroSource}; +use crate::ir::{DebugType, HydroNode, HydroNodeMetadata, HydroSource}; use crate::{Singleton, Stream, Unbounded}; pub mod external_process; @@ -27,7 +27,7 @@ pub use can_send::CanSend; pub mod tick; pub use tick::{NoTick, Tick, Timestamped}; -#[derive(PartialEq, Eq, Clone, Debug)] +#[derive(PartialEq, Eq, Clone, Debug, Hash)] pub enum LocationId { Process(usize), Cluster(usize), @@ -82,16 +82,33 @@ pub trait Location<'a>: Clone { } } + fn next_node_id(&self) -> usize { + let next_id = self.flow_state().borrow_mut().next_node_id; + self.flow_state().borrow_mut().next_node_id += 1; + next_id + } + + fn new_node_metadata(&self) -> HydroNodeMetadata { + HydroNodeMetadata { + location_kind: self.id(), + output_type: Some(DebugType(stageleft::quote_type::())), + } + } + fn spin(&self) -> Stream<(), Self, Unbounded> where Self: Sized + NoTick, { Stream::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Spin(), - location_kind: self.id(), - })), + HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Spin(), + location_kind: self.id(), + metadata: self.new_node_metadata::<()>(), + }), + metadata: self.new_node_metadata::<()>(), + }, ) } @@ -106,10 +123,14 @@ pub trait Location<'a>: Clone { Stream::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Stream(e.into()), - location_kind: self.id(), - })), + HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Stream(e.into()), + location_kind: self.id(), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }, ) } @@ -126,10 +147,14 @@ pub trait Location<'a>: Clone { Stream::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Iter(e.into()), - location_kind: self.id(), - })), + HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Iter(e.into()), + location_kind: self.id(), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }, ) } @@ -151,10 +176,17 @@ pub trait Location<'a>: Clone { // so that it grows every tick Singleton::new( self.clone(), - HydroNode::Persist(Box::new(HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Iter(e.into()), - location_kind: self.id(), - })))), + HydroNode::Persist { + inner: Box::new(HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Iter(e.into()), + location_kind: self.id(), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }), + metadata: self.new_node_metadata::(), + }, ) } diff --git a/hydro_lang/src/location/tick.rs b/hydro_lang/src/location/tick.rs index 2841db80646..a5f6f7a90c7 100644 --- a/hydro_lang/src/location/tick.rs +++ b/hydro_lang/src/location/tick.rs @@ -10,7 +10,7 @@ use crate::cycle::{ CycleCollection, CycleCollectionWithInitial, DeferTick, ForwardRef, ForwardRefMarker, TickCycle, TickCycleMarker, }; -use crate::ir::{HydroNode, HydroSource}; +use crate::ir::{HydroNode, HydroNodeMetadata, HydroSource}; use crate::{Bounded, Optional, Singleton, Stream}; #[sealed] @@ -82,6 +82,14 @@ impl<'a, L: Location<'a>> Location<'a> for Tick { fn is_top_level() -> bool { false } + + fn next_node_id(&self) -> usize { + self.l.next_node_id() + } + + fn new_node_metadata(&self) -> HydroNodeMetadata { + self.l.new_node_metadata::() + } } impl<'a, L: Location<'a>> Tick { @@ -138,6 +146,7 @@ impl<'a, L: Location<'a>> Tick { HydroNode::Source { source: HydroSource::Iter(e.into()), location_kind: self.l.id(), + metadata: self.new_node_metadata::(), }, ) } diff --git a/hydro_lang/src/optional.rs b/hydro_lang/src/optional.rs index dbfa591e0b3..b75ae8843f5 100644 --- a/hydro_lang/src/optional.rs +++ b/hydro_lang/src/optional.rs @@ -54,10 +54,11 @@ impl<'a, T, L: Location<'a>> CycleCollection<'a, TickCycleMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Optional::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -92,10 +93,11 @@ impl<'a, T, L: Location<'a>> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Optional::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -130,11 +132,15 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: L) -> Self { let location_id = location.id(); Optional::new( - location, - HydroNode::Persist(Box::new(HydroNode::CycleSource { - ident, - location_kind: location_id, - })), + location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CycleSource { + ident, + location_kind: location_id, + metadata: location.new_node_metadata::(), + }), + metadata: location.new_node_metadata::(), + }, ) } } @@ -155,7 +161,10 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleComplete<'a, ForwardRefMarker> for .push(HydroLeaf::CycleSink { ident, location_kind: self.location_kind(), - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), }); } } @@ -178,14 +187,16 @@ impl<'a, T: Clone, L: Location<'a>, B> Clone for Optional { let orig_ir_node = self.ir_node.replace(HydroNode::Placeholder); *self.ir_node.borrow_mut() = HydroNode::Tee { inner: TeeNode(Rc::new(RefCell::new(orig_ir_node))), + metadata: self.location.new_node_metadata::(), }; } - if let HydroNode::Tee { inner } = self.ir_node.borrow().deref() { + if let HydroNode::Tee { inner, metadata } = self.ir_node.borrow().deref() { Optional { location: self.location.clone(), ir_node: HydroNode::Tee { inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), } .into(), _phantom: PhantomData, @@ -218,10 +229,11 @@ impl<'a, T, L: Location<'a>, B> Optional { pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F, L>) -> Optional { let f = f.splice_fn1_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::Map { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -232,10 +244,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -246,10 +259,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -274,10 +288,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Optional { let f = f.splice_fn1_borrow_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::Filter { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -288,10 +303,11 @@ impl<'a, T, L: Location<'a>, B> Optional { ) -> Optional { let f = f.splice_fn1_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::FilterMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -301,19 +317,30 @@ impl<'a, T, L: Location<'a>, B> Optional { if L::is_top_level() { Optional::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Chain( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(other.ir_node.into_inner()))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Chain { + first: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + second: Box::new(HydroNode::Unpersist { + inner: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }, ) } else { Optional::new( - self.location, - HydroNode::Chain( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Chain { + first: Box::new(self.ir_node.into_inner()), + second: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -327,19 +354,30 @@ impl<'a, T, L: Location<'a>, B> Optional { if L::is_top_level() { Optional::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::CrossSingleton( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(other.ir_node.into_inner()))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CrossSingleton { + left: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + right: Box::new(HydroNode::Unpersist { + inner: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::<(T, O)>(), + }), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } else { Optional::new( - self.location, - HydroNode::CrossSingleton( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::CrossSingleton { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } } @@ -349,19 +387,30 @@ impl<'a, T, L: Location<'a>, B> Optional { if L::is_top_level() { Singleton::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Chain( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(other.ir_node.into_inner()))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Chain { + first: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + second: Box::new(HydroNode::Unpersist { + inner: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }, ) } else { Singleton::new( - self.location, - HydroNode::Chain( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Chain { + first: Box::new(self.ir_node.into_inner()), + second: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -371,13 +420,23 @@ impl<'a, T, L: Location<'a>, B> Optional { T: Clone, { let none: syn::Expr = parse_quote!([::std::option::Option::None]); - let core_ir = HydroNode::Persist(Box::new(HydroNode::Source { - source: HydroSource::Iter(none.into()), - location_kind: self.location.id().root().clone(), - })); + let core_ir = HydroNode::Persist { + inner: Box::new(HydroNode::Source { + source: HydroSource::Iter(none.into()), + location_kind: self.location.id().root().clone(), + metadata: self.location.new_node_metadata::>(), + }), + metadata: self.location.new_node_metadata::>(), + }; let none_singleton = if L::is_top_level() { - Singleton::new(self.location.clone(), HydroNode::Persist(Box::new(core_ir))) + Singleton::new( + self.location.clone(), + HydroNode::Persist { + inner: Box::new(core_ir), + metadata: self.location.new_node_metadata::>(), + }, + ) } else { Singleton::new(self.location.clone(), core_ir) }; @@ -427,8 +486,11 @@ impl<'a, T, L: Location<'a> + NoTick, B> Optional, B> { /// arbitrary point in time. pub unsafe fn latest_tick(self) -> Optional, Bounded> { Optional::new( - self.location.tick, - HydroNode::Unpersist(Box::new(self.ir_node.into_inner())), + self.location.clone().tick, + HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -501,39 +563,54 @@ impl<'a, T, L: Location<'a>> Optional, Bounded> { pub fn all_ticks(self) -> Stream, Unbounded> { Stream::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn latest(self) -> Optional, Unbounded> { Optional::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn defer_tick(self) -> Optional, Bounded> { Optional::new( - self.location, - HydroNode::DeferTick(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::DeferTick { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn persist(self) -> Stream, Bounded> { Stream::new( - self.location, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn delta(self) -> Optional, Bounded> { Optional::new( - self.location, - HydroNode::Delta(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Delta { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } diff --git a/hydro_lang/src/rewrites/persist_pullup.rs b/hydro_lang/src/rewrites/persist_pullup.rs index 49a0e9c4418..a4429d7a067 100644 --- a/hydro_lang/src/rewrites/persist_pullup.rs +++ b/hydro_lang/src/rewrites/persist_pullup.rs @@ -9,18 +9,23 @@ fn persist_pullup_node( ) { *node = match_box::match_box! { match std::mem::replace(node, HydroNode::Placeholder) { - HydroNode::Unpersist(mb!(* HydroNode::Persist(mb!(* behind_persist)))) => behind_persist, + HydroNode::Unpersist { inner: mb!(* HydroNode::Persist { inner: mb!(* behind_persist), .. }), .. } => behind_persist, - HydroNode::Delta(mb!(* HydroNode::Persist(mb!(* behind_persist)))) => behind_persist, + HydroNode::Delta { inner: mb!(* HydroNode::Persist { inner: mb!(* behind_persist), .. }), .. } => behind_persist, - HydroNode::Tee { inner } => { + // TODO: Figure out if persist needs to copy its metadata or can just use original metadata here. If it can just use original, figure out where that is + HydroNode::Tee { inner, metadata } => { if persist_pulled_tees.contains(&(inner.0.as_ref() as *const RefCell)) { - HydroNode::Persist(Box::new(HydroNode::Tee { - inner: TeeNode(inner.0.clone()), - })) - } else if matches!(*inner.0.borrow(), HydroNode::Persist(_)) { + HydroNode::Persist { + inner: Box::new(HydroNode::Tee { + inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + } + } else if matches!(*inner.0.borrow(), HydroNode::Persist { .. }) { persist_pulled_tees.insert(inner.0.as_ref() as *const RefCell); - if let HydroNode::Persist(behind_persist) = + if let HydroNode::Persist { inner: behind_persist, .. } = inner.0.replace(HydroNode::Placeholder) { *inner.0.borrow_mut() = *behind_persist; @@ -28,45 +33,69 @@ fn persist_pullup_node( unreachable!() } - HydroNode::Persist(Box::new(HydroNode::Tee { - inner: TeeNode(inner.0.clone()), - })) + HydroNode::Persist { + inner: Box::new(HydroNode::Tee { + inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + } } else { - HydroNode::Tee { inner } + HydroNode::Tee { inner, metadata } } } HydroNode::Map { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::Map { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::Map { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + }, HydroNode::FilterMap { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::FilterMap { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::FilterMap { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, HydroNode::FlatMap { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::FlatMap { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::FlatMap { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, HydroNode::Filter { f, - input: mb!(* HydroNode::Persist(behind_persist)), - } => HydroNode::Persist(Box::new(HydroNode::Filter { - f, - input: behind_persist, - })), + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::Filter { + f, + input: behind_persist, + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, HydroNode::Network { from_location, @@ -76,44 +105,73 @@ fn persist_pullup_node( serialize_fn, instantiate_fn, deserialize_fn, - input: mb!(* HydroNode::Persist(behind_persist)), - .. - } => HydroNode::Persist(Box::new(HydroNode::Network { - from_location, - from_key, - to_location, - to_key, - serialize_fn, - instantiate_fn, - deserialize_fn, - input: behind_persist, - })), - - HydroNode::Chain(mb!(* HydroNode::Persist(left)), mb!(* HydroNode::Persist(right))) => { - HydroNode::Persist(Box::new(HydroNode::Chain(left, right))) - } - - HydroNode::CrossProduct(mb!(* HydroNode::Persist(left)), mb!(* HydroNode::Persist(right))) => { - HydroNode::Persist(Box::new(HydroNode::Delta(Box::new( - HydroNode::CrossProduct( - Box::new(HydroNode::Persist(left)), - Box::new(HydroNode::Persist(right)), - ), - )))) - } - - HydroNode::Join(mb!(* HydroNode::Persist(left)), mb!(* HydroNode::Persist(right))) => { - HydroNode::Persist(Box::new(HydroNode::Delta(Box::new(HydroNode::Join( - Box::new(HydroNode::Persist(left)), - Box::new(HydroNode::Persist(right)), - ))))) - } - - HydroNode::Unique(mb!(* HydroNode::Persist(inner))) => { - HydroNode::Persist(Box::new(HydroNode::Delta(Box::new(HydroNode::Unique( - Box::new(HydroNode::Persist(inner)), - ))))) - } + input: mb!(* HydroNode::Persist { inner: behind_persist, .. }), + metadata, + } => HydroNode::Persist { + inner: Box::new(HydroNode::Network { + from_location, + from_key, + to_location, + to_key, + serialize_fn, + instantiate_fn, + deserialize_fn, + input: behind_persist, + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }, + + HydroNode::Chain { + first: mb!(* HydroNode::Persist { inner: first, metadata: persist_metadata }), + second: mb!(* HydroNode::Persist { inner: second, .. }), + metadata + } => HydroNode::Persist { + inner: Box::new(HydroNode::Chain { first, second, metadata }), + metadata: persist_metadata + }, + + HydroNode::CrossProduct { + left: mb!(* HydroNode::Persist { inner: left, metadata: left_metadata }), + right: mb!(* HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata + } => HydroNode::Persist { + inner: Box::new(HydroNode::Delta { + inner: Box::new(HydroNode::CrossProduct { + left: Box::new(HydroNode::Persist { inner: left, metadata: left_metadata }), + right: Box::new(HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + }, + HydroNode::Join { + left: mb!(* HydroNode::Persist { inner: left, metadata: left_metadata }), + right: mb!(* HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata + } => HydroNode::Persist { + inner: Box::new(HydroNode::Delta { + inner: Box::new(HydroNode::Join { + left: Box::new(HydroNode::Persist { inner: left, metadata: left_metadata }), + right: Box::new(HydroNode::Persist { inner: right, metadata: right_metadata }), + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }), + metadata: metadata.clone(), + }, + + HydroNode::Unique { input: mb!(* HydroNode::Persist {inner, metadata: persist_metadata } ), metadata } => HydroNode::Persist { + inner: Box::new(HydroNode::Delta { + inner: Box::new(HydroNode::Unique { + input: Box::new(HydroNode::Persist { inner, metadata: persist_metadata }), + metadata: metadata.clone() + }), + metadata: metadata.clone(), + }), + metadata: metadata.clone() + }, node => node, } diff --git a/hydro_lang/src/rewrites/profiler.rs b/hydro_lang/src/rewrites/profiler.rs index 8e2b01a0839..197dcc73f32 100644 --- a/hydro_lang/src/rewrites/profiler.rs +++ b/hydro_lang/src/rewrites/profiler.rs @@ -30,6 +30,7 @@ fn add_profiling_node<'a>( seen_tees, ); let orig_node = std::mem::replace(node, HydroNode::Placeholder); + let new_metadata = orig_node.metadata().clone(); *node = HydroNode::Inspect { f: quoted_any_fn(q!({ // Put counters on queue @@ -45,6 +46,7 @@ fn add_profiling_node<'a>( .splice_untyped() .into(), input: Box::new(orig_node), + metadata: new_metadata, } } diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap index dcb22d2ce6c..0838490b137 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee-2.snap @@ -5,8 +5,8 @@ expression: optimized.ir() [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Persist( - Map { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { inner: : Source { @@ -16,15 +16,47 @@ expression: optimized.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, }, ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Persist( - Map { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { inner: : Source { @@ -34,9 +66,41 @@ expression: optimized.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap index d960d065ddf..12b1fb81402 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_behind_tee.snap @@ -5,54 +5,194 @@ expression: built.ir() [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Persist( - Map { + input: Unpersist { + inner: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { - inner: : Persist( - Unpersist( - Persist( - Source { + inner: : Persist { + inner: Unpersist { + inner: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, }, ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Persist( - Map { + input: Unpersist { + inner: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), input: Tee { - inner: : Persist( - Unpersist( - Persist( - Source { + inner: : Persist { + inner: Unpersist { + inner: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Process( + 0, + ), + ), + output_type: Some( + i32, + ), + }, + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap index 667076ada4c..88e70589f0e 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map-2.snap @@ -14,6 +14,22 @@ expression: optimized.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, }, diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap index bd39e1a4076..47de226aada 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__persist_pullup__tests__persist_pullup_through_map.snap @@ -5,20 +5,52 @@ expression: built.ir() [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Map { + input: Unpersist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; | v | v + 1 }), - input: Persist( - Source { + input: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: persist_pullup :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, - ), + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap index 0416e6a9a06..2509e54f6b8 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators-2.snap @@ -18,9 +18,41 @@ expression: "&pushed_down.ir()" location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap index 6e0a06f8ade..f5d79eeaa2d 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__profiler__tests__profiler_wrapping_all_operators.snap @@ -5,20 +5,52 @@ expression: "&built.ir()" [ ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , () > ({ use crate :: __staged :: rewrites :: profiler :: tests :: * ; | n | println ! ("{}" , n) }), - input: Unpersist( - Map { + input: Unpersist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < i32 , i32 > ({ use crate :: __staged :: rewrites :: profiler :: tests :: * ; | v | v + 1 }), - input: Persist( - Source { + input: Persist { + inner: Source { source: Iter( { use crate :: __staged :: rewrites :: profiler :: tests :: * ; 0 .. 10 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, ), }, - ), + }, }, ] diff --git a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap index 4c74a724fe9..dfcc8af6253 100644 --- a/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap +++ b/hydro_lang/src/rewrites/snapshots/hydro_lang__rewrites__properties__tests__property_optimized.snap @@ -17,7 +17,31 @@ expression: built.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + std :: string :: String, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), }, }, }, diff --git a/hydro_lang/src/singleton.rs b/hydro_lang/src/singleton.rs index 5994670ce95..c5e8d00dea7 100644 --- a/hydro_lang/src/singleton.rs +++ b/hydro_lang/src/singleton.rs @@ -56,14 +56,16 @@ impl<'a, T, L: Location<'a>> CycleCollectionWithInitial<'a, TickCycleMarker> fn create_source(ident: syn::Ident, initial: Self, location: Tick) -> Self { let location_id = location.id(); Singleton::new( - location, - HydroNode::Chain( - Box::new(HydroNode::CycleSource { + location.clone(), + HydroNode::Chain { + first: Box::new(HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }), - initial.ir_node.into_inner().into(), - ), + second: initial.ir_node.into_inner().into(), + metadata: location.new_node_metadata::(), + }, ) } } @@ -97,10 +99,11 @@ impl<'a, T, L: Location<'a>> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Singleton::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -137,11 +140,15 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleCollection<'a, ForwardRefMarker> fn create_source(ident: syn::Ident, location: L) -> Self { let location_id = location.id(); Singleton::new( - location, - HydroNode::Persist(Box::new(HydroNode::CycleSource { - ident, - location_kind: location_id, - })), + location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CycleSource { + ident, + location_kind: location_id, + metadata: location.new_node_metadata::(), + }), + metadata: location.new_node_metadata::(), + }, ) } } @@ -164,7 +171,10 @@ impl<'a, T, L: Location<'a> + NoTick, B> CycleComplete<'a, ForwardRefMarker> .push(HydroLeaf::CycleSink { ident, location_kind: self.location_kind(), - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), }); } } @@ -175,14 +185,16 @@ impl<'a, T: Clone, L: Location<'a>, B> Clone for Singleton { let orig_ir_node = self.ir_node.replace(HydroNode::Placeholder); *self.ir_node.borrow_mut() = HydroNode::Tee { inner: TeeNode(Rc::new(RefCell::new(orig_ir_node))), + metadata: self.location.new_node_metadata::(), }; } - if let HydroNode::Tee { inner } = self.ir_node.borrow().deref() { + if let HydroNode::Tee { inner, metadata } = self.ir_node.borrow().deref() { Singleton { location: self.location.clone(), ir_node: HydroNode::Tee { inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), } .into(), _phantom: PhantomData, @@ -197,10 +209,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { pub fn map U + 'a>(self, f: impl IntoQuotedMut<'a, F, L>) -> Singleton { let f = f.splice_fn1_ctx(&self.location).into(); Singleton::new( - self.location, + self.location.clone(), HydroNode::Map { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -211,10 +224,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -225,10 +239,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -239,10 +254,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Optional { let f = f.splice_fn1_borrow_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::Filter { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -253,10 +269,11 @@ impl<'a, T, L: Location<'a>, B> Singleton { ) -> Optional { let f = f.splice_fn1_ctx(&self.location).into(); Optional::new( - self.location, + self.location.clone(), HydroNode::FilterMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -268,20 +285,42 @@ impl<'a, T, L: Location<'a>, B> Singleton { check_matching_location(&self.location, &Self::other_location(&other)); if L::is_top_level() { + let left_ir_node = self.ir_node.into_inner(); + let left_ir_node_metadata = left_ir_node.metadata().clone(); + let right_ir_node = Self::other_ir_node(other); + let right_ir_node_metadata = right_ir_node.metadata().clone(); + Self::make( - self.location, - HydroNode::Persist(Box::new(HydroNode::CrossSingleton( - Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - Box::new(HydroNode::Unpersist(Box::new(Self::other_ir_node(other)))), - ))), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CrossSingleton { + left: Box::new(HydroNode::Unpersist { + inner: Box::new(left_ir_node), + metadata: left_ir_node_metadata, + }), + right: Box::new(HydroNode::Unpersist { + inner: Box::new(right_ir_node), + metadata: right_ir_node_metadata, + }), + metadata: self + .location + .new_node_metadata::<>::ElementType>(), + }), + metadata: self + .location + .new_node_metadata::<>::ElementType>(), + }, ) } else { Self::make( - self.location, - HydroNode::CrossSingleton( - Box::new(self.ir_node.into_inner()), - Box::new(Self::other_ir_node(other)), - ), + self.location.clone(), + HydroNode::CrossSingleton { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(Self::other_ir_node(other)), + metadata: self + .location + .new_node_metadata::<>::ElementType>(), + }, ) } } @@ -322,8 +361,11 @@ impl<'a, T, L: Location<'a> + NoTick, B> Singleton, B> { /// arbitrary point in time. pub unsafe fn latest_tick(self) -> Singleton, Bounded> { Singleton::new( - self.location.tick, - HydroNode::Unpersist(Box::new(self.ir_node.into_inner())), + self.location.clone().tick, + HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -396,39 +438,54 @@ impl<'a, T, L: Location<'a>> Singleton, Bounded> { pub fn all_ticks(self) -> Stream, Unbounded> { Stream::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn latest(self) -> Singleton, Unbounded> { Singleton::new( Timestamped { - tick: self.location, + tick: self.location.clone(), + }, + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), ) } pub fn defer_tick(self) -> Singleton, Bounded> { Singleton::new( - self.location, - HydroNode::DeferTick(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::DeferTick { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn persist(self) -> Stream, Bounded> { Stream::new( - self.location, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn delta(self) -> Optional, Bounded> { Optional::new( - self.location, - HydroNode::Delta(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Delta { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -439,6 +496,7 @@ impl<'a, T, L: Location<'a>> Singleton, Bounded> { pub trait ZipResult<'a, Other> { type Out; + type ElementType; type Location; fn other_location(other: &Other) -> Self::Location; @@ -451,6 +509,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Singleton, B> { type Out = Singleton<(T, U), Timestamped, B>; + type ElementType = (T, U); type Location = Timestamped; fn other_location(other: &Singleton, B>) -> Timestamped { @@ -470,6 +529,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Optional, B> { type Out = Optional<(T, U), Timestamped, B>; + type ElementType = (T, U); type Location = Timestamped; fn other_location(other: &Optional, B>) -> Timestamped { @@ -489,6 +549,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Singleton, B> for Singleton, B> { type Out = Singleton<(T, U), Tick, B>; + type ElementType = (T, U); type Location = Tick; fn other_location(other: &Singleton, B>) -> Tick { @@ -508,6 +569,7 @@ impl<'a, T, U: Clone, L: Location<'a>, B> ZipResult<'a, Optional, B>> for Singleton, B> { type Out = Optional<(T, U), Tick, B>; + type ElementType = (T, U); type Location = Tick; fn other_location(other: &Optional, B>) -> Tick { diff --git a/hydro_lang/src/stream.rs b/hydro_lang/src/stream.rs index 6ea29059329..0d902f3cf66 100644 --- a/hydro_lang/src/stream.rs +++ b/hydro_lang/src/stream.rs @@ -113,10 +113,11 @@ impl<'a, T, L: Location<'a>, Order> CycleCollection<'a, TickCycleMarker> fn create_source(ident: syn::Ident, location: Tick) -> Self { let location_id = location.id(); Stream::new( - location, + location.clone(), HydroNode::CycleSource { ident, location_kind: location_id, + metadata: location.new_node_metadata::(), }, ) } @@ -152,12 +153,17 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> CycleCollection<'a, ForwardRefMa fn create_source(ident: syn::Ident, location: L) -> Self { let location_id = location.id(); + Stream::new( - location, - HydroNode::Persist(Box::new(HydroNode::CycleSource { - ident, - location_kind: location_id, - })), + location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::CycleSource { + ident, + location_kind: location_id, + metadata: location.new_node_metadata::(), + }), + metadata: location.new_node_metadata::(), + }, ) } } @@ -171,6 +177,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> CycleComplete<'a, ForwardRefMark expected_location, "locations do not match" ); + let metadata = self.location.new_node_metadata::(); self.location .flow_state() .borrow_mut() @@ -180,7 +187,10 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> CycleComplete<'a, ForwardRefMark .push(HydroLeaf::CycleSink { ident, location_kind: self.location_kind(), - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata, + }), }); } } @@ -201,14 +211,16 @@ impl<'a, T: Clone, L: Location<'a>, B, Order> Clone for Stream { let orig_ir_node = self.ir_node.replace(HydroNode::Placeholder); *self.ir_node.borrow_mut() = HydroNode::Tee { inner: TeeNode(Rc::new(RefCell::new(orig_ir_node))), + metadata: self.location.new_node_metadata::(), }; } - if let HydroNode::Tee { inner } = self.ir_node.borrow().deref() { + if let HydroNode::Tee { inner, metadata } = self.ir_node.borrow().deref() { Stream { location: self.location.clone(), ir_node: HydroNode::Tee { inner: TeeNode(inner.0.clone()), + metadata: metadata.clone(), } .into(), _phantom: PhantomData, @@ -244,10 +256,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::Map { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -280,10 +293,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -318,10 +332,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FlatMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -411,10 +426,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_borrow_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::Filter { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -441,10 +457,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { ) -> Stream { let f = f.splice_fn1_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FilterMap { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -483,11 +500,12 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::CrossSingleton( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::CrossSingleton { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } @@ -512,11 +530,12 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::CrossProduct( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::CrossProduct { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(T, O)>(), + }, ) } @@ -527,8 +546,11 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { T: Eq + Hash, { Stream::new( - self.location, - HydroNode::Unique(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Unique { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -543,11 +565,12 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::Difference( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Difference { + pos: Box::new(self.ir_node.into_inner()), + neg: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -562,18 +585,26 @@ impl<'a, T, L: Location<'a>, B, Order> Stream { if L::is_top_level() { Stream::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Inspect { - f, - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - })), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Inspect { + f, + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::(), + }, ) } else { Stream::new( - self.location, + self.location.clone(), HydroNode::Inspect { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }, ) } @@ -655,13 +686,17 @@ where init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { // top-level (possibly unbounded) singletons are represented as // a stream which produces all values from all ticks every tick, // so Unpersist will always give the lastest aggregation - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Singleton::new(self.location, core) @@ -699,10 +734,14 @@ where let mut core = HydroNode::Reduce { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Optional::new(self.location, core) @@ -772,10 +811,14 @@ where let mut core = HydroNode::Reduce { f: wrapped.into(), input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Optional::new(self.location, core) @@ -834,18 +877,26 @@ impl<'a, T, L: Location<'a>, B> Stream { pub fn enumerate(self) -> Stream<(usize, T), L, B, TotalOrder> { if L::is_top_level() { Stream::new( - self.location, - HydroNode::Persist(Box::new(HydroNode::Enumerate { - is_static: true, - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), - })), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(HydroNode::Enumerate { + is_static: true, + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }), + metadata: self.location.new_node_metadata::<(usize, T)>(), + }), + metadata: self.location.new_node_metadata::<(usize, T)>(), + }, ) } else { Stream::new( - self.location, + self.location.clone(), HydroNode::Enumerate { is_static: false, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(usize, T)>(), }, ) } @@ -935,13 +986,17 @@ impl<'a, T, L: Location<'a>, B> Stream { init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { // top-level (possibly unbounded) singletons are represented as // a stream which produces all values from all ticks every tick, // so Unpersist will always give the lastest aggregation - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Singleton::new(self.location, core) @@ -980,10 +1035,14 @@ impl<'a, T, L: Location<'a>, B> Stream { let mut core = HydroNode::Reduce { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), }; if L::is_top_level() { - core = HydroNode::Persist(Box::new(core)); + core = HydroNode::Persist { + inner: Box::new(core), + metadata: self.location.new_node_metadata::(), + }; } Optional::new(self.location, core) @@ -1061,8 +1120,11 @@ impl<'a, T, L: Location<'a>, Order> Stream { T: Ord, { Stream::new( - self.location, - HydroNode::Sort(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Sort { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -1103,11 +1165,12 @@ impl<'a, T, L: Location<'a>, Order> Stream { check_matching_location(&self.location, &other.location); Stream::new( - self.location, - HydroNode::Chain( - Box::new(self.ir_node.into_inner()), - Box::new(other.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Chain { + first: Box::new(self.ir_node.into_inner()), + second: Box::new(other.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -1122,11 +1185,12 @@ impl<'a, K, V1, L: Location<'a>, B, Order> Stream<(K, V1), L, B, Order> { check_matching_location(&self.location, &n.location); Stream::new( - self.location, - HydroNode::Join( - Box::new(self.ir_node.into_inner()), - Box::new(n.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::Join { + left: Box::new(self.ir_node.into_inner()), + right: Box::new(n.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, (V1, V2))>(), + }, ) } @@ -1141,11 +1205,12 @@ impl<'a, K, V1, L: Location<'a>, B, Order> Stream<(K, V1), L, B, Order> { check_matching_location(&self.location, &n.location); Stream::new( - self.location, - HydroNode::AntiJoin( - Box::new(self.ir_node.into_inner()), - Box::new(n.ir_node.into_inner()), - ), + self.location.clone(), + HydroNode::AntiJoin { + pos: Box::new(self.ir_node.into_inner()), + neg: Box::new(n.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, V1)>(), + }, ) } } @@ -1188,11 +1253,12 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>> Stream<(K, V), Tick, Bounded> { let comb = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FoldKeyed { init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, A)>(), }, ) } @@ -1231,10 +1297,11 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>> Stream<(K, V), Tick, Bounded> { let f = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::ReduceKeyed { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, V)>(), }, ) } @@ -1277,11 +1344,12 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>, Order> Stream<(K, V), Tick, Bounde let comb = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::FoldKeyed { init, acc: comb, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, A)>(), }, ) } @@ -1325,10 +1393,11 @@ impl<'a, K: Eq + Hash, V, L: Location<'a>, Order> Stream<(K, V), Tick, Bounde let f = comb.splice_fn2_borrow_mut_ctx(&self.location).into(); Stream::new( - self.location, + self.location.clone(), HydroNode::ReduceKeyed { f, input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::<(K, V)>(), }, ) } @@ -1343,8 +1412,11 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream, B, Ord /// The batch boundaries are non-deterministic and may change across executions. pub unsafe fn tick_batch(self) -> Stream, Bounded, Order> { Stream::new( - self.location.tick, - HydroNode::Unpersist(Box::new(self.ir_node.into_inner())), + self.location.clone().tick, + HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -1442,6 +1514,7 @@ impl<'a, T, L: Location<'a> + NoTick + NoTimestamp, B, Order> Stream + NoTick, B, Order> Stream { pub fn for_each(self, f: impl IntoQuotedMut<'a, F, L>) { let f = f.splice_fn1_ctx(&self.location).into(); + let metadata = self.location.new_node_metadata::(); self.location .flow_state() .borrow_mut() @@ -1449,7 +1522,10 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { .as_mut() .expect(FLOW_USED_MESSAGE) .push(HydroLeaf::ForEach { - input: Box::new(HydroNode::Unpersist(Box::new(self.ir_node.into_inner()))), + input: Box::new(HydroNode::Unpersist { + inner: Box::new(self.ir_node.into_inner()), + metadata, + }), f, }); } @@ -1477,7 +1553,10 @@ impl<'a, T, L: Location<'a>, Order> Stream, Bounded, Order> { Timestamped { tick: self.location.clone(), }, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } @@ -1486,22 +1565,31 @@ impl<'a, T, L: Location<'a>, Order> Stream, Bounded, Order> { T: Clone, { Stream::new( - self.location, - HydroNode::Persist(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Persist { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn defer_tick(self) -> Stream, Bounded, Order> { Stream::new( - self.location, - HydroNode::DeferTick(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::DeferTick { + input: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } pub fn delta(self) -> Stream, Bounded, Order> { Stream::new( - self.location, - HydroNode::Delta(Box::new(self.ir_node.into_inner())), + self.location.clone(), + HydroNode::Delta { + inner: Box::new(self.ir_node.into_inner()), + metadata: self.location.new_node_metadata::(), + }, ) } } @@ -1572,6 +1660,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { instantiate_fn: DebugInstantiate::Building(), deserialize_fn: deserialize_pipeline.map(|e| e.into()), input: Box::new(self.ir_node.into_inner()), + metadata: other.new_node_metadata::(), }, ) } @@ -1587,6 +1676,8 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { { let serialize_pipeline = Some(serialize_bincode::(L::is_demux())); + let metadata = other.new_node_metadata::(); + let mut flow_state_borrow = self.location.flow_state().borrow_mut(); let external_key = flow_state_borrow.next_external_out; @@ -1607,6 +1698,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { instantiate_fn: DebugInstantiate::Building(), deserialize_fn: None, input: Box::new(self.ir_node.into_inner()), + metadata, }), }); @@ -1643,6 +1735,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { Some(expr.into()) }, input: Box::new(self.ir_node.into_inner()), + metadata: other.new_node_metadata::(), }, ) } @@ -1651,6 +1744,8 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { where L::Root: CanSend<'a, ExternalProcess<'a, L2>, In = T, Out = Bytes>, { + let metadata = other.new_node_metadata::(); + let mut flow_state_borrow = self.location.flow_state().borrow_mut(); let external_key = flow_state_borrow.next_external_out; flow_state_borrow.next_external_out += 1; @@ -1670,6 +1765,7 @@ impl<'a, T, L: Location<'a> + NoTick, B, Order> Stream { instantiate_fn: DebugInstantiate::Building(), deserialize_fn: None, input: Box::new(self.ir_node.into_inner()), + metadata, }), }); diff --git a/hydro_std/Cargo.toml b/hydro_std/Cargo.toml index fdb4eaf80ff..8436273ea9c 100644 --- a/hydro_std/Cargo.toml +++ b/hydro_std/Cargo.toml @@ -15,14 +15,14 @@ workspace = true path = "src/lib.rs" [dependencies] -hydro_lang = { path = "../hydro_lang", version = "^0.11.0", default-features = false } +hydro_lang = { path = "../hydro_lang", version = "^0.11.0" } stageleft = { path = "../stageleft", version = "^0.6.0" } [build-dependencies] stageleft_tool = { path = "../stageleft_tool", version = "^0.5.0" } [dev-dependencies] -hydro_lang = { path = "../hydro_lang", version = "^0.11.0" } +hydro_lang = { path = "../hydro_lang", version = "^0.11.0", features = ["deploy"] } insta = "1.39" hydro_deploy = { path = "../hydro_deploy/core", version = "^0.11.0" } async-ssh2-lite = { version = "0.5.0", features = ["vendored-openssl"] } diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap index e284ea81961..9e805cb763b 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__compute_pi__tests__compute_pi_ir.snap @@ -7,11 +7,11 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (u64 , u64) , () > ({ use crate :: __staged :: cluster :: compute_pi :: * ; | (inside , total) | { println ! ("pi: {} ({} trials)" , 4.0 * inside as f64 / total as f64 , total) ; } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((u64 , u64) , ()) , (u64 , u64) > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Reduce { + input: CrossSingleton { + left: Reduce { 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 { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: compute_pi :: Worker > , (u64 , u64)) , (u64 , u64) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -45,17 +45,97 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (f64 , f64), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + bool, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (u64 , u64), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Source { source: Stream( @@ -64,9 +144,41 @@ expression: built.ir() location_kind: Process( 1, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + ((u64 , u64) , ()), + ), }, - ), + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + (u64 , u64), + ), + }, }, }, ] diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap index 74c0e31cdf6..36d54d0931e 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__many_to_many__tests__many_to_many.snap @@ -30,7 +30,31 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + i32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + i32, + ), }, }, }, diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap index 64648d904d4..22ed07d5321 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__map_reduce__tests__map_reduce_ir.snap @@ -7,8 +7,8 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: string :: String , i32) , () > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | (string , count) | println ! ("{}: {}" , string , count) }), input: ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < i32 , i32 , () > ({ use crate :: __staged :: cluster :: map_reduce :: * ; | total , count | * total += count }), - input: Persist( - Map { + input: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , (std :: string :: String , i32)) , (std :: string :: String , i32) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -62,17 +62,113 @@ expression: built.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + & str, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + std :: string :: String, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (usize , std :: string :: String), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: map_reduce :: Worker > , std :: string :: String), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + std :: string :: String, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (std :: string :: String , ()), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), }, - ), + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (std :: string :: String , i32), + ), + }, }, }, ] diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap index eb4f676ebc9..c3b6cbc7bbf 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__paxos_bench__tests__paxos_ir.snap @@ -12,6 +12,14 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + & str, + ), + }, }, }, ForEach { @@ -23,6 +31,14 @@ expression: built.ir() location_kind: Cluster( 1, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + & str, + ), + }, }, }, CycleSink { @@ -35,60 +51,148 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , u32) , u32 > ({ use crate :: __staged :: cluster :: paxos :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (__hydro_lang_cluster_self_id_0) ; move | (received_max_ballot , ballot_num) | { if received_max_ballot > (Ballot { num : ballot_num , proposer_id : CLUSTER_SELF_ID__free , }) { received_max_ballot . num + 1 } else { ballot_num } } }), - input: CrossSingleton( - Tee { - inner: : Chain( - Reduce { + input: CrossSingleton { + left: Tee { + inner: : Chain { + first: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Chain( - Chain( - CycleSource { + input: Persist { + inner: Chain { + first: Chain { + first: CycleSource { ident: Ident { sym: cycle_1, }, location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - CycleSource { + second: CycleSource { ident: Ident { sym: cycle_0, }, location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, - ), - CycleSource { + }, + second: CycleSource { ident: Ident { sym: cycle_2, }, location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, ), - ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e__free] }, ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, ), - ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Tee { - inner: : Chain( - CycleSource { + right: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_4, }, @@ -98,22 +202,86 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e__free] }, ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , u32), ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, ), }, - ), + }, }, CycleSink { ident: Ident { @@ -145,19 +313,43 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >] > (__hydro_lang_cluster_ids_0) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < u32 , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: paxos :: Proposer > :: from_raw (__hydro_lang_cluster_self_id_0) ; move | num | Ballot { num , proposer_id : CLUSTER_SELF_ID__free } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + u32, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { inner: : CycleSource { @@ -170,12 +362,52 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Source { source: Stream( @@ -184,13 +416,77 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, }, CycleSink { @@ -203,17 +499,17 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Difference( - FilterMap { + input: DeferTick { + input: Difference { + pos: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)) , core :: option :: Option < hydro_test :: cluster :: paxos :: Ballot > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { inner: : FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , usize) > ({ use hydro_std :: __staged :: quorum :: * ; move | | (0 , 0) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { - inner: : Chain( - CycleSource { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_5, }, @@ -223,8 +519,16 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + second: Tee { inner: : Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1b | println ! ("Proposer received P1b: {:?}" , p1b) }), input: Map { @@ -247,9 +551,9 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((ballot , max_ballot) , log) | (ballot . proposer_id , (ballot , if ballot == max_ballot { Ok (log) } else { Err (max_ballot) })) }), - input: CrossSingleton( - CrossSingleton( - Tee { + input: CrossSingleton { + left: CrossSingleton { + left: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -274,31 +578,71 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | println ! ("Proposer leader expired, sending P1a") }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - FilterMap { + input: CrossSingleton { + left: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , core :: option :: Option < () > > ({ use hydro_lang :: __staged :: stream :: * ; let duration__free = { use crate :: __staged :: cluster :: paxos :: * ; let i_am_leader_check_timeout__free = 1u64 ; Duration :: from_secs (i_am_leader_check_timeout__free) } ; move | latest_received | { if let Some (latest_received) = latest_received { if Instant :: now () . duration_since (latest_received) > duration__free { Some (()) } else { None } } else { Some (()) } } }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < tokio :: time :: Instant > > ({ use hydro_lang :: __staged :: stream :: * ; | | None }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < tokio :: time :: Instant > , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | latest , _ | { * latest = Some (Instant :: now ()) ; } }), - input: Persist( - Tee { + input: Persist { + inner: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < tokio :: time :: Instant >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: optional :: * ; | c | * c == 0 }), @@ -307,13 +651,61 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Source { source: Stream( @@ -322,45 +714,219 @@ expression: built.ir() location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - Tee { - inner: : Chain( - Reduce { + right: Tee { + inner: : Chain { + first: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Inspect { + input: Persist { + inner: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | p1a | println ! ("Acceptor received P1a: {:?}" , p1a) }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; Ballot { num : 0 , proposer_id : ClusterId :: from_raw (0) } } ; [e__free] }, ), location_kind: Cluster( 1, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), ), }, - ), - CycleSource { + }, + right: CycleSource { ident: Ident { sym: cycle_0, }, @@ -370,28 +936,167 @@ expression: built.ir() 1, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + ((hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - Tee { + neg: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)) , core :: option :: Option < hydro_test :: cluster :: paxos :: Ballot > > ({ use hydro_std :: __staged :: quorum :: * ; let max__free = 3usize ; move | (key , (success , error)) | if (success + error) >= max__free { Some (key) } else { None } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, }, CycleSink { ident: Ident { @@ -403,16 +1108,48 @@ expression: built.ir() 0, ), ), - input: DeferTick( - AntiJoin( - Tee { + input: DeferTick { + input: AntiJoin { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + neg: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -427,34 +1164,66 @@ expression: built.ir() input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), input: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; move | ((quorum_ballot , quorum_accepted) , my_ballot) | if quorum_ballot == my_ballot { Some (quorum_accepted) } else { None } }), - input: CrossSingleton( - Reduce { + input: CrossSingleton { + left: Reduce { f: { let key_fn = stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot > ({ use crate :: __staged :: cluster :: paxos :: * ; | t | t . 0 }) ; move | curr , new | { if key_fn (& new) > key_fn (& * curr) { * curr = new ; } } }, input: FoldKeyed { init: stageleft :: runtime_support :: fn0_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > ({ use crate :: __staged :: cluster :: paxos :: * ; | | vec ! [] }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | logs , log | { logs . push (log) ; } }), - input: Persist( - FilterMap { + input: Persist { + inner: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (v) => Some ((key , v)) , Err (_) => None , } }), - input: AntiJoin( - AntiJoin( - Tee { + input: AntiJoin { + pos: AntiJoin { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - FilterMap { + neg: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)) , core :: option :: Option < hydro_test :: cluster :: paxos :: Ballot > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success < min__free { Some (key) } else { None } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, - ), - CycleSource { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + neg: CycleSource { ident: Ident { sym: cycle_6, }, @@ -464,38 +1233,206 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >), ), }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((hydro_test :: cluster :: paxos :: Ballot , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >) , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | _ | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) , bool > ({ use crate :: __staged :: cluster :: paxos :: * ; | (received_max_ballot , cur_ballot) | * received_max_ballot <= * cur_ballot }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, @@ -513,8 +1450,41 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , core :: result :: Result < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, }, CycleSink { @@ -527,13 +1497,13 @@ expression: built.ir() 2, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { - inner: : Chain( - CycleSource { + input: CrossSingleton { + left: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_1, }, @@ -543,8 +1513,16 @@ expression: built.ir() 2, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: paxos_bench :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; move | (key , value) | KvPayload { key , value : (CLUSTER_SELF_ID__free , value) } }), input: CycleSource { ident: Ident { @@ -553,11 +1531,43 @@ expression: built.ir() location_kind: Cluster( 2, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: stream :: * ; | c | * c == 0 }), @@ -569,8 +1579,8 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; | ballot : Ballot | ballot . proposer_id }), input: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , hydro_test :: cluster :: paxos :: Ballot , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - Inspect { + input: Persist { + inner: Inspect { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , () > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; | ballot | println ! ("Client notified that leader was elected: {:?}" , ballot) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), @@ -594,54 +1604,270 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: Ballot , std :: iter :: Map < std :: slice :: Iter < hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > > , _ > > ({ use hydro_lang :: __staged :: stream :: * ; let ids__free = unsafe { :: std :: mem :: transmute :: < _ , & [hydro_lang :: ClusterId < hydro_test :: cluster :: bench_client :: Client >] > (__hydro_lang_cluster_ids_2) } ; | b | ids__free . iter () . map (move | id | (:: std :: clone :: Clone :: clone (id) , :: std :: clone :: Clone :: clone (& b))) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: Ballot , ()) , hydro_test :: cluster :: paxos :: Ballot > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (() , ()) , () > ({ use hydro_lang :: __staged :: optional :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: optional :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: optional :: * ; | c | * c == 0 }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , () , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), - input: DeferTick( - Tee { + input: DeferTick { + input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (() , ()), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: Ballot , ()), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , hydro_test :: cluster :: paxos :: Ballot), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, - ), + }, }, CycleSink { ident: Ident { @@ -653,24 +1879,24 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (num_payloads , base_slot) | base_slot + num_payloads }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , usize) , (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((index , payload) , base_slot) | (base_slot + index , payload) }), - input: CrossSingleton( - Enumerate { + input: CrossSingleton { + left: Enumerate { is_static: false, input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Map { + input: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { from_location: Cluster( @@ -690,29 +1916,120 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) > ({ use crate :: __staged :: cluster :: paxos_with_client :: * ; move | (payload , leader_id) | (leader_id , payload) }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer >), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 7, + Cluster( + 2, + ), + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), ), }, }, - Tee { - inner: : Chain( - Map { + right: Tee { + inner: : Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | max_slot | max_slot + 1 }), input: Tee { inner: : Reduce { @@ -734,20 +2051,116 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > , std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > > ({ use hydro_lang :: __staged :: optional :: * ; | v | v }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: vec :: Vec < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) >, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , core :: option :: Option < hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Chain( - CycleSource { + second: Chain { + first: CycleSource { ident: Ident { sym: cycle_7, }, @@ -757,31 +2170,151 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( { use hydro_lang :: __staged :: location :: * ; let e__free = { use crate :: __staged :: cluster :: paxos :: * ; 0 } ; [e__free] }, ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , usize), ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , usize), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, ), }, - ), + }, }, CycleSink { ident: Ident { @@ -793,9 +2326,9 @@ expression: built.ir() 0, ), ), - input: DeferTick( - Difference( - Tee { + input: DeferTick { + input: Difference { + pos: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)) , core :: option :: Option < (usize , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { @@ -803,8 +2336,8 @@ expression: built.ir() init: stageleft :: runtime_support :: fn0_type_hint :: < (usize , usize) > ({ use hydro_std :: __staged :: quorum :: * ; move | | (0 , 0) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { - inner: : Chain( - CycleSource { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_8, }, @@ -814,8 +2347,16 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + second: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -836,8 +2377,8 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)) > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | (p2a . ballot . proposer_id , ((p2a . slot , p2a . ballot) , if p2a . ballot == max_ballot { Ok (()) } else { Err (max_ballot) })) }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -863,36 +2404,92 @@ expression: built.ir() input: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , ()) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Chain( - Map { + input: CrossSingleton { + left: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , payload) , ballot) | ((slot , ballot) , Some (payload)) }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >) , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | p2a | ((p2a . slot , p2a . ballot) , p2a . value) }), - input: Chain( - FilterMap { + input: Chain { + first: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < usize >) , core :: option :: Option < hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | (((slot , (count , entry)) , ballot) , checkpoint) | { if count > f__free { return None ; } else if let Some (checkpoint) = checkpoint { if slot <= checkpoint { return None ; } } Some (P2a { ballot , slot , value : entry . value , }) } }), - input: CrossSingleton( - CrossSingleton( - Tee { + input: CrossSingleton { + left: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), - Tee { - inner: : Chain( - Map { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + right: Tee { + inner: : Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < usize > > ({ use hydro_lang :: __staged :: optional :: * ; | v | Some (v) }), input: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), @@ -900,92 +2497,504 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (checkpoint , _log) | checkpoint }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( [:: std :: option :: Option :: None], ), location_kind: Cluster( 0, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (((usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < usize >), ), }, - ), + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , ballot) | P2a { ballot , slot , value : None } }), - input: CrossSingleton( - Difference( - FlatMap { + input: CrossSingleton { + left: Difference { + pos: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < usize >) , std :: ops :: Range < usize > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (max_slot , checkpoint) | { if let Some (checkpoint) = checkpoint { (checkpoint + 1) .. max_slot } else { 0 .. max_slot } } }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , core :: option :: Option < usize >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, }, - Map { + neg: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (slot , _) | slot }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , (usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), }, }, - ), - Tee { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + usize, + ), + }, + }, + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), ), }, - ), - Map { + }, + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - Tee { + neg: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)) , core :: option :: Option < (usize , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; let max__free = 3usize ; move | (key , (success , error)) | if (success + error) >= max__free { Some (key) } else { None } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, }, CycleSink { ident: Ident { @@ -997,16 +3006,48 @@ expression: built.ir() 0, ), ), - input: DeferTick( - AntiJoin( - Tee { + input: DeferTick { + input: AntiJoin { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, }, - Tee { + neg: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1018,11 +3059,11 @@ expression: built.ir() 0, ), ), - input: DeferTick( - AntiJoin( - Tee { - inner: : Chain( - CycleSource { + input: DeferTick { + input: AntiJoin { + pos: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_10, }, @@ -1032,22 +3073,65 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - Tee { + second: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - ), - }, - Map { - f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) , (usize , hydro_test :: cluster :: paxos :: Ballot) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , _) | key }), - input: Tee { - inner: : Map { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + neg: Map { + f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) , (usize , hydro_test :: cluster :: paxos :: Ballot) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , _) | key }), + input: Tee { + inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , hydro_test :: cluster :: paxos :: Ballot) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()) > ({ use crate :: __staged :: cluster :: paxos :: * ; | k | (k , ()) }), - input: Difference( - Tee { + input: Difference { + pos: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - CycleSource { + neg: CycleSource { ident: Ident { sym: cycle_9, }, @@ -1057,13 +3141,72 @@ expression: built.ir() 0, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (usize , hydro_test :: cluster :: paxos :: Ballot), + ), + }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1078,34 +3221,66 @@ expression: built.ir() input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | | (None , HashMap :: new ()) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >) , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | (prev_checkpoint , log) , checkpoint_or_p2a | { match checkpoint_or_p2a { CheckpointOrP2a :: Checkpoint (new_checkpoint) => { if prev_checkpoint . map (| prev | new_checkpoint > prev) . unwrap_or (true) { for slot in (prev_checkpoint . unwrap_or (0)) .. new_checkpoint { log . remove (& slot) ; } * prev_checkpoint = Some (new_checkpoint) ; } } CheckpointOrP2a :: P2a (p2a) => { if prev_checkpoint . map (| prev | p2a . slot > prev) . unwrap_or (true) && log . get (& p2a . slot) . map (| prev_p2a : & LogValue < _ > | p2a . ballot > prev_p2a . ballot) . unwrap_or (true) { log . insert (p2a . slot , LogValue { ballot : p2a . ballot , value : p2a . value , } ,) ; } } } } }), - input: Persist( - Chain( - FilterMap { + input: Persist { + inner: Chain { + first: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | (p2a , max_ballot) | if p2a . ballot >= max_ballot { Some (CheckpointOrP2a :: P2a (p2a)) } else { None } }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_test :: cluster :: paxos :: P2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , hydro_test :: cluster :: paxos :: Ballot), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: paxos :: * ; | min_seq | CheckpointOrP2a :: Checkpoint (min_seq) }), - input: Delta( - Reduce { + input: Delta { + inner: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new < * curr { * curr = new ; } } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , usize > ({ use crate :: __staged :: cluster :: paxos :: * ; | (_sender , seq) | seq }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , ()) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) > ({ use hydro_lang :: __staged :: stream :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: : ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use crate :: __staged :: cluster :: paxos :: * ; | curr_seq , seq | { if seq > * curr_seq { * curr_seq = seq ; } } }), - input: Persist( - Network { + input: Persist { + inner: Network { from_location: Cluster( 3, ), @@ -1130,13 +3305,61 @@ expression: built.ir() location_kind: Cluster( 3, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Acceptor > , usize), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < bool , () > ({ use hydro_lang :: __staged :: stream :: * ; | _u | () }), input: FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < bool > > ({ use crate :: __staged :: cluster :: paxos :: * ; let f__free = 1usize ; move | num_received | if num_received == f__free + 1 { Some (true) } else { None } }), @@ -1145,18 +3368,125 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + bool, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + ((hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize) , ()), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + hydro_test :: cluster :: paxos :: CheckpointOrP2a < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 2, + Cluster( + 1, + ), ), - ), + output_type: Some( + (core :: option :: Option < usize > , std :: collections :: hash_map :: HashMap < usize , hydro_test :: cluster :: paxos :: LogValue < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > >), + ), + }, }, }, CycleSink { @@ -1172,8 +3502,41 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >) , core :: option :: Option < ((usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot) > > ({ use hydro_std :: __staged :: quorum :: * ; move | (key , res) | match res { Ok (_) => None , Err (e) => Some ((key , e)) , } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: result :: Result < () , hydro_test :: cluster :: paxos :: Ballot >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , hydro_test :: cluster :: paxos :: Ballot), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + hydro_test :: cluster :: paxos :: Ballot, + ), + }, }, }, CycleSink { @@ -1186,16 +3549,16 @@ expression: built.ir() 3, ), ), - input: DeferTick( - Map { + input: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq > * highest_seq }), - input: CrossSingleton( - Tee { - inner: : Sort( - Chain( - Map { + input: CrossSingleton { + left: Tee { + inner: : Sort { + input: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (slot , kv) | SequencedKv { seq : slot , kv } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: paxos :: Proposer > , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)) , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), @@ -1221,21 +3584,96 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >) > ({ use crate :: __staged :: cluster :: paxos :: * ; | ((slot , _ballot) , (value , _)) | (slot , value) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) , ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())) > ({ use hydro_std :: __staged :: request_response :: * ; | (key , (meta , resp)) | (key , (meta , resp)) }), - input: Join( - Tee { + input: Join { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + ((usize , hydro_test :: cluster :: paxos :: Ballot) , (core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > , ())), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 1, + Cluster( + 0, + ), + ), + output_type: Some( + (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (usize , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, }, - CycleSource { + second: CycleSource { ident: Ident { sym: cycle_1, }, @@ -1245,22 +3683,62 @@ expression: built.ir() 3, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, ), - ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Tee { + right: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < usize > , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | v | v }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | | None }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < core :: option :: Option < usize > , (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , () > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | filled_slot , (sorted_payload , highest_seq) | { let expected_next_slot = std :: cmp :: max (filled_slot . map (| v | v + 1) . unwrap_or (0) , highest_seq . map (| v | v + 1) . unwrap_or (0) ,) ; if sorted_payload . seq == expected_next_slot { * filled_slot = Some (sorted_payload . seq) ; } } }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Chain( - Map { + right: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < usize > > ({ use hydro_lang :: __staged :: optional :: * ; | v | Some (v) }), input: CycleSource { ident: Ident { @@ -1272,27 +3750,131 @@ expression: built.ir() 3, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( [:: std :: option :: Option :: None], ), location_kind: Cluster( 3, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, - ), - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, }, CycleSink { ident: Ident { @@ -1304,35 +3886,123 @@ expression: built.ir() 3, ), ), - input: DeferTick( - Tee { + input: DeferTick { + input: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (_kv_store , highest_seq) | highest_seq }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | | (HashMap :: new () , None) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , () > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (kv_store , last_seq) , payload | { if let Some (kv) = payload . kv { kv_store . insert (kv . key , kv . value) ; } debug_assert ! (payload . seq == (last_seq . map (| s | s + 1) . unwrap_or (0)) , "Hole in log between seq {:?} and {}" , * last_seq , payload . seq) ; * last_seq = Some (payload . seq) ; } }), - input: Persist( - Tee { + input: Persist { + inner: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , _) | { sorted_payload } }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize) , bool > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | (sorted_payload , highest_seq) | sorted_payload . seq <= * highest_seq }), - input: CrossSingleton( - Tee { + input: CrossSingleton { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (std :: collections :: hash_map :: HashMap < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < usize >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, }, CycleSink { ident: Ident { @@ -1344,18 +4014,18 @@ expression: built.ir() 3, ), ), - input: DeferTick( - Tee { + input: DeferTick { + input: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < (core :: option :: Option < usize > , usize) , core :: option :: Option < usize > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; let checkpoint_frequency__free = 1usize ; move | (max_checkpointed_seq , new_highest_seq) | if max_checkpointed_seq . map (| m | new_highest_seq - m >= checkpoint_frequency__free) . unwrap_or (true) { Some (new_highest_seq) } else { None } }), - input: CrossSingleton( - Chain( - Map { + input: CrossSingleton { + left: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , core :: option :: Option < usize > > ({ use hydro_lang :: __staged :: optional :: * ; | v | Some (v) }), input: Reduce { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , usize , () > ({ use hydro_lang :: __staged :: stream :: * ; | curr , new | { if new > * curr { * curr = new ; } } }), - input: Persist( - CycleSource { + input: Persist { + inner: CycleSource { ident: Ident { sym: cycle_3, }, @@ -1365,28 +4035,124 @@ expression: built.ir() 3, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, ), }, }, - Persist( - Source { + second: Persist { + inner: Source { source: Iter( [:: std :: option :: Option :: None], ), location_kind: Cluster( 3, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, }, - ), - ), - Tee { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + core :: option :: Option < usize >, + ), + }, + }, + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (core :: option :: Option < usize > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, + }, }, CycleSink { ident: Ident { @@ -1397,6 +4163,14 @@ expression: built.ir() ), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + usize, + ), + }, }, }, CycleSink { @@ -1409,11 +4183,11 @@ expression: built.ir() 2, ), ), - input: DeferTick( - AntiJoin( - Tee { - inner: : Chain( - CycleSource { + input: DeferTick { + input: AntiJoin { + pos: Tee { + inner: : Chain { + first: CycleSource { ident: Ident { sym: cycle_2, }, @@ -1423,8 +4197,16 @@ expression: built.ir() 2, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, }, - Tee { + second: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: kv_replica :: Replica > , ((u32 , u32) , core :: result :: Result < () , () >)) , ((u32 , u32) , core :: result :: Result < () , () >) > ({ use hydro_lang :: __staged :: stream :: * ; | (_ , b) | b }), input: Network { @@ -1449,15 +4231,82 @@ expression: built.ir() f: stageleft :: runtime_support :: fn1_type_hint :: < hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > , core :: option :: Option < hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) > > > ({ use crate :: __staged :: cluster :: kv_replica :: * ; | payload | payload . kv }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: SequencedKv < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + hydro_test :: cluster :: kv_replica :: KvPayload < u32 , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , u32) >, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 3, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < hydro_test :: cluster :: bench_client :: Client > , ((u32 , u32) , core :: result :: Result < () , () >)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 9, + Cluster( + 2, + ), + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, }, - Tee { + neg: Tee { inner: : FilterMap { f: stageleft :: runtime_support :: fn1_type_hint :: < ((u32 , u32) , (usize , usize)) , core :: option :: Option < (u32 , u32) > > ({ use hydro_std :: __staged :: quorum :: * ; let min__free = 2usize ; move | (key , (success , _error)) | if success >= min__free { Some (key) } else { None } }), input: Tee { @@ -1466,13 +4315,69 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (usize , usize) , core :: result :: Result < () , () > , () > ({ use hydro_std :: __staged :: quorum :: * ; move | accum , value | { if value . is_ok () { accum . 0 += 1 ; } else { accum . 1 += 1 ; } } }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , (usize , usize)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , (usize , usize)), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), }, }, - ), - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((u32 , u32) , core :: result :: Result < () , () >), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1481,8 +4386,8 @@ expression: built.ir() location_kind: Cluster( 2, ), - input: Chain( - FlatMap { + input: Chain { + first: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < () , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: bench_client :: * ; let CLUSTER_SELF_ID__free = hydro_lang :: ClusterId :: < hydro_test :: cluster :: bench_client :: Client > :: from_raw (__hydro_lang_cluster_self_id_2) ; let num_clients_per_node__free = 1usize ; move | _ | (0 .. num_clients_per_node__free) . map (move | i | ((CLUSTER_SELF_ID__free . raw_id * (num_clients_per_node__free as u32)) + i as u32 , 0)) }), input: Tee { inner: : Source { @@ -1492,18 +4397,74 @@ expression: built.ir() location_kind: Cluster( 2, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, }, - Map { + second: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (u32 , u32) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | payload | (payload . 0 , payload . 1 + 1) }), input: Tee { inner: : Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, }, CycleSink { ident: Ident { @@ -1515,12 +4476,12 @@ expression: built.ir() 2, ), ), - input: DeferTick( - ReduceKeyed { + input: DeferTick { + input: ReduceKeyed { f: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < tokio :: time :: Instant , tokio :: time :: Instant , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; | curr_time , new_time | { if new_time > * curr_time { * curr_time = new_time ; } } }), - input: Chain( - Chain( - Tee { + input: Chain { + first: Chain { + first: Tee { inner: : CycleSource { ident: Ident { sym: cycle_3, @@ -1531,58 +4492,186 @@ expression: built.ir() 2, ), ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), }, }, - FlatMap { + second: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , std :: iter :: Map < std :: ops :: Range < usize > , _ > > ({ use crate :: __staged :: cluster :: bench_client :: * ; let num_clients_per_node__free = 1usize ; move | now | (0 .. num_clients_per_node__free) . map (move | virtual_id | (virtual_id , now)) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < () , tokio :: time :: Instant > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | Instant :: now () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, }, - ), - Tee { + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + second: Tee { inner: : Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (u32 , u32) , (usize , tokio :: time :: Instant) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (key , _prev_count) | (key as usize , Instant :: now ()) }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), ), }, - ), + }, }, ForEach { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; move | (latencies , throughput) | { let mut latencies_mut = latencies . borrow_mut () ; if latencies_mut . len () > 0 { let middle_idx = latencies_mut . len () / 2 ; let (_ , median , _) = latencies_mut . select_nth_unstable (middle_idx) ; println ! ("Median latency: {}ms" , median . as_micros () as f64 / 1000.0) ; } println ! ("Throughput: {} requests/s" , throughput) ; } }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < ((std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , ()) , (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - CrossSingleton( - Map { + input: CrossSingleton { + left: CrossSingleton { + left: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (latencies , _) | latencies }), input: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) > ({ use crate :: __staged :: cluster :: bench_client :: * ; let median_latency_window_size__free = 1usize ; move | | (Rc :: new (RefCell :: new (Vec :: < Duration > :: with_capacity (median_latency_window_size__free))) , 0usize ,) }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , core :: time :: Duration , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; let median_latency_window_size__free = 1usize ; move | (latencies , write_index) , latency | { let mut latencies_mut = latencies . borrow_mut () ; if * write_index < latencies_mut . len () { latencies_mut [* write_index] = latency ; } else { latencies_mut . push (latency) ; } * write_index = (* write_index + 1) % median_latency_window_size__free ; } }), - input: Persist( - FlatMap { + input: Persist { + inner: FlatMap { f: stageleft :: runtime_support :: fn1_type_hint :: < core :: option :: Option < core :: time :: Duration > , core :: option :: Option < core :: time :: Duration > > ({ use hydro_lang :: __staged :: stream :: * ; | d | d }), - input: Chain( - Map { + input: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , (tokio :: time :: Instant , tokio :: time :: Instant)) , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | (_virtual_id , (prev_time , curr_time)) | Some (curr_time . duration_since (prev_time)) }), - input: Join( - Tee { + input: Join { + left: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, }, - Tee { + right: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , tokio :: time :: Instant), + ), + }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , (tokio :: time :: Instant , tokio :: time :: Instant)), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), + }, }, - DeferTick( - Map { + second: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , core :: option :: Option < core :: time :: Duration > > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | None }), input: Tee { inner: : Source { @@ -1592,33 +4681,133 @@ expression: built.ir() location_kind: Cluster( 2, ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + core :: option :: Option < core :: time :: Duration >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + core :: time :: Duration, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, ), ), + output_type: Some( + core :: time :: Duration, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > >, ), }, }, - Fold { + right: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use crate :: __staged :: cluster :: bench_client :: * ; | | 0 }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (usize , bool) , () > ({ use crate :: __staged :: cluster :: bench_client :: * ; | total , (batch_size , reset) | { if reset { * total = 0 ; } else { * total += batch_size ; } } }), - input: Persist( - Chain( - Map { + input: Persist { + inner: Chain { + first: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , (usize , bool) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | batch_size | (batch_size , false) }), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (usize , ()) , usize > ({ use hydro_lang :: __staged :: singleton :: * ; | (d , _signal) | d }), - input: CrossSingleton( - Fold { + input: CrossSingleton { + left: Fold { init: stageleft :: runtime_support :: fn0_type_hint :: < usize > ({ use hydro_lang :: __staged :: stream :: * ; | | 0usize }), acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , (u32 , u32) , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (u32 , u32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), }, }, - Map { + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < usize , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Filter { f: stageleft :: runtime_support :: fn1_borrow_type_hint :: < usize , bool > ({ use hydro_lang :: __staged :: singleton :: * ; | c | * c == 0 }), @@ -1627,32 +4816,182 @@ expression: built.ir() acc: stageleft :: runtime_support :: fn2_borrow_mut_type_hint :: < usize , tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: stream :: * ; | count , _ | * count += 1 }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), ), }, }, - DeferTick( - Map { + second: DeferTick { + input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , (usize , bool) > ({ use crate :: __staged :: cluster :: bench_client :: * ; | _ | (0 , true) }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (usize , bool), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, ), ), + output_type: Some( + usize, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Tick( + 0, + Cluster( + 2, + ), + ), + output_type: Some( + (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize), ), }, - ), - Map { + }, + right: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < tokio :: time :: Instant , () > ({ use hydro_lang :: __staged :: singleton :: * ; | _u | () }), input: Tee { inner: , + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + tokio :: time :: Instant, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (), + ), }, }, - ), + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + ((std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize) , ()), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 2, + ), + output_type: Some( + (std :: rc :: Rc < core :: cell :: RefCell < std :: vec :: Vec < core :: time :: Duration > > > , usize), + ), + }, }, }, ] diff --git a/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap b/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap index 7c48e8e03c4..c27da6e316d 100644 --- a/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap +++ b/hydro_test/src/cluster/snapshots/hydro_test__cluster__simple_cluster__tests__simple_cluster.snap @@ -41,10 +41,10 @@ expression: built.ir() ), input: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32) , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32)) > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | (id , n) | (id , (id , n)) }), - input: Delta( - CrossProduct( - Persist( - Map { + input: Delta { + inner: CrossProduct { + left: Persist { + inner: Map { f: stageleft :: runtime_support :: fn1_type_hint :: < & hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > > ({ use crate :: __staged :: cluster :: simple_cluster :: * ; | & id | id }), input: Source { source: Iter( @@ -53,23 +53,111 @@ expression: built.ir() location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + & hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () >, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () >, + ), }, }, - ), - Persist( - Source { + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () >, + ), + }, + }, + right: Persist { + inner: Source { source: Iter( { use crate :: __staged :: cluster :: simple_cluster :: * ; 0 .. 5 }, ), location_kind: Process( 0, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + i32, + ), }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32)), ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), ), }, }, + metadata: HydroNodeMetadata { + location_kind: Cluster( + 1, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 0, + ), + output_type: Some( + (hydro_lang :: location :: cluster :: cluster_id :: ClusterId < () > , i32), + ), }, }, }, diff --git a/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap b/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap index 1bc38df8f75..0f408f4ec20 100644 --- a/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap +++ b/hydro_test/src/distributed/snapshots/hydro_test__distributed__first_ten__tests__first_ten_distributed.snap @@ -26,6 +26,22 @@ expression: built.ir() location_kind: ExternalProcess( 0, ), + metadata: HydroNodeMetadata { + location_kind: ExternalProcess( + 0, + ), + output_type: Some( + std :: string :: String, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + std :: string :: String, + ), }, }, }, @@ -56,8 +72,32 @@ expression: built.ir() location_kind: Process( 1, ), + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + u32, + ), + }, + }, + metadata: HydroNodeMetadata { + location_kind: Process( + 1, + ), + output_type: Some( + hydro_test :: distributed :: first_ten :: SendOverNetwork, + ), }, }, + metadata: HydroNodeMetadata { + location_kind: Process( + 2, + ), + output_type: Some( + hydro_test :: distributed :: first_ten :: SendOverNetwork, + ), + }, }, }, ]