From c9dc66db2bde34aa1385c8847d0ba7c16472f75d Mon Sep 17 00:00:00 2001 From: Shadaj Laddad Date: Tue, 23 Apr 2024 12:39:48 -0700 Subject: [PATCH] feat(hydroflow_datalog)!: add detupling syntax and allow interleaving with flattening (#1166) --- hydroflow/tests/datalog_frontend.rs | 107 ++++- hydroflow_datalog_core/src/grammar.rs | 21 +- hydroflow_datalog_core/src/join_plan.rs | 261 +++++++---- hydroflow_datalog_core/src/lib.rs | 111 +++-- ...ore__tests__anti_join@datalog_program.snap | 3 +- ...e__tests__collect_vec@datalog_program.snap | 3 +- ..._core__tests__detuple@datalog_program.snap | 341 ++++++++++++++ ...og_core__tests__detuple@surface_graph.snap | 18 + ...ts__detuple_then_flat@datalog_program.snap | 425 ++++++++++++++++++ ...ests__detuple_then_flat@surface_graph.snap | 22 + ...tests__expr_predicate@datalog_program.snap | 3 +- ...s__flat_then_detuple@datalog_program.snap} | 60 +-- ...sts__flat_then_detuple@surface_graph.snap} | 10 +- ..._core__tests__flatten@datalog_program.snap | 349 ++++++++++++++ ...og_core__tests__flatten@surface_graph.snap | 18 + ...ests__join_with_other@datalog_program.snap | 3 +- ...tests__join_with_self@datalog_program.snap | 3 +- ...__local_constraints@datalog_program-2.snap | 3 +- ...ts__local_constraints@datalog_program.snap | 3 +- ..._tests__multi_detuple@datalog_program.snap | 386 ++++++++++++++++ ...e__tests__multi_detuple@surface_graph.snap | 20 + ..._core__tests__persist@datalog_program.snap | 3 +- ..._tests__simple_filter@datalog_program.snap | 3 +- ...single_column_program@datalog_program.snap | 3 +- ...s__transitive_closure@datalog_program.snap | 3 +- ..._triple_relation_join@datalog_program.snap | 3 +- ...ests__wildcard_fields@datalog_program.snap | 3 +- ...__wildcard_join_count@datalog_program.snap | 3 +- 28 files changed, 1985 insertions(+), 206 deletions(-) create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@datalog_program.snap create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@surface_graph.snap create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@datalog_program.snap create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@surface_graph.snap rename hydroflow_datalog_core/src/snapshots/{hydroflow_datalog_core__tests__splat@datalog_program.snap => hydroflow_datalog_core__tests__flat_then_detuple@datalog_program.snap} (75%) rename hydroflow_datalog_core/src/snapshots/{hydroflow_datalog_core__tests__splat@surface_graph.snap => hydroflow_datalog_core__tests__flat_then_detuple@surface_graph.snap} (59%) create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@datalog_program.snap create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@surface_graph.snap create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@datalog_program.snap create mode 100644 hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@surface_graph.snap diff --git a/hydroflow/tests/datalog_frontend.rs b/hydroflow/tests/datalog_frontend.rs index dec618424be2..86bc845fe42e 100644 --- a/hydroflow/tests/datalog_frontend.rs +++ b/hydroflow/tests/datalog_frontend.rs @@ -1162,7 +1162,7 @@ fn test_collect_vec() { } #[multiplatform_test] -fn test_splat() { +fn test_flatten() { let (ints1_send, ints1) = hydroflow::util::unbounded_channel::<(i64, Vec)>(); let (result, mut result_recv) = hydroflow::util::unbounded_channel::<(i64, i64)>(); @@ -1172,7 +1172,7 @@ fn test_splat() { .output result `for_each(|v| result.send(v).unwrap())` - result(a, *b) :- ints1(a, b) + result(a, b) :- ints1(a, *b) "# ); @@ -1187,6 +1187,109 @@ fn test_splat() { ); } +#[test] +fn test_detuple() { + let (ints1_send, ints1) = hydroflow::util::unbounded_channel::<((i64, i64),)>(); + let (result, mut result_recv) = hydroflow::util::unbounded_channel::<(i64, i64)>(); + + let mut flow = datalog!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b) :- ints1((a, b)) + "# + ); + + ints1_send.send(((1, 2),)).unwrap(); + ints1_send.send(((3, 4),)).unwrap(); + + flow.run_tick(); + + assert_eq!( + &*collect_ready::, _>(&mut result_recv), + &[(1, 2), (3, 4)] + ); +} + +#[test] +fn test_multi_detuple() { + let (ints1_send, ints1) = hydroflow::util::unbounded_channel::<((i64, i64), (i64, i64))>(); + let (result, mut result_recv) = hydroflow::util::unbounded_channel::<(i64, i64, i64, i64)>(); + + let mut flow = datalog!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b, c, d) :- ints1((a, b), (c, d)) + "# + ); + + ints1_send.send(((1, 2), (3, 4))).unwrap(); + ints1_send.send(((5, 6), (7, 8))).unwrap(); + + flow.run_tick(); + + assert_eq!( + &*collect_ready::, _>(&mut result_recv), + &[(1, 2, 3, 4), (5, 6, 7, 8)] + ); +} + +#[test] +fn test_flat_then_detuple() { + let (ints1_send, ints1) = hydroflow::util::unbounded_channel::<(Vec<(i64, i64)>,)>(); + let (result, mut result_recv) = hydroflow::util::unbounded_channel::<(i64, i64)>(); + + let mut flow = datalog!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b) :- ints1(*(a, b)) + "# + ); + + ints1_send.send((vec![(1, 2), (3, 4)],)).unwrap(); + ints1_send.send((vec![(5, 6), (7, 8)],)).unwrap(); + + flow.run_tick(); + + assert_eq!( + &*collect_ready::, _>(&mut result_recv), + &[(1, 2), (3, 4), (5, 6), (7, 8)] + ); +} + +#[test] +fn test_detuple_then_flat() { + let (ints1_send, ints1) = hydroflow::util::unbounded_channel::<((Vec, Vec),)>(); + let (result, mut result_recv) = hydroflow::util::unbounded_channel::<(i64, i64)>(); + + let mut flow = datalog!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b) :- ints1((*a, *b)) + "# + ); + + ints1_send.send(((vec![1, 2], vec![3, 4]),)).unwrap(); + + flow.run_tick(); + + assert_eq!( + &*collect_ready::, _>(&mut result_recv), + &[(1, 3), (1, 4), (2, 3), (2, 4)] + ); +} + // #[ignore] doesn't seem to work for #[multiplatform_test] // #[ignore] // This test depends on the ordering of specific tuples which is undefined. // #[multiplatform_test] diff --git a/hydroflow_datalog_core/src/grammar.rs b/hydroflow_datalog_core/src/grammar.rs index 4fd48dfd866a..5076bc6811f6 100644 --- a/hydroflow_datalog_core/src/grammar.rs +++ b/hydroflow_datalog_core/src/grammar.rs @@ -74,17 +74,28 @@ pub mod datalog { #[derive(Debug, Clone)] pub enum Atom { - Relation( - #[rust_sitter::leaf(text = "!")] Option<()>, + NegRelation( + #[rust_sitter::leaf(text = "!")] (), Spanned, ), + PosRelation(Spanned), Predicate(Spanned), } #[derive(Debug, Clone)] - pub enum IdentOrUnderscore { + pub enum ExtractExpr { Ident(Spanned), Underscore(#[rust_sitter::leaf(text = "_")] Spanned<()>), + Flatten(#[rust_sitter::leaf(text = "*")] (), Box), + Untuple( + #[rust_sitter::leaf(text = "(")] (), + #[rust_sitter::delimited( + #[rust_sitter::leaf(text = ",")] + () + )] + Vec>, + #[rust_sitter::leaf(text = ")")] (), + ), } #[derive(Debug, Clone)] @@ -98,7 +109,7 @@ pub mod datalog { #[rust_sitter::leaf(text = ",")] () )] - pub fields: Vec>, + pub fields: Vec>, #[rust_sitter::leaf(text = ")")] _r_paren: (), @@ -135,7 +146,6 @@ pub mod datalog { #[derive(Debug, Clone)] pub enum TargetExpr { Expr(IntExpr), - Splat(#[rust_sitter::leaf(text = "*")] (), Spanned), Aggregation(Aggregation), Index( #[rust_sitter::leaf(text = "index")] (), @@ -148,7 +158,6 @@ pub mod datalog { pub fn idents(&self) -> Vec<&Ident> { match self { TargetExpr::Expr(e) => e.idents(), - TargetExpr::Splat(_, i) => vec![&i.value], TargetExpr::Aggregation(Aggregation::Count(_)) => vec![], TargetExpr::Aggregation( Aggregation::CountUnique(_, _, idents, _) diff --git a/hydroflow_datalog_core/src/join_plan.rs b/hydroflow_datalog_core/src/join_plan.rs index 76f683c96afa..ba7b7431dbab 100644 --- a/hydroflow_datalog_core/src/join_plan.rs +++ b/hydroflow_datalog_core/src/join_plan.rs @@ -1,6 +1,5 @@ use std::collections::btree_map::Entry; use std::collections::{BTreeMap, HashMap}; -use std::ops::Deref; use hydroflow_lang::diagnostic::{Diagnostic, Level}; use hydroflow_lang::graph::FlatGraphBuilder; @@ -9,7 +8,7 @@ use proc_macro2::Span; use rust_sitter::Spanned; use syn::{parse_quote, parse_quote_spanned}; -use crate::grammar::datalog::{BoolExpr, BoolOp, IdentOrUnderscore, InputRelationExpr, IntExpr}; +use crate::grammar::datalog::{BoolExpr, BoolOp, ExtractExpr, InputRelationExpr, IntExpr}; use crate::util::{repeat_tuple, Counter}; /// Captures the tree of joins used to compute contributions from a single rule. @@ -23,7 +22,7 @@ pub enum JoinPlan<'a> { Predicate(Vec<&'a Spanned>, Box>), /// A join between some relation and a magic relation that emits values between /// 0 and some value in the input relation (upper-exclusive). - MagicNatLt(Box>, IdentOrUnderscore, IdentOrUnderscore), + MagicNatLt(Box>, ExtractExpr, ExtractExpr), } /// Tracks the Hydroflow node that corresponds to a subtree of a join plan. @@ -139,33 +138,6 @@ fn emit_join_input_pipeline( flat_graph_builder.add_statement(statement); } -/// Creates a mapping from variable names to the indices where that variable appears in `fields`. -/// -/// Only return entries for variables that appear more than once. Those correspond to additional -/// constraints: the relation is only true when the values at those indices are equal. -/// -/// For example, `rel(a, b, a) := ...` requires that the values in the 0th and 2nd slots be the -/// same, so we would return a map `{ "a" => [0, 2] }`. Note that since `b` is not repeated, it is -/// not in the map. -fn find_relation_local_constraints<'a>( - fields: impl Iterator>, -) -> BTreeMap> { - let mut indices_grouped_by_var = BTreeMap::new(); - for (i, ident) in fields.enumerate() { - if let IdentOrUnderscore::Ident(ident) = ident.deref() { - let entry: &mut Vec<_> = indices_grouped_by_var - // TODO(shadaj): Can we avoid cloning here? - .entry(ident.name.clone()) - .or_default(); - entry.push(i); - } - } - - indices_grouped_by_var.retain(|_, v| v.len() > 1); - - indices_grouped_by_var -} - /// Given a mapping from variable names to their repeated indices, builds a Rust expression that /// tests whether the values at those indices are equal for each variable. /// @@ -217,6 +189,144 @@ fn gen_predicate_value_expr( ) } +#[allow(clippy::too_many_arguments)] +/// Processes an extract expression to generate a Hydroflow pipeline that reads the input +/// data from the IDB/EDB. +/// +/// `row_width` is the number of elements in the tuples emitted by the **current** pipeline, +/// with all transformations that have been applied while extracting variable so far. The +/// `cur_row_offset` specifies the index of the current `ExtractExpr` in that tuple. If it +/// is `None`, then we are the top-level expression and already have a tuple. +/// +/// This function returns the number of elements in the tuple that will be emitted by the +/// extraction of the `ExtractExpr`. So for a single variable, it will return `1`, for a +/// tuple, it will return sum of the number of elements emitted by its children. +fn process_extract( + extract: &ExtractExpr, + variable_mapping: &mut BTreeMap, + local_constraints: &mut BTreeMap>, + wildcard_indices: &mut Vec, + reader_pipeline: &mut Pipeline, + row_width: usize, + cur_row_offset: Option, // None if at the root and we are already a tuple + rule_span: Span, +) -> usize { + match extract { + ExtractExpr::Underscore(_) => { + wildcard_indices.push(cur_row_offset.unwrap()); + 1 + } + ExtractExpr::Ident(ident) => { + if let Entry::Vacant(e) = variable_mapping.entry(ident.name.clone()) { + e.insert(cur_row_offset.unwrap()); + } + + local_constraints + .entry(ident.name.clone()) + .or_default() + .push(cur_row_offset.unwrap()); + + 1 + } + ExtractExpr::Flatten(_, expr) => { + let cur_row_offset = cur_row_offset.unwrap(); + let tuple_elems_post_flat = (0..row_width) + .map(|i| { + if i == cur_row_offset { + parse_quote!(__flattened_element) + } else { + let idx: syn::Index = syn::Index::from(i); + parse_quote!(::std::clone::Clone::clone(&row.#idx)) + } + }) + .collect::>(); + + let flat_idx = syn::Index::from(cur_row_offset); + + let mut row_types: Vec = vec![]; + for _ in 0..row_width { + row_types.push(parse_quote!(_)); + } + + let row_type: syn::Type = parse_quote!((#(#row_types, )*)); + + *reader_pipeline = parse_quote_spanned! {rule_span=> + #reader_pipeline -> flat_map(|row: #row_type| row.#flat_idx.into_iter().map(move |__flattened_element| (#(#tuple_elems_post_flat, )*))) + }; + + process_extract( + expr, + variable_mapping, + local_constraints, + wildcard_indices, + reader_pipeline, + row_width, + Some(cur_row_offset), + rule_span, + ) + } + ExtractExpr::Untuple(_, tuple_elems, _) => { + let mut new_row_width = if let Some(cur_row_offset) = cur_row_offset { + let flat_idx = syn::Index::from(cur_row_offset); + + let tuple_elems_post_flat = (0..row_width) + .flat_map(|i| { + if i == cur_row_offset { + (0..tuple_elems.len()) + .map(|tuple_i| { + let idx: syn::Index = syn::Index::from(tuple_i); + parse_quote!(row_untuple.#flat_idx.#idx) + }) + .collect::>() + } else { + let idx: syn::Index = syn::Index::from(i); + vec![parse_quote!(row_untuple.#idx)] + } + }) + .collect::>(); + + let mut row_types: Vec = vec![]; + for _ in 0..row_width { + row_types.push(parse_quote!(_)); + } + + let row_type: syn::Type = parse_quote!((#(#row_types, )*)); + + *reader_pipeline = parse_quote_spanned! {rule_span=> + #reader_pipeline -> map(|row_untuple: #row_type| (#(#tuple_elems_post_flat, )*)) + }; + + row_width - 1 + tuple_elems.len() + } else { + row_width + }; + + let base_offset = cur_row_offset.unwrap_or_default(); + let mut expanded_row_elements = 0; + for expr in tuple_elems { + let expanded_width = process_extract( + expr, + variable_mapping, + local_constraints, + wildcard_indices, + reader_pipeline, + new_row_width, + Some(base_offset + expanded_row_elements), + rule_span, + ); + + // as we process each child of the tuple, the prefix of the + // tuple emitted by the pipeline will grow, so we need to update + // our cursor and the current overall width appropriately + expanded_row_elements += expanded_width; + new_row_width = new_row_width - 1 + expanded_width; + } + + expanded_row_elements + } + } +} + /// Generates a Hydroflow pipeline that computes the output to a given [`JoinPlan`]. pub fn expand_join_plan( // The plan we are converting to a Hydroflow pipeline. @@ -231,24 +341,6 @@ pub fn expand_join_plan( ) -> IntermediateJoinNode { match plan { JoinPlan::Source(target, persisted) => { - let mut variable_mapping = BTreeMap::new(); - let mut wildcard_indices = vec![]; - let mut row_types: Vec = vec![]; - - let local_constraints = find_relation_local_constraints(target.fields.iter()); - - for (i, ident) in target.fields.iter().enumerate() { - row_types.push(parse_quote!(_)); - - if let IdentOrUnderscore::Ident(ident) = ident.deref() { - if let Entry::Vacant(e) = variable_mapping.entry(ident.name.clone()) { - e.insert(i); - } - } else { - wildcard_indices.push(i); - } - } - // Because this is a node corresponding to some Datalog relation, we need to tee from it. let tee_index = tee_counter .entry(target.name.name.clone()) @@ -256,39 +348,57 @@ pub fn expand_join_plan( .next() .expect("Out of tee indices"); - let row_type = parse_quote!((#(#row_types, )*)); - - if local_constraints.is_empty() { - return IntermediateJoinNode { - name: syn::Ident::new(&target.name.name, get_span(target.name.span)), - persisted: *persisted, - tee_idx: Some(tee_index), - variable_mapping, - wildcard_indices, - tuple_type: row_type, - span: get_span(target.span), - }; - } - let relation_node = syn::Ident::new(&target.name.name, get_span(target.name.span)); let relation_idx = syn::LitInt::new(&tee_index.to_string(), Span::call_site()); - let filter_node = syn::Ident::new( + let source_node = syn::Ident::new( &format!( - "join_{}_filter", + "source_reader_{}", next_join_idx.next().expect("Out of join indices") ), Span::call_site(), ); - let conditions = build_local_constraint_conditions(&local_constraints); + let mut variable_mapping = BTreeMap::new(); + let mut local_constraints = BTreeMap::new(); + let mut wildcard_indices = vec![]; + + let mut pipeline: Pipeline = parse_quote_spanned! {get_span(rule_span)=> + #relation_node [#relation_idx] + }; + + let final_row_width = process_extract( + &ExtractExpr::Untuple((), target.fields.clone(), ()), + &mut variable_mapping, + &mut local_constraints, + &mut wildcard_indices, + &mut pipeline, + target.fields.len(), + None, + get_span(rule_span), + ); + + let mut row_types: Vec = vec![]; + for _ in 0..final_row_width { + row_types.push(parse_quote!(_)); + } + + let row_type = parse_quote!((#(#row_types, )*)); + + if local_constraints.values().any(|v| v.len() > 1) { + let conditions = build_local_constraint_conditions(&local_constraints); + + pipeline = parse_quote_spanned! {get_span(rule_span)=> + #pipeline -> filter(|row: &#row_type| #conditions) + }; + } flat_graph_builder.add_statement(parse_quote_spanned! {get_span(rule_span)=> - #filter_node = #relation_node [#relation_idx] -> filter(|row: &#row_type| #conditions); + #source_node = #pipeline; }); IntermediateJoinNode { - name: filter_node, + name: source_node, persisted: *persisted, tee_idx: None, variable_mapping, @@ -596,16 +706,17 @@ pub fn expand_join_plan( let inner_name = inner_expanded.name.clone(); let row_type = inner_expanded.tuple_type; - if let IdentOrUnderscore::Ident(less_than) = less_than { - if inner_expanded - .variable_mapping - .contains_key(&less_than.name) - { - todo!("The values generated by less_than cannot currently be used in other parts of the query"); + match &less_than { + ExtractExpr::Ident(ident) => { + if inner_expanded.variable_mapping.contains_key(&ident.name) { + todo!("The values generated by less_than cannot currently be used in other parts of the query"); + } } + ExtractExpr::Underscore(_) => {} + _ => panic!("The values generated by less_than must be a single variable"), } - let threshold_name = if let IdentOrUnderscore::Ident(threshold) = threshold { + let threshold_name = if let ExtractExpr::Ident(threshold) = threshold { threshold.name.clone() } else { panic!("The threshold must be a variable") @@ -633,7 +744,7 @@ pub fn expand_join_plan( flattened_elements.push(parse_quote!(row.#syn_wildcard_idx.clone())); } - if let IdentOrUnderscore::Ident(less_than) = less_than { + if let ExtractExpr::Ident(less_than) = less_than { if less_than.name == threshold_name { panic!("The threshold and less_than variables must be different") } diff --git a/hydroflow_datalog_core/src/lib.rs b/hydroflow_datalog_core/src/lib.rs index eb25c6595194..4cec6b702452 100644 --- a/hydroflow_datalog_core/src/lib.rs +++ b/hydroflow_datalog_core/src/lib.rs @@ -456,8 +456,8 @@ fn compute_join_plan<'a>(sources: &'a [Atom], persisted_rules: &HashSet) let mut plan: JoinPlan = sources .iter() .filter_map(|x| match x { - Atom::Relation(negated, e) => { - if negated.is_none() && !MAGIC_RELATIONS.contains(&e.name.name.as_str()) { + Atom::PosRelation(e) => { + if !MAGIC_RELATIONS.contains(&e.name.name.as_str()) { Some(JoinPlan::Source(e, persisted_rules.contains(&e.name.name))) } else { None @@ -471,12 +471,8 @@ fn compute_join_plan<'a>(sources: &'a [Atom], persisted_rules: &HashSet) plan = sources .iter() .filter_map(|x| match x { - Atom::Relation(negated, e) => { - if negated.is_some() { - Some(JoinPlan::Source(e, persisted_rules.contains(&e.name.name))) - } else { - None - } + Atom::NegRelation(_, e) => { + Some(JoinPlan::Source(e, persisted_rules.contains(&e.name.name))) } _ => None, }) @@ -495,17 +491,14 @@ fn compute_join_plan<'a>(sources: &'a [Atom], persisted_rules: &HashSet) } plan = sources.iter().fold(plan, |acc, atom| match atom { - Atom::Relation(negated, e) => { + Atom::PosRelation(e) => { if MAGIC_RELATIONS.contains(&e.name.name.as_str()) { match e.name.name.as_str() { - "less_than" => { - assert!(negated.is_none()); - JoinPlan::MagicNatLt( - Box::new(acc), - e.fields[0].value.clone(), - e.fields[1].value.clone(), - ) - } + "less_than" => JoinPlan::MagicNatLt( + Box::new(acc), + e.fields[0].value.clone(), + e.fields[1].value.clone(), + ), o => panic!("Unknown magic relation {}", o), } } else { @@ -563,7 +556,6 @@ fn gen_target_expr( ) -> syn::Expr { match expr { TargetExpr::Expr(expr) => gen_value_expr(expr, lookup_ident, get_span), - TargetExpr::Splat(_, ident) => lookup_ident(ident), TargetExpr::Aggregation(Aggregation::Count(_)) => parse_quote!(()), TargetExpr::Aggregation(Aggregation::CountUnique(_, _, keys, _)) | TargetExpr::Aggregation(Aggregation::CollectVec(_, _, keys, _)) => { @@ -614,7 +606,6 @@ fn apply_aggregations( let mut copy_group_key_lookups: Vec = vec![]; let mut after_group_lookups: Vec = vec![]; - let mut splat_indices = vec![]; let mut group_key_idx = 0; let mut agg_idx = 0; @@ -670,18 +661,6 @@ fn apply_aggregations( .push(parse_quote_spanned!(get_span(field.span)=> g.#idx)); group_key_idx += 1; } - TargetExpr::Splat(_, _) => { - fold_keyed_exprs.push(expr); - - let idx = syn::Index::from(group_key_idx); - after_group_lookups.push(parse_quote_spanned!(get_span(field.span)=> g.#idx)); - copy_group_key_lookups - .push(parse_quote_spanned!(get_span(field.span)=> g.#idx)); - - splat_indices.push(group_key_idx); - - group_key_idx += 1; - } TargetExpr::Aggregation(a) => { aggregations.push(a.clone()); agg_exprs.push(expr); @@ -729,19 +708,7 @@ fn apply_aggregations( parse_quote!(map(|(g, a): (#fold_keyed_input_type, _)| (#(#after_group_lookups, )*))) }; - let mut pre_fold_keyed_map: Pipeline = parse_quote!(map(|row: #flattened_tuple_type| ((#(#fold_keyed_exprs, )*), (#(#agg_exprs, )*)))); - for idx in splat_indices { - let mut lookups: Vec = copy_group_key_lookups - .iter() - .cloned() - .map(|e| parse_quote!(#e.clone())) - .collect::>(); - lookups[idx] = parse_quote!(__splatted); - - let syn_idx = syn::Index::from(idx); - - pre_fold_keyed_map = parse_quote!(#pre_fold_keyed_map -> flat_map(|(g, a): (#fold_keyed_input_type, _)| g.#syn_idx.into_iter().map(move |__splatted| ((#(#lookups, )*), a.clone()))) -> unique::<'tick>()); - } + let pre_fold_keyed_map: Pipeline = parse_quote!(map(|row: #flattened_tuple_type| ((#(#fold_keyed_exprs, )*), (#(#agg_exprs, )*)))); if agg_exprs.is_empty() { if out_expanded.persisted && !consumer_is_persist { @@ -1250,14 +1217,66 @@ mod tests { } #[test] - fn test_splat() { + fn test_flatten() { + test_snapshots!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b) :- ints1(a, *b) + "# + ); + } + + #[test] + fn test_detuple() { + test_snapshots!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b) :- ints1((a, b)) + "# + ); + } + + #[test] + fn test_multi_detuple() { + test_snapshots!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b, c, d) :- ints1((a, b), (c, d)) + "# + ); + } + + #[test] + fn test_flat_then_detuple() { + test_snapshots!( + r#" + .input ints1 `source_stream(ints1)` + + .output result `for_each(|v| result.send(v).unwrap())` + + result(a, b) :- ints1(*(a, b)) + "# + ); + } + + #[test] + fn test_detuple_then_flat() { test_snapshots!( r#" .input ints1 `source_stream(ints1)` .output result `for_each(|v| result.send(v).unwrap())` - result(a, *b) :- ints1(a, b) + result(a, b) :- ints1((*a, *b)) "# ); } diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__anti_join@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__anti_join@datalog_program.snap index 3f887c3812fa..5338c9c5ad3d 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__anti_join@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__anti_join@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints_1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints_2)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints_3)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 ,) , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"anti_join ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , (_ , _ ,)) | (kv . 0 . 0 , kv . 1 . 0 , kv . 1 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ , _ ,) | ((_v . 0 ,) , (_v . 1 , _v . 2 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (_v . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ ,) | ((row . 1 , row . 2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":5},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":9,\"version\":3}],\"version\":5},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":3},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":3},{\"idx\":21,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":12,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1},{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_1\",\"version\":1},{\"value\":\"join_1\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":4,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":4,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":4,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints_1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints_2)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints_3)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 ,) , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"anti_join ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , (_ , _ ,)) | (kv . 0 . 0 , kv . 1 . 0 , kv . 1 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ , _ ,) | ((_v . 0 ,) , (_v . 1 , _v . 2 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (_v . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ ,) | ((row . 1 , row . 2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":5},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":9,\"version\":3}],\"version\":5},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":3},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":3},{\"idx\":21,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":12,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1},{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_4\",\"version\":1},{\"value\":\"join_4\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":4,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":4,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":4,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_6v3_send, hoff_6v3_recv) = df @@ -1013,4 +1013,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__collect_vec@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__collect_vec@datalog_program.snap index cedabe72a0ad..4ef499e19518 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__collect_vec@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__collect_vec@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : (() , ((_ ,) , (_ ,))) | (kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | (() , ((row . 0 , row . 1) ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"fold_keyed :: < 'tick , () , (Option < _ > ,) > (| | (None ,) , | old : & mut (Option < _ > ,) , val : (_ ,) | { old . 0 = if let Some (prev) = old . 0 . take () { Some ({ let mut set : hydroflow :: rustc_hash :: FxHashSet < _ > = prev ; set . insert (val . 0) ; set }) } else { Some ({ let mut set = hydroflow :: rustc_hash :: FxHashSet :: < _ > :: default () ; set . insert (val . 0) ; set }) } ; })\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : (() , _) | (a . 0 . unwrap () . into_iter () . collect :: < Vec < _ > > () ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":3},{\"idx\":18,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":9,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : (() , ((_ ,) , (_ ,))) | (kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | (() , ((row . 0 , row . 1) ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"fold_keyed :: < 'tick , () , (Option < _ > ,) > (| | (None ,) , | old : & mut (Option < _ > ,) , val : (_ ,) | { old . 0 = if let Some (prev) = old . 0 . take () { Some ({ let mut set : hydroflow :: rustc_hash :: FxHashSet < _ > = prev ; set . insert (val . 0) ; set }) } else { Some ({ let mut set = hydroflow :: rustc_hash :: FxHashSet :: < _ > :: default () ; set . insert (val . 0) ; set }) } ; })\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : (() , _) | (a . 0 . unwrap () . into_iter () . collect :: < Vec < _ > > () ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":3},{\"idx\":18,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":9,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_9v3_send, hoff_9v3_recv) = df @@ -743,4 +743,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@datalog_program.snap new file mode 100644 index 000000000000..7526c37826c4 --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@datalog_program.snap @@ -0,0 +1,341 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: "prettyplease::unparse(&wrapped)" +--- +fn main() { + { + #[allow(unused_qualifications)] + { + use hydroflow::{var_expr, var_args}; + let mut df = hydroflow::scheduled::graph::Hydroflow::new(); + df.__assign_meta_graph( + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row_untuple : (_ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"source_reader_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + ); + df.__assign_diagnostics("[]"); + let mut sg_1v1_node_7v1_stream = { + #[inline(always)] + fn check_stream< + Stream: hydroflow::futures::stream::Stream + + ::std::marker::Unpin, + Item, + >( + stream: Stream, + ) -> impl hydroflow::futures::stream::Stream< + Item = Item, + > + ::std::marker::Unpin { + stream + } + check_stream(ints1) + }; + let sg_1v1_node_2v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + let sg_1v1_node_5v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + df.add_subgraph_stratified( + "Subgraph GraphSubgraphId(1v1)", + 0, + var_expr!(), + var_expr!(), + false, + move |context, var_args!(), var_args!()| { + let op_7v1 = std::iter::from_fn(|| { + match hydroflow::futures::stream::Stream::poll_next( + ::std::pin::Pin::new(&mut sg_1v1_node_7v1_stream), + &mut std::task::Context::from_waker(&context.waker()), + ) { + std::task::Poll::Ready(maybe) => maybe, + std::task::Poll::Pending => None, + } + }); + let op_7v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_7v1__source_stream__loc_unknown_start_2_26_end_2_46< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_7v1__source_stream__loc_unknown_start_2_26_end_2_46(op_7v1) + }; + let op_2v1 = op_7v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_2v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_2v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_2v1__unique__loc_unknown_start_2_19_end_2_24< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_2v1__unique__loc_unknown_start_2_19_end_2_24(op_2v1) + }; + let op_9v1 = op_2v1 + .map(|row_untuple: (_,)| (row_untuple.0.0, row_untuple.0.1)); + let op_9v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_9v1__map__loc_unknown_start_6_12_end_6_41< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_9v1__map__loc_unknown_start_6_12_end_6_41(op_9v1) + }; + let op_10v1 = op_9v1.map(|row: (_, _)| ((row.0, row.1), ())); + let op_10v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_10v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_10v1__map__loc_unknown_start_1_0_end_1_0(op_10v1) + }; + let op_11v1 = op_10v1.map(|(g, a): ((_, _), _)| (g.0, g.1)); + let op_11v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_11v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_11v1__map__loc_unknown_start_1_0_end_1_0(op_11v1) + }; + let op_5v1 = op_11v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_5v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_5v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_5v1__unique__loc_unknown_start_4_20_end_4_26< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_5v1__unique__loc_unknown_start_4_20_end_4_26(op_5v1) + }; + let op_8v1 = hydroflow::pusherator::for_each::ForEach::new(|v| { + result.send(v).unwrap() + }); + let op_8v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_8v1__for_each__loc_unknown_start_4_28_end_4_65< + Item, + Input: hydroflow::pusherator::Pusherator, + >( + input: Input, + ) -> impl hydroflow::pusherator::Pusherator { + #[repr(transparent)] + struct Push< + Item, + Input: hydroflow::pusherator::Pusherator, + > { + inner: Input, + } + impl< + Item, + Input: hydroflow::pusherator::Pusherator, + > hydroflow::pusherator::Pusherator for Push { + type Item = Item; + #[inline(always)] + fn give(&mut self, item: Self::Item) { + self.inner.give(item) + } + } + Push { inner: input } + } + op_8v1__for_each__loc_unknown_start_4_28_end_4_65(op_8v1) + }; + #[inline(always)] + fn pivot_run_sg_1v1< + Pull: ::std::iter::Iterator, + Push: hydroflow::pusherator::Pusherator, + Item, + >(pull: Pull, push: Push) { + hydroflow::pusherator::pivot::Pivot::new(pull, push).run(); + } + pivot_run_sg_1v1(op_5v1, op_8v1); + }, + ); + df + } + } +} diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@surface_graph.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@surface_graph.snap new file mode 100644 index 000000000000..ee9656ed1099 --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple@surface_graph.snap @@ -0,0 +1,18 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: flat_graph_ref.surface_syntax_string() +--- +2v1 = unique :: < 'tick > (); +5v1 = unique :: < 'tick > (); +7v1 = source_stream (ints1); +8v1 = for_each (| v | result . send (v) . unwrap ()); +9v1 = map (| row_untuple : (_ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 ,)); +10v1 = map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ())); +11v1 = map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,)); + +7v1 -> 2v1; +11v1 -> 5v1; +5v1 -> 8v1; +2v1 -> 9v1; +10v1 -> 11v1; +9v1 -> 10v1; diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@datalog_program.snap new file mode 100644 index 000000000000..d3f74f121316 --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@datalog_program.snap @@ -0,0 +1,425 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: "prettyplease::unparse(&wrapped)" +--- +fn main() { + { + #[allow(unused_qualifications)] + { + use hydroflow::{var_expr, var_args}; + let mut df = hydroflow::scheduled::graph::Hydroflow::new(); + df.__assign_meta_graph( + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row_untuple : (_ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"flat_map (| row : (_ , _ ,) | row . 0 . into_iter () . map (move | __flattened_element | (__flattened_element , :: std :: clone :: Clone :: clone (& row . 1) ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"flat_map (| row : (_ , _ ,) | row . 1 . into_iter () . map (move | __flattened_element | (:: std :: clone :: Clone :: clone (& row . 0) , __flattened_element ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"source_reader_0\",\"version\":1},{\"value\":\"source_reader_0\",\"version\":1},{\"value\":\"source_reader_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + ); + df.__assign_diagnostics("[]"); + let mut sg_1v1_node_7v1_stream = { + #[inline(always)] + fn check_stream< + Stream: hydroflow::futures::stream::Stream + + ::std::marker::Unpin, + Item, + >( + stream: Stream, + ) -> impl hydroflow::futures::stream::Stream< + Item = Item, + > + ::std::marker::Unpin { + stream + } + check_stream(ints1) + }; + let sg_1v1_node_2v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + let sg_1v1_node_5v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + df.add_subgraph_stratified( + "Subgraph GraphSubgraphId(1v1)", + 0, + var_expr!(), + var_expr!(), + false, + move |context, var_args!(), var_args!()| { + let op_7v1 = std::iter::from_fn(|| { + match hydroflow::futures::stream::Stream::poll_next( + ::std::pin::Pin::new(&mut sg_1v1_node_7v1_stream), + &mut std::task::Context::from_waker(&context.waker()), + ) { + std::task::Poll::Ready(maybe) => maybe, + std::task::Poll::Pending => None, + } + }); + let op_7v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_7v1__source_stream__loc_unknown_start_2_26_end_2_46< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_7v1__source_stream__loc_unknown_start_2_26_end_2_46(op_7v1) + }; + let op_2v1 = op_7v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_2v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_2v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_2v1__unique__loc_unknown_start_2_19_end_2_24< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_2v1__unique__loc_unknown_start_2_19_end_2_24(op_2v1) + }; + let op_9v1 = op_2v1 + .map(|row_untuple: (_,)| (row_untuple.0.0, row_untuple.0.1)); + let op_9v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_9v1__map__loc_unknown_start_6_12_end_6_43< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_9v1__map__loc_unknown_start_6_12_end_6_43(op_9v1) + }; + let op_10v1 = op_9v1 + .flat_map(|row: (_, _)| { + row + .0 + .into_iter() + .map(move |__flattened_element| ( + __flattened_element, + ::std::clone::Clone::clone(&row.1), + )) + }); + let op_10v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_10v1__flat_map__loc_unknown_start_6_12_end_6_43< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_10v1__flat_map__loc_unknown_start_6_12_end_6_43(op_10v1) + }; + let op_11v1 = op_10v1 + .flat_map(|row: (_, _)| { + row + .1 + .into_iter() + .map(move |__flattened_element| ( + ::std::clone::Clone::clone(&row.0), + __flattened_element, + )) + }); + let op_11v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_11v1__flat_map__loc_unknown_start_6_12_end_6_43< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_11v1__flat_map__loc_unknown_start_6_12_end_6_43(op_11v1) + }; + let op_12v1 = op_11v1.map(|row: (_, _)| ((row.0, row.1), ())); + let op_12v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_12v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_12v1__map__loc_unknown_start_1_0_end_1_0(op_12v1) + }; + let op_13v1 = op_12v1.map(|(g, a): ((_, _), _)| (g.0, g.1)); + let op_13v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_13v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_13v1__map__loc_unknown_start_1_0_end_1_0(op_13v1) + }; + let op_5v1 = op_13v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_5v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_5v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_5v1__unique__loc_unknown_start_4_20_end_4_26< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_5v1__unique__loc_unknown_start_4_20_end_4_26(op_5v1) + }; + let op_8v1 = hydroflow::pusherator::for_each::ForEach::new(|v| { + result.send(v).unwrap() + }); + let op_8v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_8v1__for_each__loc_unknown_start_4_28_end_4_65< + Item, + Input: hydroflow::pusherator::Pusherator, + >( + input: Input, + ) -> impl hydroflow::pusherator::Pusherator { + #[repr(transparent)] + struct Push< + Item, + Input: hydroflow::pusherator::Pusherator, + > { + inner: Input, + } + impl< + Item, + Input: hydroflow::pusherator::Pusherator, + > hydroflow::pusherator::Pusherator for Push { + type Item = Item; + #[inline(always)] + fn give(&mut self, item: Self::Item) { + self.inner.give(item) + } + } + Push { inner: input } + } + op_8v1__for_each__loc_unknown_start_4_28_end_4_65(op_8v1) + }; + #[inline(always)] + fn pivot_run_sg_1v1< + Pull: ::std::iter::Iterator, + Push: hydroflow::pusherator::Pusherator, + Item, + >(pull: Pull, push: Push) { + hydroflow::pusherator::pivot::Pivot::new(pull, push).run(); + } + pivot_run_sg_1v1(op_5v1, op_8v1); + }, + ); + df + } + } +} diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@surface_graph.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@surface_graph.snap new file mode 100644 index 000000000000..d6e0a64106fc --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__detuple_then_flat@surface_graph.snap @@ -0,0 +1,22 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: flat_graph_ref.surface_syntax_string() +--- +2v1 = unique :: < 'tick > (); +5v1 = unique :: < 'tick > (); +7v1 = source_stream (ints1); +8v1 = for_each (| v | result . send (v) . unwrap ()); +9v1 = map (| row_untuple : (_ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 ,)); +10v1 = flat_map (| row : (_ , _ ,) | row . 0 . into_iter () . map (move | __flattened_element | (__flattened_element , :: std :: clone :: Clone :: clone (& row . 1) ,))); +11v1 = flat_map (| row : (_ , _ ,) | row . 1 . into_iter () . map (move | __flattened_element | (:: std :: clone :: Clone :: clone (& row . 0) , __flattened_element ,))); +12v1 = map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ())); +13v1 = map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,)); + +7v1 -> 2v1; +13v1 -> 5v1; +5v1 -> 8v1; +10v1 -> 11v1; +9v1 -> 10v1; +2v1 -> 9v1; +12v1 -> 13v1; +11v1 -> 12v1; diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__expr_predicate@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__expr_predicate@datalog_program.snap index e328ea2a8b7a..e5e663dc4209 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__expr_predicate@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__expr_predicate@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"union ()\"},\"version\":1},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 == 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 != 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 - 1 == 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((3 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 - 1 == 1 - 1)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((4 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":10,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":1,\"version\":3},{\"idx\":13,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":1,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":22,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"2\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"3\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":4,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_insert\",\"version\":1},{\"value\":\"ints\",\"version\":1},{\"value\":\"result_insert\",\"version\":1},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_0_filter\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_1_filter\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_2_filter\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_3_filter\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":17,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"union ()\"},\"version\":1},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 == 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 != 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 - 1 == 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((3 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ ,) | row . 0 - 1 == 1 - 1)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((4 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":10,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":1,\"version\":3},{\"idx\":13,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":1,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":22,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[{\"Int\":\"2\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"2\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[{\"Int\":\"3\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"3\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":4,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints_insert\",\"version\":1},{\"value\":\"ints\",\"version\":1},{\"value\":\"result_insert\",\"version\":1},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_1_filter\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_3_filter\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_5_filter\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_7_filter\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":17,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_1v3_send, hoff_1v3_recv) = df @@ -801,4 +801,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__splat@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flat_then_detuple@datalog_program.snap similarity index 75% rename from hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__splat@datalog_program.snap rename to hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flat_then_detuple@datalog_program.snap index d6ab19bdbe8a..66dff455539b 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__splat@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flat_then_detuple@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"flat_map (| (g , a) : ((_ , _ ,) , _) | g . 1 . into_iter () . map (move | __splatted | ((g . 0 . clone () , __splatted ,) , a . clone ())))\"},\"version\":1},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"flat_map (| row : (_ ,) | row . 0 . into_iter () . map (move | __flattened_element | (__flattened_element ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row_untuple : (_ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"source_reader_0\",\"version\":1},{\"value\":\"source_reader_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let mut sg_1v1_node_7v1_stream = { @@ -36,15 +36,6 @@ fn main() { >::default(), ), ); - let sg_1v1_node_11v1_uniquedata = df - .add_state( - ::std::cell::RefCell::new( - hydroflow::util::monotonic_map::MonotonicMap::< - _, - hydroflow::rustc_hash::FxHashSet<_>, - >::default(), - ), - ); let sg_1v1_node_5v1_uniquedata = df .add_state( ::std::cell::RefCell::new( @@ -151,11 +142,17 @@ fn main() { } op_2v1__unique__loc_unknown_start_2_19_end_2_24(op_2v1) }; - let op_9v1 = op_2v1.map(|row: (_, _)| ((row.0, row.1), ())); + let op_9v1 = op_2v1 + .flat_map(|row: (_,)| { + row + .0 + .into_iter() + .map(move |__flattened_element| (__flattened_element,)) + }); let op_9v1 = { #[allow(non_snake_case)] #[inline(always)] - pub fn op_9v1__map__loc_unknown_start_1_0_end_1_0< + pub fn op_9v1__flat_map__loc_unknown_start_6_12_end_6_42< Item, Input: ::std::iter::Iterator, >(input: Input) -> impl ::std::iter::Iterator { @@ -182,22 +179,14 @@ fn main() { } Pull { inner: input } } - op_9v1__map__loc_unknown_start_1_0_end_1_0(op_9v1) + op_9v1__flat_map__loc_unknown_start_6_12_end_6_42(op_9v1) }; let op_10v1 = op_9v1 - .flat_map(|(g, a): ((_, _), _)| { - g - .1 - .into_iter() - .map(move |__splatted| ( - (g.0.clone(), __splatted), - a.clone(), - )) - }); + .map(|row_untuple: (_,)| (row_untuple.0.0, row_untuple.0.1)); let op_10v1 = { #[allow(non_snake_case)] #[inline(always)] - pub fn op_10v1__flat_map__loc_unknown_start_1_0_end_1_0< + pub fn op_10v1__map__loc_unknown_start_6_12_end_6_42< Item, Input: ::std::iter::Iterator, >(input: Input) -> impl ::std::iter::Iterator { @@ -224,29 +213,13 @@ fn main() { } Pull { inner: input } } - op_10v1__flat_map__loc_unknown_start_1_0_end_1_0(op_10v1) + op_10v1__map__loc_unknown_start_6_12_end_6_42(op_10v1) }; - let op_11v1 = op_10v1 - .filter(|item| { - let mut borrow = context - .state_ref(sg_1v1_node_11v1_uniquedata) - .borrow_mut(); - let set = borrow - .get_mut_clear(( - context.current_tick(), - context.current_stratum(), - )); - if !set.contains(item) { - set.insert(::std::clone::Clone::clone(item)); - true - } else { - false - } - }); + let op_11v1 = op_10v1.map(|row: (_, _)| ((row.0, row.1), ())); let op_11v1 = { #[allow(non_snake_case)] #[inline(always)] - pub fn op_11v1__unique__loc_unknown_start_1_0_end_1_0< + pub fn op_11v1__map__loc_unknown_start_1_0_end_1_0< Item, Input: ::std::iter::Iterator, >(input: Input) -> impl ::std::iter::Iterator { @@ -273,7 +246,7 @@ fn main() { } Pull { inner: input } } - op_11v1__unique__loc_unknown_start_1_0_end_1_0(op_11v1) + op_11v1__map__loc_unknown_start_1_0_end_1_0(op_11v1) }; let op_12v1 = op_11v1.map(|(g, a): ((_, _), _)| (g.0, g.1)); let op_12v1 = { @@ -405,4 +378,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__splat@surface_graph.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flat_then_detuple@surface_graph.snap similarity index 59% rename from hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__splat@surface_graph.snap rename to hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flat_then_detuple@surface_graph.snap index 75feb6cab16e..3fd1db96c50a 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__splat@surface_graph.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flat_then_detuple@surface_graph.snap @@ -6,15 +6,15 @@ expression: flat_graph_ref.surface_syntax_string() 5v1 = unique :: < 'tick > (); 7v1 = source_stream (ints1); 8v1 = for_each (| v | result . send (v) . unwrap ()); -9v1 = map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ())); -10v1 = flat_map (| (g , a) : ((_ , _ ,) , _) | g . 1 . into_iter () . map (move | __splatted | ((g . 0 . clone () , __splatted ,) , a . clone ()))); -11v1 = unique :: < 'tick > (); +9v1 = flat_map (| row : (_ ,) | row . 0 . into_iter () . map (move | __flattened_element | (__flattened_element ,))); +10v1 = map (| row_untuple : (_ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 ,)); +11v1 = map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ())); 12v1 = map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,)); 7v1 -> 2v1; 12v1 -> 5v1; 5v1 -> 8v1; -11v1 -> 12v1; -10v1 -> 11v1; 9v1 -> 10v1; 2v1 -> 9v1; +11v1 -> 12v1; +10v1 -> 11v1; diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@datalog_program.snap new file mode 100644 index 000000000000..c1a9ccc292ac --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@datalog_program.snap @@ -0,0 +1,349 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: "prettyplease::unparse(&wrapped)" +--- +fn main() { + { + #[allow(unused_qualifications)] + { + use hydroflow::{var_expr, var_args}; + let mut df = hydroflow::scheduled::graph::Hydroflow::new(); + df.__assign_meta_graph( + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"flat_map (| row : (_ , _ ,) | row . 1 . into_iter () . map (move | __flattened_element | (:: std :: clone :: Clone :: clone (& row . 0) , __flattened_element ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"source_reader_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + ); + df.__assign_diagnostics("[]"); + let mut sg_1v1_node_7v1_stream = { + #[inline(always)] + fn check_stream< + Stream: hydroflow::futures::stream::Stream + + ::std::marker::Unpin, + Item, + >( + stream: Stream, + ) -> impl hydroflow::futures::stream::Stream< + Item = Item, + > + ::std::marker::Unpin { + stream + } + check_stream(ints1) + }; + let sg_1v1_node_2v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + let sg_1v1_node_5v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + df.add_subgraph_stratified( + "Subgraph GraphSubgraphId(1v1)", + 0, + var_expr!(), + var_expr!(), + false, + move |context, var_args!(), var_args!()| { + let op_7v1 = std::iter::from_fn(|| { + match hydroflow::futures::stream::Stream::poll_next( + ::std::pin::Pin::new(&mut sg_1v1_node_7v1_stream), + &mut std::task::Context::from_waker(&context.waker()), + ) { + std::task::Poll::Ready(maybe) => maybe, + std::task::Poll::Pending => None, + } + }); + let op_7v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_7v1__source_stream__loc_unknown_start_2_26_end_2_46< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_7v1__source_stream__loc_unknown_start_2_26_end_2_46(op_7v1) + }; + let op_2v1 = op_7v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_2v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_2v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_2v1__unique__loc_unknown_start_2_19_end_2_24< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_2v1__unique__loc_unknown_start_2_19_end_2_24(op_2v1) + }; + let op_9v1 = op_2v1 + .flat_map(|row: (_, _)| { + row + .1 + .into_iter() + .map(move |__flattened_element| ( + ::std::clone::Clone::clone(&row.0), + __flattened_element, + )) + }); + let op_9v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_9v1__flat_map__loc_unknown_start_6_12_end_6_40< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_9v1__flat_map__loc_unknown_start_6_12_end_6_40(op_9v1) + }; + let op_10v1 = op_9v1.map(|row: (_, _)| ((row.0, row.1), ())); + let op_10v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_10v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_10v1__map__loc_unknown_start_1_0_end_1_0(op_10v1) + }; + let op_11v1 = op_10v1.map(|(g, a): ((_, _), _)| (g.0, g.1)); + let op_11v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_11v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_11v1__map__loc_unknown_start_1_0_end_1_0(op_11v1) + }; + let op_5v1 = op_11v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_5v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_5v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_5v1__unique__loc_unknown_start_4_20_end_4_26< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_5v1__unique__loc_unknown_start_4_20_end_4_26(op_5v1) + }; + let op_8v1 = hydroflow::pusherator::for_each::ForEach::new(|v| { + result.send(v).unwrap() + }); + let op_8v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_8v1__for_each__loc_unknown_start_4_28_end_4_65< + Item, + Input: hydroflow::pusherator::Pusherator, + >( + input: Input, + ) -> impl hydroflow::pusherator::Pusherator { + #[repr(transparent)] + struct Push< + Item, + Input: hydroflow::pusherator::Pusherator, + > { + inner: Input, + } + impl< + Item, + Input: hydroflow::pusherator::Pusherator, + > hydroflow::pusherator::Pusherator for Push { + type Item = Item; + #[inline(always)] + fn give(&mut self, item: Self::Item) { + self.inner.give(item) + } + } + Push { inner: input } + } + op_8v1__for_each__loc_unknown_start_4_28_end_4_65(op_8v1) + }; + #[inline(always)] + fn pivot_run_sg_1v1< + Pull: ::std::iter::Iterator, + Push: hydroflow::pusherator::Pusherator, + Item, + >(pull: Pull, push: Push) { + hydroflow::pusherator::pivot::Pivot::new(pull, push).run(); + } + pivot_run_sg_1v1(op_5v1, op_8v1); + }, + ); + df + } + } +} diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@surface_graph.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@surface_graph.snap new file mode 100644 index 000000000000..22f0e69de7d2 --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__flatten@surface_graph.snap @@ -0,0 +1,18 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: flat_graph_ref.surface_syntax_string() +--- +2v1 = unique :: < 'tick > (); +5v1 = unique :: < 'tick > (); +7v1 = source_stream (ints1); +8v1 = for_each (| v | result . send (v) . unwrap ()); +9v1 = flat_map (| row : (_ , _ ,) | row . 1 . into_iter () . map (move | __flattened_element | (:: std :: clone :: Clone :: clone (& row . 0) , __flattened_element ,))); +10v1 = map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ())); +11v1 = map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,)); + +7v1 -> 2v1; +11v1 -> 5v1; +5v1 -> 8v1; +2v1 -> 9v1; +10v1 -> 11v1; +9v1 -> 10v1; diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_other@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_other@datalog_program.snap index f8b2fc555aff..e0b6380071d3 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_other@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_other@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (in1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ , _ ,) , (() , ())) | (kv . 0 . 0 , kv . 0 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 , _v . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 , _v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (in1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ , _ ,) , (() , ())) | (kv . 0 . 0 , kv . 0 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 , _v . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 , _v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let mut sg_1v1_node_10v1_stream = { @@ -614,4 +614,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_self@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_self@datalog_program.snap index f12dc0d75c00..9a2532ab3c52 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_self@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__join_with_self@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ , _ ,) , (() , ())) | (kv . 0 . 0 , kv . 0 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 , _v . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 , _v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":3},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":\"input\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ , _ ,) , (() , ())) | (kv . 0 . 0 , kv . 0 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 , _v . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 , _v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":3},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":\"input\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_4v3_send, hoff_4v3_recv) = df @@ -576,4 +576,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program-2.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program-2.snap index 5099c865f409..c88195a2c6ae 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program-2.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program-2.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ , _ , _ , _ ,) | row . 0 == row . 1 && row . 2 == row . 3)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ , _ ,) | ((row . 0 . clone () , row . 0 , row . 2 . clone () , row . 2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 , g . 3 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0_filter\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ , _ , _ , _ ,) | row . 0 == row . 1 && row . 2 == row . 3)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ , _ ,) | ((row . 0 . clone () , row . 0 , row . 2 . clone () , row . 2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 , g . 3 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"source_reader_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let mut sg_1v1_node_7v1_stream = { @@ -344,4 +344,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program.snap index 93907ffa532f..4cf7307edee6 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__local_constraints@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ , _ ,) | row . 0 == row . 1)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 . clone () , row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0_filter\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ , _ ,) | row . 0 == row . 1)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 . clone () , row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"source_reader_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let mut sg_1v1_node_7v1_stream = { @@ -338,4 +338,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@datalog_program.snap new file mode 100644 index 000000000000..7528632083d6 --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@datalog_program.snap @@ -0,0 +1,386 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: "prettyplease::unparse(&wrapped)" +--- +fn main() { + { + #[allow(unused_qualifications)] + { + use hydroflow::{var_expr, var_args}; + let mut df = hydroflow::scheduled::graph::Hydroflow::new(); + df.__assign_meta_graph( + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row_untuple : (_ , _ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 , row_untuple . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row_untuple : (_ , _ , _ ,) | (row_untuple . 0 , row_untuple . 1 , row_untuple . 2 . 0 , row_untuple . 2 . 1 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ , _ ,) | ((row . 0 , row . 1 , row . 2 , row . 3 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 , g . 3 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"source_reader_0\",\"version\":1},{\"value\":\"source_reader_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + ); + df.__assign_diagnostics("[]"); + let mut sg_1v1_node_7v1_stream = { + #[inline(always)] + fn check_stream< + Stream: hydroflow::futures::stream::Stream + + ::std::marker::Unpin, + Item, + >( + stream: Stream, + ) -> impl hydroflow::futures::stream::Stream< + Item = Item, + > + ::std::marker::Unpin { + stream + } + check_stream(ints1) + }; + let sg_1v1_node_2v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + let sg_1v1_node_5v1_uniquedata = df + .add_state( + ::std::cell::RefCell::new( + hydroflow::util::monotonic_map::MonotonicMap::< + _, + hydroflow::rustc_hash::FxHashSet<_>, + >::default(), + ), + ); + df.add_subgraph_stratified( + "Subgraph GraphSubgraphId(1v1)", + 0, + var_expr!(), + var_expr!(), + false, + move |context, var_args!(), var_args!()| { + let op_7v1 = std::iter::from_fn(|| { + match hydroflow::futures::stream::Stream::poll_next( + ::std::pin::Pin::new(&mut sg_1v1_node_7v1_stream), + &mut std::task::Context::from_waker(&context.waker()), + ) { + std::task::Poll::Ready(maybe) => maybe, + std::task::Poll::Pending => None, + } + }); + let op_7v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_7v1__source_stream__loc_unknown_start_2_26_end_2_46< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_7v1__source_stream__loc_unknown_start_2_26_end_2_46(op_7v1) + }; + let op_2v1 = op_7v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_2v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_2v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_2v1__unique__loc_unknown_start_2_19_end_2_24< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_2v1__unique__loc_unknown_start_2_19_end_2_24(op_2v1) + }; + let op_9v1 = op_2v1 + .map(|row_untuple: (_, _)| ( + row_untuple.0.0, + row_untuple.0.1, + row_untuple.1, + )); + let op_9v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_9v1__map__loc_unknown_start_6_12_end_6_55< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_9v1__map__loc_unknown_start_6_12_end_6_55(op_9v1) + }; + let op_10v1 = op_9v1 + .map(|row_untuple: (_, _, _)| ( + row_untuple.0, + row_untuple.1, + row_untuple.2.0, + row_untuple.2.1, + )); + let op_10v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_10v1__map__loc_unknown_start_6_12_end_6_55< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_10v1__map__loc_unknown_start_6_12_end_6_55(op_10v1) + }; + let op_11v1 = op_10v1 + .map(|row: (_, _, _, _)| ((row.0, row.1, row.2, row.3), ())); + let op_11v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_11v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_11v1__map__loc_unknown_start_1_0_end_1_0(op_11v1) + }; + let op_12v1 = op_11v1 + .map(|(g, a): ((_, _, _, _), _)| (g.0, g.1, g.2, g.3)); + let op_12v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_12v1__map__loc_unknown_start_1_0_end_1_0< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_12v1__map__loc_unknown_start_1_0_end_1_0(op_12v1) + }; + let op_5v1 = op_12v1 + .filter(|item| { + let mut borrow = context + .state_ref(sg_1v1_node_5v1_uniquedata) + .borrow_mut(); + let set = borrow + .get_mut_clear(( + context.current_tick(), + context.current_stratum(), + )); + if !set.contains(item) { + set.insert(::std::clone::Clone::clone(item)); + true + } else { + false + } + }); + let op_5v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_5v1__unique__loc_unknown_start_4_20_end_4_26< + Item, + Input: ::std::iter::Iterator, + >(input: Input) -> impl ::std::iter::Iterator { + #[repr(transparent)] + struct Pull< + Item, + Input: ::std::iter::Iterator, + > { + inner: Input, + } + impl< + Item, + Input: ::std::iter::Iterator, + > Iterator for Pull { + type Item = Item; + #[inline(always)] + fn next(&mut self) -> Option { + self.inner.next() + } + #[inline(always)] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } + } + Pull { inner: input } + } + op_5v1__unique__loc_unknown_start_4_20_end_4_26(op_5v1) + }; + let op_8v1 = hydroflow::pusherator::for_each::ForEach::new(|v| { + result.send(v).unwrap() + }); + let op_8v1 = { + #[allow(non_snake_case)] + #[inline(always)] + pub fn op_8v1__for_each__loc_unknown_start_4_28_end_4_65< + Item, + Input: hydroflow::pusherator::Pusherator, + >( + input: Input, + ) -> impl hydroflow::pusherator::Pusherator { + #[repr(transparent)] + struct Push< + Item, + Input: hydroflow::pusherator::Pusherator, + > { + inner: Input, + } + impl< + Item, + Input: hydroflow::pusherator::Pusherator, + > hydroflow::pusherator::Pusherator for Push { + type Item = Item; + #[inline(always)] + fn give(&mut self, item: Self::Item) { + self.inner.give(item) + } + } + Push { inner: input } + } + op_8v1__for_each__loc_unknown_start_4_28_end_4_65(op_8v1) + }; + #[inline(always)] + fn pivot_run_sg_1v1< + Pull: ::std::iter::Iterator, + Push: hydroflow::pusherator::Pusherator, + Item, + >(pull: Pull, push: Push) { + hydroflow::pusherator::pivot::Pivot::new(pull, push).run(); + } + pivot_run_sg_1v1(op_5v1, op_8v1); + }, + ); + df + } + } +} diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@surface_graph.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@surface_graph.snap new file mode 100644 index 000000000000..bf8fe38aadd8 --- /dev/null +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__multi_detuple@surface_graph.snap @@ -0,0 +1,20 @@ +--- +source: hydroflow_datalog_core/src/lib.rs +expression: flat_graph_ref.surface_syntax_string() +--- +2v1 = unique :: < 'tick > (); +5v1 = unique :: < 'tick > (); +7v1 = source_stream (ints1); +8v1 = for_each (| v | result . send (v) . unwrap ()); +9v1 = map (| row_untuple : (_ , _ ,) | (row_untuple . 0 . 0 , row_untuple . 0 . 1 , row_untuple . 1 ,)); +10v1 = map (| row_untuple : (_ , _ , _ ,) | (row_untuple . 0 , row_untuple . 1 , row_untuple . 2 . 0 , row_untuple . 2 . 1 ,)); +11v1 = map (| row : (_ , _ , _ , _ ,) | ((row . 0 , row . 1 , row . 2 , row . 3 ,) , ())); +12v1 = map (| (g , a) : ((_ , _ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 , g . 3 ,)); + +7v1 -> 2v1; +12v1 -> 5v1; +5v1 -> 8v1; +9v1 -> 10v1; +2v1 -> 9v1; +11v1 -> 12v1; +10v1 -> 11v1; diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__persist@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__persist@datalog_program.snap index 91c30b77f995..bd3b9533abc4 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__persist@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__persist@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"difference :: < 'tick , 'static > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"defer_tick ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"difference :: < 'tick , 'static > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"defer_tick ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"difference :: < 'tick , 'static > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"defer_tick ()\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints2)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints3)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result2 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result3 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result4 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'static , 'static , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : (() , ((_ ,) , (_ ,))) | (kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'static , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : (() , ((_ , _ ,) , (_ ,))) | (kv . 1 . 0 . 0 , kv . 1 . 0 . 1 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | (() , (_v . 0 , _v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ ,) | ((row . 0 , row . 1 , row . 2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"anti_join ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ()) | (kv . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (_v . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":34,\"version\":1},{\"idx\":14,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":28,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":25,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":35,\"version\":1},{\"idx\":11,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":22,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":19,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":36,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":13,\"version\":3},{\"idx\":33,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":50,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":3},{\"idx\":31,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":58,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":19,\"version\":3},{\"idx\":10,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":63,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":22,\"version\":3},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":68,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":25,\"version\":3},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":61,\"version\":1},{\"idx\":27,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":28,\"version\":3},{\"idx\":3,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":65,\"version\":1},{\"idx\":30,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":31,\"version\":1},{\"idx\":32,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":30,\"version\":1},{\"idx\":31,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":33,\"version\":1},{\"idx\":16,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":32,\"version\":1},{\"idx\":13,\"version\":3}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":11,\"version\":3},{\"idx\":7,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":37,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":38,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":39,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":40,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":41,\"version\":1},{\"idx\":42,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":43,\"version\":1},{\"idx\":41,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":29,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":44,\"version\":1},{\"idx\":41,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":26,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":45,\"version\":1},{\"idx\":46,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":47,\"version\":1},{\"idx\":45,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":42,\"version\":1},{\"idx\":47,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":48,\"version\":1},{\"idx\":45,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":48,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":14,\"version\":3},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":49,\"version\":1},{\"idx\":50,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":46,\"version\":1},{\"idx\":49,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":51,\"version\":1},{\"idx\":52,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":54,\"version\":1},{\"idx\":51,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":53,\"version\":1},{\"idx\":54,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":23,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":56,\"version\":1},{\"idx\":20,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":55,\"version\":1},{\"idx\":56,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":55,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":3},{\"idx\":65,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":57,\"version\":1},{\"idx\":58,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":52,\"version\":1},{\"idx\":57,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":26,\"version\":3},{\"idx\":44,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":60,\"version\":1},{\"idx\":61,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":59,\"version\":1},{\"idx\":60,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":59,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":20,\"version\":3},{\"idx\":51,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":62,\"version\":1},{\"idx\":63,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":27,\"version\":1},{\"idx\":62,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":29,\"version\":3},{\"idx\":43,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":64,\"version\":1},{\"idx\":17,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":64,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":3},{\"idx\":53,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":67,\"version\":1},{\"idx\":68,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":66,\"version\":1},{\"idx\":67,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":32,\"version\":1},{\"idx\":66,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"2\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[{\"Int\":\"3\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":9,\"version\":1},\"version\":1},{\"value\":{\"idx\":10,\"version\":1},\"version\":1},{\"value\":{\"idx\":11,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":5,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":33,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":43,\"version\":1},{\"idx\":44,\"version\":1},{\"idx\":41,\"version\":1},{\"idx\":42,\"version\":1},{\"idx\":47,\"version\":1},{\"idx\":48,\"version\":1},{\"idx\":45,\"version\":1},{\"idx\":46,\"version\":1},{\"idx\":49,\"version\":1},{\"idx\":50,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":37,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":53,\"version\":1},{\"idx\":54,\"version\":1},{\"idx\":51,\"version\":1},{\"idx\":52,\"version\":1},{\"idx\":57,\"version\":1},{\"idx\":58,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":38,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":1},{\"idx\":59,\"version\":1},{\"idx\":60,\"version\":1},{\"idx\":61,\"version\":1},{\"idx\":27,\"version\":1},{\"idx\":62,\"version\":1},{\"idx\":63,\"version\":1},{\"idx\":21,\"version\":1},{\"idx\":39,\"version\":1},{\"idx\":64,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":65,\"version\":1},{\"idx\":30,\"version\":1},{\"idx\":31,\"version\":1},{\"idx\":32,\"version\":1},{\"idx\":66,\"version\":1},{\"idx\":67,\"version\":1},{\"idx\":68,\"version\":1},{\"idx\":24,\"version\":1},{\"idx\":40,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":55,\"version\":1},{\"idx\":56,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":34,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":35,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":36,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":1,\"version\":1},{\"value\":2,\"version\":1},{\"value\":1,\"version\":1},{\"value\":1,\"version\":1},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":\"ints1\",\"version\":1},{\"value\":\"ints1\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints2_insert\",\"version\":1},{\"value\":\"ints2\",\"version\":1},{\"value\":\"ints2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result4_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"intermediate_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"intermediate_persist_insert\",\"version\":1},{\"value\":\"intermediate_persist\",\"version\":1},{\"value\":\"intermediate_persist\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_1\",\"version\":1},{\"value\":\"join_1\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":40,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":33,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":10,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":10,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":10,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":22,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":22,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":24,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":24,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":24,\"lattice_flow_type\":null},\"version\":3}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"difference :: < 'tick , 'static > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"defer_tick ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"difference :: < 'tick , 'static > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"defer_tick ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"difference :: < 'tick , 'static > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"defer_tick ()\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints2)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints3)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result2 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result3 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result4 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'static , 'static , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : (() , ((_ ,) , (_ ,))) | (kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'static , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : (() , ((_ , _ ,) , (_ ,))) | (kv . 1 . 0 . 0 , kv . 1 . 0 . 1 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | (() , (_v . 0 , _v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (() , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ ,) | ((row . 0 , row . 1 , row . 2 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"anti_join ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ()) | (kv . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | (_v . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"persist ()\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":5},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":34,\"version\":1},{\"idx\":14,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":28,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":25,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":35,\"version\":1},{\"idx\":11,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":22,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":19,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":36,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":5},{\"value\":[{\"idx\":13,\"version\":3},{\"idx\":33,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":50,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":3},{\"idx\":31,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":58,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":19,\"version\":3},{\"idx\":10,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":63,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":22,\"version\":3},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":68,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":25,\"version\":3},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":61,\"version\":1},{\"idx\":27,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":28,\"version\":3},{\"idx\":3,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":65,\"version\":1},{\"idx\":30,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":31,\"version\":1},{\"idx\":32,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":30,\"version\":1},{\"idx\":31,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":33,\"version\":1},{\"idx\":16,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":32,\"version\":1},{\"idx\":13,\"version\":3}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":11,\"version\":3},{\"idx\":7,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":37,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":38,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":39,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":40,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":41,\"version\":1},{\"idx\":42,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":43,\"version\":1},{\"idx\":41,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":29,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":44,\"version\":1},{\"idx\":41,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":26,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":45,\"version\":1},{\"idx\":46,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":47,\"version\":1},{\"idx\":45,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":42,\"version\":1},{\"idx\":47,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":48,\"version\":1},{\"idx\":45,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":48,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":14,\"version\":3},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":49,\"version\":1},{\"idx\":50,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":46,\"version\":1},{\"idx\":49,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":51,\"version\":1},{\"idx\":52,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":54,\"version\":1},{\"idx\":51,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":53,\"version\":1},{\"idx\":54,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":23,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":56,\"version\":1},{\"idx\":20,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":55,\"version\":1},{\"idx\":56,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":55,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":3},{\"idx\":65,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":57,\"version\":1},{\"idx\":58,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":52,\"version\":1},{\"idx\":57,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":26,\"version\":3},{\"idx\":44,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":60,\"version\":1},{\"idx\":61,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":59,\"version\":1},{\"idx\":60,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":59,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":20,\"version\":3},{\"idx\":51,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":62,\"version\":1},{\"idx\":63,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":27,\"version\":1},{\"idx\":62,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":29,\"version\":3},{\"idx\":43,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":64,\"version\":1},{\"idx\":17,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":1},{\"idx\":64,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":3},{\"idx\":53,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":67,\"version\":1},{\"idx\":68,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":66,\"version\":1},{\"idx\":67,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":32,\"version\":1},{\"idx\":66,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":5},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"pos\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"2\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Path\":\"neg\"}],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[{\"Int\":\"3\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":9,\"version\":1},\"version\":1},{\"value\":{\"idx\":10,\"version\":1},\"version\":1},{\"value\":{\"idx\":11,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":8,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1},{\"value\":{\"idx\":7,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":5,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":33,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":43,\"version\":1},{\"idx\":44,\"version\":1},{\"idx\":41,\"version\":1},{\"idx\":42,\"version\":1},{\"idx\":47,\"version\":1},{\"idx\":48,\"version\":1},{\"idx\":45,\"version\":1},{\"idx\":46,\"version\":1},{\"idx\":49,\"version\":1},{\"idx\":50,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":37,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":53,\"version\":1},{\"idx\":54,\"version\":1},{\"idx\":51,\"version\":1},{\"idx\":52,\"version\":1},{\"idx\":57,\"version\":1},{\"idx\":58,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":38,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":1},{\"idx\":59,\"version\":1},{\"idx\":60,\"version\":1},{\"idx\":61,\"version\":1},{\"idx\":27,\"version\":1},{\"idx\":62,\"version\":1},{\"idx\":63,\"version\":1},{\"idx\":21,\"version\":1},{\"idx\":39,\"version\":1},{\"idx\":64,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":65,\"version\":1},{\"idx\":30,\"version\":1},{\"idx\":31,\"version\":1},{\"idx\":32,\"version\":1},{\"idx\":66,\"version\":1},{\"idx\":67,\"version\":1},{\"idx\":68,\"version\":1},{\"idx\":24,\"version\":1},{\"idx\":40,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":55,\"version\":1},{\"idx\":56,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":34,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":35,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":36,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":1,\"version\":1},{\"value\":2,\"version\":1},{\"value\":1,\"version\":1},{\"value\":1,\"version\":1},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":\"ints1\",\"version\":1},{\"value\":\"ints1\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints2_insert\",\"version\":1},{\"value\":\"ints2\",\"version\":1},{\"value\":\"ints2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result4_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"intermediate_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"intermediate_persist_insert\",\"version\":1},{\"value\":\"intermediate_persist\",\"version\":1},{\"value\":\"intermediate_persist\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_4\",\"version\":1},{\"value\":\"join_4\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_7\",\"version\":1},{\"value\":\"join_7\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":40,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":33,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":10,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":10,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":10,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":22,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":22,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":24,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":24,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":24,\"lattice_flow_type\":null},\"version\":3}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_6v3_send, hoff_6v3_recv) = df @@ -2941,4 +2941,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__simple_filter@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__simple_filter@datalog_program.snap index 6ee3b4ea2664..7641c1e9eb16 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__simple_filter@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__simple_filter@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ , _ ,) | row . 0 > row . 1 && row . 1 == row . 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_0_filter\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"filter (| row : & (_ , _ ,) | row . 0 > row . 1 && row . 1 == row . 0)\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 0 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ ,) , _) | (g . 0 , g . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"predicate_1_filter\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let mut sg_1v1_node_7v1_stream = { @@ -339,4 +339,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__single_column_program@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__single_column_program@datalog_program.snap index 8be43f94eb2c..4fd794477977 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__single_column_program@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__single_column_program@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (in1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , (() , ())) | (kv . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (in1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , (() , ())) | (kv . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let mut sg_1v1_node_10v1_stream = { @@ -614,4 +614,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__transitive_closure@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__transitive_closure@datalog_program.snap index 19f4d11d703c..abda8cc9e805 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__transitive_closure@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__transitive_closure@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"union ()\"},\"version\":1},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (edges)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (seed_reachable)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | reachable . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , (() , (_ ,))) | (kv . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":17,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":7,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":7,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"edges_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"seed_reachable_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"reachable_insert\",\"version\":1},{\"value\":\"reachable_insert\",\"version\":1},{\"value\":\"reachable\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":1,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"union ()\"},\"version\":1},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (edges)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (seed_reachable)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | reachable . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , (() , (_ ,))) | (kv . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | ((row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":17,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":7,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":7,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":16,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":16,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":7,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":12,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"edges_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"seed_reachable_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"reachable_insert\",\"version\":1},{\"value\":\"reachable_insert\",\"version\":1},{\"value\":\"reachable\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_3\",\"version\":1},{\"value\":\"join_3\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":1,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_6v3_send, hoff_6v3_recv) = df @@ -771,4 +771,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__triple_relation_join@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__triple_relation_join@datalog_program.snap index 30c46fb44cf3..356d221790ee 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__triple_relation_join@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__triple_relation_join@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (in1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in2)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in3)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 ,) , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ , _ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 0 . 1 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ , _ ,) | ((_v . 2 ,) , (_v . 1 , _v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ , _ ,) | ((row . 3 , row . 0 , row . 2 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 , g . 3 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1},{\"idx\":24,\"version\":1},{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_1\",\"version\":1},{\"value\":\"join_1\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"source_stream (in1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in2)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (in3)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 ,) , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ , _ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 0 . 1 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ , _ ,) | ((_v . 2 ,) , (_v . 1 , _v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ , _ ,) | ((row . 3 , row . 0 , row . 2 , row . 1 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ , _ , _ , _ ,) , _) | (g . 0 , g . 1 , g . 2 , g . 3 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":19,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":20,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":15,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":23,\"version\":1},{\"idx\":24,\"version\":1},{\"idx\":21,\"version\":1},{\"idx\":22,\"version\":1},{\"idx\":25,\"version\":1},{\"idx\":26,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in1_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"in3_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_4\",\"version\":1},{\"value\":\"join_4\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":2,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let mut sg_1v1_node_13v1_stream = { @@ -932,4 +932,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_fields@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_fields@datalog_program.snap index 4277ca1668b3..b823c522fbb4 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_fields@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_fields@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 ,) , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":3},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":\"input\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":2},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (input)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | out . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , (_ ,))) | (kv . 0 . 0 , kv . 1 . 0 . 0 , kv . 1 . 1 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 1 ,) , (_v . 0 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ , _ ,) | ((row . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : ((_ ,) , _) | (g . 0 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":6,\"version\":3},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":null,\"version\":2},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":6,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":4,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":3},{\"idx\":12,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":7,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":12,\"version\":1},{\"idx\":9,\"version\":1},{\"idx\":10,\"version\":1},{\"idx\":13,\"version\":1},{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"input_insert\",\"version\":1},{\"value\":\"input\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"out_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":5,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_4v3_send, hoff_4v3_recv) = df @@ -577,4 +577,3 @@ fn main() { } } } - diff --git a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_join_count@datalog_program.snap b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_join_count@datalog_program.snap index e608e101cafd..9497bf72af1b 100644 --- a/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_join_count@datalog_program.snap +++ b/hydroflow_datalog_core/src/snapshots/hydroflow_datalog_core__tests__wildcard_join_count@datalog_program.snap @@ -9,7 +9,7 @@ fn main() { use hydroflow::{var_expr, var_args}; let mut df = hydroflow::scheduled::graph::Hydroflow::new(); df.__assign_meta_graph( - "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result2 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , ())) | (kv . 0 . 0 , kv . 1 . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | (() , (() ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"fold_keyed :: < 'tick , () , (Option < _ > ,) > (| | (None ,) , | old : & mut (Option < _ > ,) , val : (_ ,) | { old . 0 = if let Some (prev) = old . 0 . take () { Some (prev + 1) } else { Some (1) } ; })\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : (() , _) | (a . 0 . unwrap () ,))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , ())) | (kv . 0 . 0 , kv . 1 . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | (() , ((row . 0) ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"fold_keyed :: < 'tick , () , (Option < _ > ,) > (| | (None ,) , | old : & mut (Option < _ > ,) , val : (_ ,) | { old . 0 = if let Some (prev) = old . 0 . take () { Some ({ let prev : (hydroflow :: rustc_hash :: FxHashSet < _ > , _) = prev ; let mut set : hydroflow :: rustc_hash :: FxHashSet < _ > = prev . 0 ; if set . insert (val . 0) { (set , prev . 1 + 1) } else { (set , prev . 1) } }) } else { Some ({ let mut set = hydroflow :: rustc_hash :: FxHashSet :: < _ > :: default () ; set . insert (val . 0) ; (set , 1) }) } ; })\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : (() , _) | (a . 0 . unwrap () . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":6,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":3},{\"idx\":20,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":30,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":3},{\"idx\":19,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":1,\"version\":3},{\"idx\":29,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":3},{\"idx\":27,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":12,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":6,\"version\":1},{\"idx\":9,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":7,\"version\":3},{\"idx\":26,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":23,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":10,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":25,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":7,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":27,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":6,\"version\":1},{\"idx\":4,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":10,\"version\":3},{\"idx\":22,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":29,\"version\":1},{\"idx\":30,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":28,\"version\":1},{\"idx\":1,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":25,\"version\":1},{\"idx\":28,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":6,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":23,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":29,\"version\":1},{\"idx\":30,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":27,\"version\":1},{\"idx\":24,\"version\":1},{\"idx\":25,\"version\":1},{\"idx\":28,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":1,\"version\":1},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":\"ints1\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"ints2_insert\",\"version\":1},{\"value\":\"ints2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_0\",\"version\":1},{\"value\":\"join_0\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_1\",\"version\":1},{\"value\":\"join_1\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", + "{\"nodes\":[{\"value\":null,\"version\":0},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Operator\":\"tee ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"unique :: < 'tick > ()\"},\"version\":1},{\"value\":{\"Handoff\":{}},\"version\":3},{\"value\":{\"Operator\":\"source_stream (ints1)\"},\"version\":1},{\"value\":{\"Operator\":\"source_stream (ints2)\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"for_each (| v | result2 . send (v) . unwrap ())\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , ())) | (kv . 0 . 0 , kv . 1 . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | (() , (() ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"fold_keyed :: < 'tick , () , (Option < _ > ,) > (| | (None ,) , | old : & mut (Option < _ > ,) , val : (_ ,) | { old . 0 = if let Some (prev) = old . 0 . take () { Some (prev + 1) } else { Some (1) } ; })\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : (() , _) | (a . 0 . unwrap () ,))\"},\"version\":1},{\"value\":{\"Operator\":\"join :: < 'tick , 'tick , hydroflow :: compiled :: pull :: HalfMultisetJoinState > ()\"},\"version\":1},{\"value\":{\"Operator\":\"map (| kv : ((_ ,) , ((_ ,) , ())) | (kv . 0 . 0 , kv . 1 . 0 . 0 ,))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ , _ ,) | ((_v . 0 ,) , (_v . 1 ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| _v : (_ ,) | ((_v . 0 ,) , ()))\"},\"version\":1},{\"value\":{\"Operator\":\"map (| row : (_ , _ ,) | (() , ((row . 0) ,)))\"},\"version\":1},{\"value\":{\"Operator\":\"fold_keyed :: < 'tick , () , (Option < _ > ,) > (| | (None ,) , | old : & mut (Option < _ > ,) , val : (_ ,) | { old . 0 = if let Some (prev) = old . 0 . take () { Some ({ let prev : (hydroflow :: rustc_hash :: FxHashSet < _ > , _) = prev ; let mut set : hydroflow :: rustc_hash :: FxHashSet < _ > = prev . 0 ; if set . insert (val . 0) { (set , prev . 1 + 1) } else { (set , prev . 1) } }) } else { Some ({ let mut set = hydroflow :: rustc_hash :: FxHashSet :: < _ > :: default () ; set . insert (val . 0) ; (set , 1) }) } ; })\"},\"version\":1},{\"value\":{\"Operator\":\"map (| (g , a) : (() , _) | (a . 0 . unwrap () . 1 ,))\"},\"version\":1}],\"edge_types\":[{\"value\":null,\"version\":0},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1},{\"value\":\"Value\",\"version\":3},{\"value\":\"Value\",\"version\":1}],\"graph\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":5,\"version\":1},{\"idx\":6,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":23,\"version\":1},{\"idx\":8,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":9,\"version\":3},{\"idx\":20,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":30,\"version\":1},{\"idx\":11,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":12,\"version\":3},{\"idx\":19,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":1,\"version\":3},{\"idx\":29,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":4,\"version\":3},{\"idx\":27,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":8,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":12,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":6,\"version\":1},{\"idx\":9,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":7,\"version\":3},{\"idx\":26,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":23,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":21,\"version\":1},{\"idx\":10,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":18,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":24,\"version\":1},{\"idx\":25,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":3,\"version\":1},{\"idx\":7,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":27,\"version\":1},{\"idx\":24,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":6,\"version\":1},{\"idx\":4,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":10,\"version\":3},{\"idx\":22,\"version\":1}],\"version\":3},{\"value\":[{\"idx\":29,\"version\":1},{\"idx\":30,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":28,\"version\":1},{\"idx\":1,\"version\":3}],\"version\":3},{\"value\":[{\"idx\":25,\"version\":1},{\"idx\":28,\"version\":1}],\"version\":1}],\"ports\":[{\"value\":null,\"version\":0},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"0\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",{\"Int\":\"0\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",{\"Int\":\"1\"}],\"version\":1},{\"value\":[{\"Int\":\"1\"},\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1},{\"value\":[\"Elided\",\"Elided\"],\"version\":3},{\"value\":[\"Elided\",\"Elided\"],\"version\":1}],\"node_subgraph\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":null,\"version\":0},{\"value\":{\"idx\":1,\"version\":1},\"version\":1},{\"value\":{\"idx\":2,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":5,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":3,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":6,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1},{\"value\":{\"idx\":4,\"version\":1},\"version\":1}],\"subgraph_nodes\":[{\"value\":null,\"version\":0},{\"value\":[{\"idx\":13,\"version\":1},{\"idx\":2,\"version\":1},{\"idx\":3,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":14,\"version\":1},{\"idx\":5,\"version\":1},{\"idx\":6,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":22,\"version\":1},{\"idx\":23,\"version\":1},{\"idx\":8,\"version\":1},{\"idx\":15,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":29,\"version\":1},{\"idx\":30,\"version\":1},{\"idx\":11,\"version\":1},{\"idx\":16,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":19,\"version\":1},{\"idx\":20,\"version\":1},{\"idx\":17,\"version\":1},{\"idx\":18,\"version\":1},{\"idx\":21,\"version\":1}],\"version\":1},{\"value\":[{\"idx\":26,\"version\":1},{\"idx\":27,\"version\":1},{\"idx\":24,\"version\":1},{\"idx\":25,\"version\":1},{\"idx\":28,\"version\":1}],\"version\":1}],\"subgraph_stratum\":[{\"value\":null,\"version\":0},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1},{\"value\":1,\"version\":1},{\"value\":1,\"version\":1},{\"value\":0,\"version\":1},{\"value\":0,\"version\":1}],\"node_singleton_references\":[{\"value\":null,\"version\":0},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1},{\"value\":[],\"version\":1}],\"node_varnames\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"ints1_insert\",\"version\":1},{\"value\":\"ints1\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"ints2_insert\",\"version\":1},{\"value\":\"ints2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":\"result_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"result2_insert\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_2\",\"version\":1},{\"value\":\"join_2\",\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":\"join_5\",\"version\":1},{\"value\":\"join_5\",\"version\":1}],\"flow_props\":[{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":1},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":14,\"lattice_flow_type\":null},\"version\":3},{\"value\":null,\"version\":0},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":3},{\"value\":{\"star_ord\":6,\"lattice_flow_type\":null},\"version\":1}],\"subgraph_laziness\":[{\"value\":null,\"version\":0}]}", ); df.__assign_diagnostics("[]"); let (hoff_1v3_send, hoff_1v3_recv) = df @@ -1390,4 +1390,3 @@ fn main() { } } } -