From 4c7db75db373023a028a8d653f2247bf8e771783 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Mon, 23 Feb 2026 22:08:22 -0500 Subject: [PATCH 1/6] :WIP --- Cargo.toml | 20 +- advent_of_code_2017/src/bin/day_01.rs | 12 +- advent_of_code_2017/src/bin/day_05.rs | 22 +- advent_of_code_2017/src/bin/day_06.rs | 14 +- advent_of_code_2017/src/bin/day_07.rs | 8 +- advent_of_code_2017/src/bin/day_08.rs | 2 +- advent_of_code_2017/src/bin/day_09.rs | 12 +- advent_of_code_2017/src/bin/day_12.rs | 12 +- advent_of_code_2017/src/bin/day_13.rs | 2 +- differential-dataflow/examples/arrange.rs | 12 +- differential-dataflow/examples/bfs.rs | 10 +- differential-dataflow/examples/columnar.rs | 12 +- differential-dataflow/examples/dynamic.rs | 11 +- differential-dataflow/examples/graspan.rs | 22 +- differential-dataflow/examples/interpreted.rs | 16 +- .../examples/itembased_cf.rs | 11 +- .../examples/iterate_container.rs | 12 +- differential-dataflow/examples/monoid-bfs.rs | 11 +- .../examples/multitemporal.rs | 2 +- differential-dataflow/examples/pagerank.rs | 22 +- differential-dataflow/examples/progress.rs | 21 +- differential-dataflow/examples/projekt.rs | 17 +- differential-dataflow/examples/spines.rs | 6 +- .../examples/stackoverflow.rs | 10 +- .../src/algorithms/graphs/bfs.rs | 12 +- .../src/algorithms/graphs/bijkstra.rs | 52 +-- .../src/algorithms/graphs/propagate.rs | 22 +- .../src/algorithms/graphs/scc.rs | 27 +- .../src/algorithms/graphs/sequential.rs | 28 +- .../src/algorithms/identifiers.rs | 16 +- .../src/algorithms/prefix_sum.rs | 33 +- differential-dataflow/src/capture.rs | 28 +- differential-dataflow/src/collection.rs | 230 +++++------ differential-dataflow/src/dynamic/mod.rs | 10 +- differential-dataflow/src/input.rs | 6 +- differential-dataflow/src/lib.rs | 8 +- .../src/operators/arrange/arrangement.rs | 48 +-- .../src/operators/arrange/upsert.rs | 15 +- differential-dataflow/src/operators/count.rs | 12 +- .../src/operators/iterate.rs | 46 ++- differential-dataflow/src/operators/join.rs | 11 +- differential-dataflow/src/operators/reduce.rs | 15 +- .../src/operators/threshold.rs | 16 +- .../trace/implementations/merge_batcher.rs | 4 +- differential-dataflow/tests/bfs.rs | 10 +- differential-dataflow/tests/import.rs | 2 +- differential-dataflow/tests/join.rs | 14 +- differential-dataflow/tests/reduce.rs | 2 +- differential-dataflow/tests/scc.rs | 30 +- dogsdogsdogs/README.md | 4 +- dogsdogsdogs/examples/delta_query.rs | 46 +-- dogsdogsdogs/examples/delta_query2.rs | 16 +- dogsdogsdogs/examples/delta_query_wcoj.rs | 12 +- dogsdogsdogs/examples/dogsdogsdogs.rs | 18 +- dogsdogsdogs/examples/ngo.rs | 30 +- dogsdogsdogs/src/calculus.rs | 10 +- dogsdogsdogs/src/lib.rs | 41 +- dogsdogsdogs/src/operators/count.rs | 2 +- dogsdogsdogs/src/operators/half_join.rs | 15 +- dogsdogsdogs/src/operators/lookup_map.rs | 6 +- dogsdogsdogs/src/operators/propose.rs | 4 +- dogsdogsdogs/src/operators/validate.rs | 2 +- doop/src/main.rs | 366 +++++++++--------- experiments/src/bin/attend.rs | 2 +- experiments/src/bin/deals-interactive.rs | 26 +- experiments/src/bin/deals.rs | 16 +- experiments/src/bin/graphs-interactive-alt.rs | 72 ++-- .../src/bin/graphs-interactive-neu-zwei.rs | 54 +-- experiments/src/bin/graphs-interactive-neu.rs | 76 ++-- experiments/src/bin/graphs-interactive.rs | 54 +-- experiments/src/bin/graphs-static.rs | 26 +- experiments/src/bin/graphs.rs | 18 +- experiments/src/bin/graspan-interactive.rs | 4 +- experiments/src/bin/graspan1.rs | 6 +- experiments/src/bin/graspan2.rs | 50 +-- experiments/src/bin/multitemporal.rs | 16 +- experiments/src/bin/ysb.rs | 2 +- interactive/src/plan/join.rs | 2 +- mdbook/src/chapter_0/chapter_0_1.md | 3 +- mdbook/src/chapter_0/chapter_0_3.md | 3 +- mdbook/src/chapter_1/chapter_1_2.md | 6 +- mdbook/src/chapter_1/chapter_1_3.md | 6 +- mdbook/src/chapter_2/chapter_2_1.md | 7 +- mdbook/src/chapter_2/chapter_2_2.md | 2 +- mdbook/src/chapter_2/chapter_2_3.md | 5 +- mdbook/src/chapter_2/chapter_2_4.md | 10 +- mdbook/src/chapter_2/chapter_2_5.md | 5 +- mdbook/src/chapter_2/chapter_2_6.md | 2 +- mdbook/src/chapter_2/chapter_2_7.md | 18 +- mdbook/src/chapter_3/chapter_3_2.md | 3 +- mdbook/src/chapter_4/chapter_4_1.md | 4 +- mdbook/src/chapter_5/chapter_5_1.md | 8 +- mdbook/src/chapter_5/chapter_5_2.md | 26 +- mdbook/src/chapter_5/chapter_5_3.md | 2 +- mdbook/src/chapter_5/chapter_5_4.md | 6 +- mdbook/src/chapter_a/chapter_a_3.md | 3 +- server/dataflows/neighborhood/src/lib.rs | 6 +- server/dataflows/reachability/src/lib.rs | 4 +- tpchlike/src/bin/arrange.rs | 2 +- tpchlike/src/bin/just-arrange.rs | 2 +- tpchlike/src/lib.rs | 16 +- tpchlike/src/queries/query02.rs | 24 +- tpchlike/src/queries/query03.rs | 8 +- tpchlike/src/queries/query04.rs | 4 +- tpchlike/src/queries/query05.rs | 22 +- tpchlike/src/queries/query07.rs | 20 +- tpchlike/src/queries/query08.rs | 28 +- tpchlike/src/queries/query09.rs | 20 +- tpchlike/src/queries/query10.rs | 12 +- tpchlike/src/queries/query11.rs | 8 +- tpchlike/src/queries/query12.rs | 4 +- tpchlike/src/queries/query13.rs | 4 +- tpchlike/src/queries/query14.rs | 4 +- tpchlike/src/queries/query15.rs | 4 +- tpchlike/src/queries/query16.rs | 8 +- tpchlike/src/queries/query17.rs | 4 +- tpchlike/src/queries/query18.rs | 8 +- tpchlike/src/queries/query19.rs | 12 +- tpchlike/src/queries/query20.rs | 18 +- tpchlike/src/queries/query21.rs | 16 +- tpchlike/src/queries/query22.rs | 8 +- 121 files changed, 1219 insertions(+), 1158 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 60fe1ebe7..3bfb487a4 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -3,15 +3,15 @@ members = [ "differential-dataflow", # "advent_of_code_2017", "dogsdogsdogs", - "experiments", - "interactive", - "server", - "server/dataflows/degr_dist", - "server/dataflows/neighborhood", - "server/dataflows/random_graph", - "server/dataflows/reachability", + #"experiments", + #"interactive", + #"server", + #"server/dataflows/degr_dist", + #"server/dataflows/neighborhood", + #"server/dataflows/random_graph", + #"server/dataflows/reachability", #"tpchlike", - "doop", + #"doop", "mdbook", ] resolver = "2" @@ -21,9 +21,9 @@ edition = "2021" [workspace.dependencies] differential-dataflow = { path = "differential-dataflow", default-features = false, version = "0.19.1" } -timely = { version = "0.26", default-features = false } +#timely = { version = "0.26", default-features = false } columnar = { version = "0.11", default-features = false } -#timely = { path = "../timely-dataflow/timely/", default-features = false } +timely = { path = "../timely-dataflow/timely/", default-features = false } [profile.release] opt-level = 3 diff --git a/advent_of_code_2017/src/bin/day_01.rs b/advent_of_code_2017/src/bin/day_01.rs index 96250b374..c6159872b 100644 --- a/advent_of_code_2017/src/bin/day_01.rs +++ b/advent_of_code_2017/src/bin/day_01.rs @@ -16,7 +16,7 @@ fn main() { let index = worker.index(); let peers = worker.peers(); - let worker_input = + let worker_input = input .iter() .map(|digit| digit - b'0') @@ -28,17 +28,17 @@ fn main() { let digits = scope.new_collection_from(worker_input).1; - // The two parts ask to line up elements with the next one in the sequence (part 1) and - // the one half way around the sequence (part 2). To find matches, we will shift the - // associated position field while keeping the part identifier as a field (`1` or `2` + // The two parts ask to line up elements with the next one in the sequence (part 1) and + // the one half way around the sequence (part 2). To find matches, we will shift the + // associated position field while keeping the part identifier as a field (`1` or `2` // respectively). We then restrict both by the original `digits` to find matches. let part1 = digits.map(move |(digit, position)| ((digit, (position + 1) % length), 1)); let part2 = digits.map(move |(digit, position)| ((digit, (position + length/2) % length), 2)); part1 - .concat(&part2) // merge collections. - .semijoin(&digits) // restrict to matches. + .concat(part2) // merge collections. + .semijoin(digits) // restrict to matches. .explode(|((digit, _pos), part)| Some((part, digit as isize))) // `part` with weight `digit`. .consolidate() // consolidate weights by `part`. .inspect(|elt| println!("part {} accumulation: {:?}", elt.0, elt.2)); // check out answers. diff --git a/advent_of_code_2017/src/bin/day_05.rs b/advent_of_code_2017/src/bin/day_05.rs index 0c4ec4343..25be4f410 100644 --- a/advent_of_code_2017/src/bin/day_05.rs +++ b/advent_of_code_2017/src/bin/day_05.rs @@ -11,7 +11,7 @@ use differential_dataflow::operators::iterate::Variable; fn main() { - let input = + let input = "2 0 0 @@ -1051,7 +1051,7 @@ fn main() { let index = worker.index(); let peers = worker.peers(); - let worker_input = + let worker_input = input .split('\n') .map(|phrase| phrase.parse::().unwrap()) @@ -1070,27 +1070,27 @@ fn main() { let address = Variable::from(address.enter(nested)); // (addr, jump, steps) - let instruction = program.join(&address); + let instruction = program.join(address); fn part1(jump: isize) -> isize { jump+1 } // fn part2(jump: isize) -> isize { if jump >= 3 { jump-1 } else { jump+1 } } - let new_program = + let new_program = instruction .map(move |(addr, jump, _)| (addr, part1(jump))) - .concat(&instruction.map(|(addr, jump, _)| (addr, jump)).negate()) - .concat(&program) + .concat(instruction.map(|(addr, jump, _)| (addr, jump)).negate()) + .concat(program) .consolidate(); - let new_address = + let new_address = instruction .map(|(addr, jump, steps)| (addr + jump, steps + 1)) - .concat(&instruction.map(|(addr, _, steps)| (addr, steps)).negate()) - .concat(&address) + .concat(instruction.map(|(addr, _, steps)| (addr, steps)).negate()) + .concat(address) .consolidate(); - program.set(&new_program); - address.set(&new_address) + program.set(new_program); + address.set(new_address) .leave() }) .consolidate() diff --git a/advent_of_code_2017/src/bin/day_06.rs b/advent_of_code_2017/src/bin/day_06.rs index 81628341f..a7686ef42 100644 --- a/advent_of_code_2017/src/bin/day_06.rs +++ b/advent_of_code_2017/src/bin/day_06.rs @@ -12,7 +12,7 @@ fn main() { timely::execute_from_args(std::env::args(), move |worker| { - let worker_input = + let worker_input = input .split_whitespace() .map(|phrase| phrase.parse::().unwrap()) @@ -24,7 +24,7 @@ fn main() { let stable = banks.iterate(|iter| iter.map_in_place(|banks| recycle(banks)) - .concat(&banks.enter(&iter.scope())) + .concat(banks.enter(&iter.scope())) .distinct() ); @@ -36,14 +36,14 @@ fn main() { // determine the repeated state by stepping all states and subtracting. let loop_point = stable .map_in_place(|banks| recycle(banks)) - .concat(&stable.negate()) - .concat(&banks); + .concat(stable.negate()) + .concat(banks); // restart iteration from known repeated element. - loop_point + loop_point .iterate(|iter| iter.map_in_place(|banks| recycle(banks)) - .concat(&loop_point.enter(&iter.scope())) + .concat(loop_point.enter(&iter.scope())) .distinct() ) .map(|_| ((),())) @@ -67,5 +67,5 @@ fn recycle(banks: &mut [u8]) { let banks_len = banks.len(); for i in 1 .. (redistribute + 1) { banks[(max_idx + i) % banks_len] += 1; - } + } } \ No newline at end of file diff --git a/advent_of_code_2017/src/bin/day_07.rs b/advent_of_code_2017/src/bin/day_07.rs index be4146b32..42a84b854 100644 --- a/advent_of_code_2017/src/bin/day_07.rs +++ b/advent_of_code_2017/src/bin/day_07.rs @@ -1095,7 +1095,7 @@ tvhftq (35)"; input.flat_map(|(_,_,links)| links) .negate() - .concat(&input.map(|(name,_,_)| name)) + .concat(input.map(|(name,_,_)| name)) .consolidate() .inspect(|line| println!("part1: {:?}", line.0)); @@ -1105,13 +1105,13 @@ tvhftq (35)"; let total_weights: VecCollection<_,String> = weights .iterate(|inner| { parents.enter(&inner.scope()) - .semijoin(&inner) + .semijoin(inner) .map(|(_, parent)| parent) - .concat(&weights.enter(&inner.scope())) + .concat(weights.enter(&inner.scope())) }); parents - .semijoin(&total_weights) + .semijoin(total_weights) .map(|(link,name)| (name,link)) .group(|key, input, output| { if input.len() > 0 { diff --git a/advent_of_code_2017/src/bin/day_08.rs b/advent_of_code_2017/src/bin/day_08.rs index 64627fba6..62dc6a12b 100644 --- a/advent_of_code_2017/src/bin/day_08.rs +++ b/advent_of_code_2017/src/bin/day_08.rs @@ -1111,7 +1111,7 @@ wui inc -120 if i > -2038"; valid .prefix_sum_at(edits.map(|(key,_)| key), 0, |_k,x,y| *x + *y) - .join(&edits) + .join(edits) .filter(|&(_, sum, ((ref src_cmp, src_val), _))| match src_cmp.as_str() { ">" => sum > src_val, ">=" => sum >= src_val, diff --git a/advent_of_code_2017/src/bin/day_09.rs b/advent_of_code_2017/src/bin/day_09.rs index 8ec779552..9e700e44f 100644 --- a/advent_of_code_2017/src/bin/day_09.rs +++ b/advent_of_code_2017/src/bin/day_09.rs @@ -64,7 +64,7 @@ fn main() { let values = pp_broadcast(ranges, 0, [0, 1, 2, 3], |state, trans| trans[*state]); // line up (position, symbol, state). - let symbols_state = input.join(&values); + let symbols_state = input.join(values); // restrict the positions down to those that are neither '!' nor themselves cancelled. let active = symbols_state.filter(|&(_, symbol, state)| symbol != '!' && (state == 0 || state == 1)); @@ -78,7 +78,7 @@ fn main() { parens .filter(|&(_pos, symbol, _)| symbol == '}') .map(|(pos, symbol, _)| (pos, symbol)) - .join(&values) + .join(values) .explode(|(_pos, _sym, sum)| Some(((), sum))) .consolidate() .inspect(|x| println!("part1: {:?}", x.2)); @@ -120,7 +120,7 @@ where if input.len() > 1 { result = combine(result, &(input[1].0).1); } output.push((result, 1)); }) - .concat(&unit_ranges.enter(&ranges.scope())) + .concat(unit_ranges.enter(&ranges.scope())) ) } @@ -141,9 +141,9 @@ where let zero_ranges = ranges .map(move |((pos, log),_)| ((pos, if log > 0 { log - 1 } else { 0 }), zero.clone())) - .antijoin(&ranges.map(|((pos, log),_)| (pos, log))); + .antijoin(ranges.map(|((pos, log),_)| (pos, log))); - let aggregates = ranges.concat(&zero_ranges); + let aggregates = ranges.concat(zero_ranges); let init_state = Some(((0, seed), Default::default(), 1)) @@ -157,7 +157,7 @@ where .enter(&state.scope()) .map(|((pos, log), data)| (pos, (log, data))) .join_map(state, move |&pos, &(log, ref data), state| (pos + (1 << log), combine(state, data))) - .concat(&init_state.enter(&state.scope())) + .concat(init_state.enter(&state.scope())) .distinct() }) .consolidate() diff --git a/advent_of_code_2017/src/bin/day_12.rs b/advent_of_code_2017/src/bin/day_12.rs index 9648ca96f..1db76553e 100644 --- a/advent_of_code_2017/src/bin/day_12.rs +++ b/advent_of_code_2017/src/bin/day_12.rs @@ -2014,7 +2014,7 @@ fn main() { let index = worker.index(); let peers = worker.peers(); - let worker_input = + let worker_input = input .lines() .enumerate() @@ -2032,14 +2032,14 @@ fn main() { let edges = scope.new_collection_from(worker_input).1; let nodes = edges.map(|(src, _tgt)| (src, src)).distinct(); - let labels = + let labels = nodes .iterate(|label| { let edges = edges.enter(&label.scope()); let nodes = nodes.enter(&label.scope()); label - .join_map(&edges, |_src, &lbl, &tgt| (tgt, lbl)) - .concat(&nodes) + .join_map(edges, |_src, &lbl, &tgt| (tgt, lbl)) + .concat(nodes) .group(|_, input, output| output.push((*input[0].0, 1))) }) .map(|(_src, lbl)| lbl); @@ -2048,13 +2048,13 @@ fn main() { .filter(|&lbl| lbl == 0) .consolidate() .inspect(|x| println!("part1: {:?}", x.2)); - + labels .distinct() .map(|_| ()) .consolidate() .inspect(|x| println!("part2: {:?}", x.2)); - + }); }).unwrap(); diff --git a/advent_of_code_2017/src/bin/day_13.rs b/advent_of_code_2017/src/bin/day_13.rs index 6c40bd969..54251b135 100644 --- a/advent_of_code_2017/src/bin/day_13.rs +++ b/advent_of_code_2017/src/bin/day_13.rs @@ -84,7 +84,7 @@ fn main() { data.flat_map(move |(pos, wid)| (0..limit).filter(move |x| (x + pos) % (2 * (wid-1)) == 0)) .distinct() .negate() - .concat(&scope.new_collection_from(0 .. limit).1) + .concat(scope.new_collection_from(0 .. limit).1) .consolidate() .inspect(|x| println!("part2: {:?}", x.0)); }); diff --git a/differential-dataflow/examples/arrange.rs b/differential-dataflow/examples/arrange.rs index 6cbbad407..201627e2c 100644 --- a/differential-dataflow/examples/arrange.rs +++ b/differential-dataflow/examples/arrange.rs @@ -107,14 +107,14 @@ fn main() { let roots = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - roots.iterate(|dists| { + roots.clone().iterate(|dists| { let edges = edges.enter(&dists.scope()); let roots = roots.enter(&dists.scope()); dists.arrange_by_key() - .join_core(&edges, |_k,l,d| Some((*d, l+1))) - .concat(&roots) + .join_core(edges, |_k,l,d| Some((*d, l+1))) + .concat(roots) .reduce(|_, s, t| t.push((*s[0].0, 1))) }) .map(|(_node, dist)| dist) @@ -131,9 +131,9 @@ fn main() { let (input, query) = scope.new_collection(); query.map(|x| (x, x)) - .join_core(&edges, |_n, &q, &d| Some((d, q))) - .join_core(&edges, |_n, &q, &d| Some((d, q))) - .join_core(&edges, |_n, &q, &d| Some((d, q))) + .join_core(edges.clone(), |_n, &q, &d| Some((d, q))) + .join_core(edges.clone(), |_n, &q, &d| Some((d, q))) + .join_core(edges.clone(), |_n, &q, &d| Some((d, q))) .filter(move |_| inspect) .map(|x| x.1) .consolidate() diff --git a/differential-dataflow/examples/bfs.rs b/differential-dataflow/examples/bfs.rs index b95540825..0a35bb4e5 100644 --- a/differential-dataflow/examples/bfs.rs +++ b/differential-dataflow/examples/bfs.rs @@ -31,7 +31,7 @@ fn main() { let (root_input, roots) = scope.new_collection(); let (edge_input, graph) = scope.new_collection(); - let mut result = bfs(&graph, &roots); + let mut result = bfs(graph, roots); if !inspect { result = result.filter(|_| false); @@ -91,7 +91,7 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: &VecCollection, roots: &VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where G: Scope, { @@ -99,13 +99,13 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.iterate(|inner| { + nodes.clone().iterate(|inner| { let edges = edges.enter(&inner.scope()); let nodes = nodes.enter(&inner.scope()); - inner.join_map(&edges, |_k,l,d| (*d, l+1)) - .concat(&nodes) + inner.join_map(edges, |_k,l,d| (*d, l+1)) + .concat(nodes) .reduce(|_, s, t| t.push((*s[0].0, 1))) }) } diff --git a/differential-dataflow/examples/columnar.rs b/differential-dataflow/examples/columnar.rs index bf502ce15..9a97922f1 100644 --- a/differential-dataflow/examples/columnar.rs +++ b/differential-dataflow/examples/columnar.rs @@ -1,7 +1,7 @@ //! Wordcount based on `columnar`. use timely::container::{ContainerBuilder, PushInto}; -use timely::dataflow::InputHandleCore; +use timely::dataflow::InputHandle; use timely::dataflow::ProbeHandle; use differential_dataflow::operators::arrange::arrangement::arrange_core; @@ -25,8 +25,8 @@ fn main() { // initializes and runs a timely dataflow. timely::execute_from_args(std::env::args(), move |worker| { - let mut data_input = >::new_with_builder(); - let mut keys_input = >::new_with_builder(); + let mut data_input = >::new_with_builder(); + let mut keys_input = >::new_with_builder(); let mut probe = ProbeHandle::new(); // create a new input, exchange data, and inspect its output @@ -39,10 +39,10 @@ fn main() { let data_pact = KeyPact { hashfunc: |k: columnar::Ref<'_, Vec>| k.hashed() }; let keys_pact = KeyPact { hashfunc: |k: columnar::Ref<'_, Vec>| k.hashed() }; - let data = arrange_core::<_,_,KeyBatcher<_,_,_>, KeyBuilder<_,_,_>, KeySpine<_,_,_>>(&data, data_pact, "Data"); - let keys = arrange_core::<_,_,KeyBatcher<_,_,_>, KeyBuilder<_,_,_>, KeySpine<_,_,_>>(&keys, keys_pact, "Keys"); + let data = arrange_core::<_,_,KeyBatcher<_,_,_>, KeyBuilder<_,_,_>, KeySpine<_,_,_>>(data, data_pact, "Data"); + let keys = arrange_core::<_,_,KeyBatcher<_,_,_>, KeyBuilder<_,_,_>, KeySpine<_,_,_>>(keys, keys_pact, "Keys"); - keys.join_core(&data, |_k, (), ()| { Option::<()>::None }) + keys.join_core(data, |_k, (), ()| { Option::<()>::None }) .probe_with(&mut probe); }); diff --git a/differential-dataflow/examples/dynamic.rs b/differential-dataflow/examples/dynamic.rs index c63dec185..387571c27 100644 --- a/differential-dataflow/examples/dynamic.rs +++ b/differential-dataflow/examples/dynamic.rs @@ -31,7 +31,7 @@ fn main() { let (root_input, roots) = scope.new_collection(); let (edge_input, graph) = scope.new_collection(); - let mut result = bfs(&graph, &roots); + let mut result = bfs(graph, roots); if !inspect { result = result.filter(|_| false); @@ -91,7 +91,7 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: &VecCollection, roots: &VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where G: Scope, { @@ -117,12 +117,13 @@ where let next = label - .join_map(&edges, |_k,l,d| (*d, l+1)) - .concat(&nodes) + .clone() + .join_map(edges, |_k,l,d| (*d, l+1)) + .concat(nodes) .reduce(|_, s, t| t.push((*s[0].0, 1))) ; - label.set(&next); + label.set(next.clone()); // Leave the dynamic iteration, stripping off the last timestamp coordinate. next .leave_dynamic(1) diff --git a/differential-dataflow/examples/graspan.rs b/differential-dataflow/examples/graspan.rs index 0a2e95663..b108a0d40 100644 --- a/differential-dataflow/examples/graspan.rs +++ b/differential-dataflow/examples/graspan.rs @@ -88,7 +88,7 @@ pub struct EdgeVariable> { impl> EdgeVariable { /// Creates a new variable initialized with `source`. - pub fn from(source: &VecCollection, step: ::Summary) -> Self { + pub fn from(source: VecCollection, step: ::Summary) -> Self { let variable = VecVariable::new(&mut source.scope(), step); EdgeVariable { variable: variable, @@ -98,8 +98,8 @@ impl> EdgeVariable { } } /// Concatenates `production` into the definition of the variable. - pub fn add_production(&mut self, production: &VecCollection) { - self.current = self.current.concat(production); + pub fn add_production(&mut self, production: VecCollection) { + self.current = self.current.clone().concat(production); } /// Finalizes the variable, connecting its recursive definition. /// @@ -108,19 +108,19 @@ impl> EdgeVariable { pub fn complete(self) { let distinct = self.current.distinct(); // distinct.map(|_| ()).consolidate().inspect(|x| println!("{:?}", x)); - self.variable.set(&distinct); + self.variable.set(distinct); } /// The collection arranged in the forward direction. pub fn forward(&mut self) -> &Arrange { if self.forward.is_none() { - self.forward = Some(self.variable.arrange_by_key()); + self.forward = Some(self.variable.clone().arrange_by_key()); } self.forward.as_ref().unwrap() } /// The collection arranged in the reverse direction. pub fn reverse(&mut self) -> &Arrange { if self.reverse.is_none() { - self.reverse = Some(self.variable.map(|(x,y)| (y,x)).arrange_by_key()); + self.reverse = Some(self.variable.clone().map(|(x,y)| (y,x)).arrange_by_key()); } self.reverse.as_ref().unwrap() } @@ -168,8 +168,8 @@ impl Query { // create variables and result handles for each named relation. for (name, (input, collection)) in input_map { - let edge_variable = EdgeVariable::from(&collection.enter(subscope), Product::new(Default::default(), 1)); - let trace = edge_variable.variable.leave().arrange_by_self().trace; + let edge_variable = EdgeVariable::from(collection.enter(subscope), Product::new(Default::default(), 1)); + let trace = edge_variable.variable.clone().leave().arrange_by_self().trace; result_map.insert(name.clone(), RelationHandles { input, trace }); variable_map.insert(name.clone(), edge_variable); } @@ -191,8 +191,8 @@ impl Query { for relation in rule[1..].iter() { let to_join = match relation { - Relation::Forward(name) => variable_map.get_mut(name).unwrap().forward(), - Relation::Reverse(name) => variable_map.get_mut(name).unwrap().reverse(), + Relation::Forward(name) => variable_map.get_mut(name).unwrap().forward().clone(), + Relation::Reverse(name) => variable_map.get_mut(name).unwrap().reverse().clone(), }; transposed = @@ -202,7 +202,7 @@ impl Query { } // Reverse the direction before adding it as a production. - variable_map.get_mut(name).unwrap().add_production(&transposed.as_collection(|&dst,&src| (src,dst))); + variable_map.get_mut(name).unwrap().add_production(transposed.as_collection(|&dst,&src| (src,dst))); } } diff --git a/differential-dataflow/examples/interpreted.rs b/differential-dataflow/examples/interpreted.rs index fdfa201c2..4c6e3429c 100644 --- a/differential-dataflow/examples/interpreted.rs +++ b/differential-dataflow/examples/interpreted.rs @@ -31,21 +31,21 @@ fn main() { println!("loaded {} nodes, {} edges", nodes, edges.len()); worker.dataflow::<(),_,_>(|scope| { - interpret(&VecCollection::new(edges.to_stream(scope)), &[(0,2), (1,2)]); + interpret(VecCollection::new(edges.to_stream(scope)), &[(0,2), (1,2)]); }); }).unwrap(); } -fn interpret(edges: &VecCollection, relations: &[(usize, usize)]) -> VecCollection> +fn interpret(edges: VecCollection, relations: &[(usize, usize)]) -> VecCollection> where G: Scope, { // arrange the edge relation three ways. - let as_self = edges.arrange_by_self(); - let forward = edges.arrange_by_key(); - let reverse = edges.map_in_place(|x| ::std::mem::swap(&mut x.0, &mut x.1)) + let as_self = edges.clone().arrange_by_self(); + let forward = edges.clone().arrange_by_key(); + let reverse = edges.clone().map_in_place(|x| ::std::mem::swap(&mut x.0, &mut x.1)) .arrange_by_key(); let mut field_present = ::std::collections::HashSet::new(); @@ -65,14 +65,14 @@ where // Both variables are bound, so this is a semijoin. results .map(move |vec| ((vec[src], vec[dst]), vec)) - .join_core(&as_self, |_key, vec, &()| Some(vec.clone())) + .join_core(as_self.clone(), |_key, vec, &()| Some(vec.clone())) } (true, false) => { // Only `src` is bound, so we must use `forward` to propose `dst`. field_present.insert(dst); results .map(move |vec| (vec[src], vec)) - .join_core(&forward, move |_src_val, vec, &dst_val| { + .join_core(forward.clone(), move |_src_val, vec, &dst_val| { let mut temp = vec.clone(); while temp.len() <= dst { temp.push(0); } temp[dst] = dst_val; @@ -84,7 +84,7 @@ where field_present.insert(src); results .map(move |vec| (vec[dst], vec)) - .join_core(&reverse, move |_dst_val, vec, &src_val| { + .join_core(reverse.clone(), move |_dst_val, vec, &src_val| { let mut temp = vec.clone(); while temp.len() <= src { temp.push(0); } temp[src] = src_val; diff --git a/differential-dataflow/examples/itembased_cf.rs b/differential-dataflow/examples/itembased_cf.rs index 4909b4bf3..5b8018e1f 100644 --- a/differential-dataflow/examples/itembased_cf.rs +++ b/differential-dataflow/examples/itembased_cf.rs @@ -18,6 +18,7 @@ fn main() { // Find all users with less than 500 interactions let users_with_enough_interactions = interactions + .clone() .map(|(user, _item)| user) .count_total() .filter(move |(_user, count): &(u32, isize)| *count < 500) @@ -25,9 +26,10 @@ fn main() { // Remove users with too many interactions let remaining_interactions = interactions - .semijoin(&users_with_enough_interactions); + .semijoin(users_with_enough_interactions); let num_interactions_per_item = remaining_interactions + .clone() .map(|(_user, item)| item) .count_total(); @@ -35,7 +37,8 @@ fn main() { // Compute the number of cooccurrences of each item pair let cooccurrences = arranged_remaining_interactions - .join_core(&arranged_remaining_interactions, |_user, &item_a, &item_b| { + .clone() + .join_core(arranged_remaining_interactions, |_user, &item_a, &item_b| { if item_a > item_b { Some((item_a, item_b)) } else { None } }) .count(); @@ -48,12 +51,12 @@ fn main() { // Find the number of interactions for item_a .map(|((item_a, item_b), num_cooc)| (item_a, (item_b, num_cooc))) .join_core( - &arranged_num_interactions_per_item, + arranged_num_interactions_per_item.clone(), |&item_a, &(item_b, num_cooc), &occ_a| Some((item_b, (item_a, num_cooc, occ_a))) ) // Find the number of interactions for item_b .join_core( - &arranged_num_interactions_per_item, + arranged_num_interactions_per_item, |&item_b, &(item_a, num_cooc, occ_a), &occ_b| { Some(((item_a, item_b), (num_cooc, occ_a, occ_b))) }, diff --git a/differential-dataflow/examples/iterate_container.rs b/differential-dataflow/examples/iterate_container.rs index 72c4b5fcb..dcd330baa 100644 --- a/differential-dataflow/examples/iterate_container.rs +++ b/differential-dataflow/examples/iterate_container.rs @@ -3,7 +3,7 @@ use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Operator; use timely::order::Product; -use timely::dataflow::{Scope, StreamCore}; +use timely::dataflow::{Scope, Stream}; use timely::dataflow::operators::generic::builder_rc::OperatorBuilder; use differential_dataflow::{AsCollection, Collection}; use differential_dataflow::input::Input; @@ -33,7 +33,7 @@ impl, TS> ResultsIn for ContainerWrapper { #[inline(always)] fn results_in(self, step: &TS) -> Self { ContainerWrapper(self.0.results_in(step)) } } -fn wrap(stream: &StreamCore) -> StreamCore> { +fn wrap(stream: Stream) -> Stream> { let mut builder = OperatorBuilder::new("Wrap".to_string(), stream.scope()); let (mut output, stream_out) = builder.new_output(); let mut input = builder.new_input(stream, Pipeline); @@ -51,12 +51,12 @@ fn main() { timely::example(|scope| { let numbers = scope.new_collection_from(1 .. 10u32).1; - let numbers: Collection<_, _> = wrap(&numbers.inner).as_collection(); + let numbers: Collection<_, _> = wrap(numbers.inner).as_collection(); scope.iterative::(|nested| { let summary = Product::new(Default::default(), 1); let variable = Variable::new_from(numbers.enter(nested), summary); - let mapped: Collection<_, _> = variable.inner.unary(Pipeline, "Map", |_,_| { + let mapped: Collection<_, _> = variable.clone().inner.unary(Pipeline, "Map", |_,_| { |input, output| { input.for_each(|time, data| { let mut session = output.session(&time); @@ -75,8 +75,8 @@ fn main() { }); } }).as_collection().consolidate(); - let result = wrap(&result.inner).as_collection(); - variable.set(&result) + let result = wrap(result.inner).as_collection(); + variable.set(result) .leave() }); }) diff --git a/differential-dataflow/examples/monoid-bfs.rs b/differential-dataflow/examples/monoid-bfs.rs index 76bf192ed..c99f3b24a 100644 --- a/differential-dataflow/examples/monoid-bfs.rs +++ b/differential-dataflow/examples/monoid-bfs.rs @@ -55,7 +55,7 @@ fn main() { let (root_input, roots) = scope.new_collection(); let (edge_input, graph) = scope.new_collection(); - let mut result = bfs(&graph, &roots); + let mut result = bfs(graph, roots); if !inspect { result = result.filter(|_| false); @@ -122,7 +122,7 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: &VecCollection, roots: &VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where G: Scope, { @@ -140,9 +140,10 @@ where let result = variable + .clone() .map(|n| (n,())) - .join_map(&edges, |_k,&(),d| *d) - .concat(&roots) + .join_map(edges, |_k,&(),d| *d) + .concat(roots) .map(|x| (x,())) .reduce_core::<_,KeyBuilder<_,_,_>,KeySpine<_,_,_>>("Reduce", |_key, input, output, updates| { if output.is_empty() || input[0].1 < output[0].1 { @@ -151,7 +152,7 @@ where }) .as_collection(|k,()| *k); - variable.set(&result); + variable.set(result.clone()); result.leave() }) } diff --git a/differential-dataflow/examples/multitemporal.rs b/differential-dataflow/examples/multitemporal.rs index 8a0140ae7..63040c6f6 100644 --- a/differential-dataflow/examples/multitemporal.rs +++ b/differential-dataflow/examples/multitemporal.rs @@ -1,7 +1,7 @@ use std::io::BufRead; use timely::dataflow::ProbeHandle; -use timely::dataflow::operators::unordered_input::UnorderedInput; +use timely::dataflow::operators::vec::unordered_input::UnorderedInput; use timely::dataflow::operators::Probe; use timely::progress::frontier::AntichainRef; use timely::PartialOrder; diff --git a/differential-dataflow/examples/pagerank.rs b/differential-dataflow/examples/pagerank.rs index b80f0b7da..cc0cd1444 100644 --- a/differential-dataflow/examples/pagerank.rs +++ b/differential-dataflow/examples/pagerank.rs @@ -1,5 +1,5 @@ use timely::order::Product; -use timely::dataflow::{*, operators::Filter}; +use timely::dataflow::{*, operators::vec::Filter}; use differential_dataflow::VecCollection; use differential_dataflow::lattice::Lattice; @@ -33,7 +33,7 @@ fn main() { worker.dataflow::(|scope| { let edges = input.to_collection(scope); - pagerank(iterations, &edges) + pagerank(iterations, edges) .filter(move |_| inspect) .consolidate() .inspect(|x| println!("{:?}", x)) @@ -77,17 +77,19 @@ fn main() { // Returns a weighted collection in which the weight of each node is proportional // to its PageRank in the input graph `edges`. -fn pagerank(iters: Iter, edges: &VecCollection) -> VecCollection +fn pagerank(iters: Iter, edges: VecCollection) -> VecCollection where G: Scope, { // initialize many surfers at each node. let nodes = - edges.flat_map(|(x,y)| Some(x).into_iter().chain(Some(y))) + edges.clone() + .flat_map(|(x,y)| Some(x).into_iter().chain(Some(y))) .distinct(); // snag out-degrees for each node. - let degrs = edges.map(|(src,_dst)| src) + let degrs = edges.clone() + .map(|(src,_dst)| src) .count(); edges.scope().iterative::(|inner| { @@ -98,7 +100,7 @@ where let degrs = degrs.enter(inner); // Initial and reset numbers of surfers at each node. - let inits = nodes.explode(|node| Some((node, 6_000_000))); + let inits = nodes.clone().explode(|node| Some((node, 6_000_000))); let reset = nodes.explode(|node| Some((node, 1_000_000))); // Define a recursive variable to track surfers. @@ -107,15 +109,15 @@ where // Match each surfer with the degree, scale numbers down. let to_push = - degrs.semijoin(&ranks) + degrs.semijoin(ranks.clone()) .threshold(|(_node, degr), rank| (5 * rank) / (6 * degr)) .map(|(node, _degr)| node); // Propagate surfers along links, blend in reset surfers. let mut pushed = - edges.semijoin(&to_push) + edges.semijoin(to_push) .map(|(_node, dest)| dest) - .concat(&reset) + .concat(reset) .consolidate(); if iters > 0 { @@ -127,7 +129,7 @@ where } // Bind the recursive variable, return its limit. - ranks.set(&pushed); + ranks.set(pushed.clone()); pushed.leave() }) } diff --git a/differential-dataflow/examples/progress.rs b/differential-dataflow/examples/progress.rs index 9be35a9cc..67783e488 100644 --- a/differential-dataflow/examples/progress.rs +++ b/differential-dataflow/examples/progress.rs @@ -126,15 +126,16 @@ where // Translate node and edge transitions into a common Location to Location edge with an associated Summary. let nodes = nodes.map(|(target, source, summary)| (Location::from(target), (Location::from(source), summary))); let edges = edges.map(|(source, target)| (Location::from(source), (Location::from(target), Default::default()))); - let transitions: VecCollection = nodes.concat(&edges); + let transitions: VecCollection = nodes.concat(edges); times + .clone() .iterate(|reach| { transitions .enter(&reach.scope()) - .join_map(&reach, |_from, (dest, summ), time| (dest.clone(), summ.results_in(time))) + .join_map(reach.clone(), |_from, (dest, summ), time| (dest.clone(), summ.results_in(time))) .flat_map(|(dest, time)| time.map(move |time| (dest, time))) - .concat(×.enter(&reach.scope())) + .concat(times.enter(&reach.scope())) .reduce(|_location, input, output: &mut Vec<(T, isize)>| { // retain the lower envelope of times. for (t1, _count1) in input.iter() { @@ -159,6 +160,7 @@ where // Start from trivial reachability from each input to itself. let zero_inputs = edges + .clone() .map(|(_source, target)| Location::from(target)) .filter(|location| location.node == 0) .map(|location| (location, (location, Default::default()))); @@ -166,15 +168,16 @@ where // Retain node connections along "default" timestamp summaries. let nodes = nodes.map(|(target, source, summary)| (Location::from(source), (Location::from(target), summary))); let edges = edges.map(|(source, target)| (Location::from(target), (Location::from(source), Default::default()))); - let transitions: VecCollection = nodes.concat(&edges); + let transitions: VecCollection = nodes.concat(edges); zero_inputs + .clone() .iterate(|summaries| { transitions .enter(&summaries.scope()) - .join_map(summaries, |_middle, (from, summ1), (to, summ2)| (from.clone(), to.clone(), summ1.followed_by(summ2))) + .join_map(summaries.clone(), |_middle, (from, summ1), (to, summ2)| (from.clone(), to.clone(), summ1.followed_by(summ2))) .flat_map(|(from, to, summ)| summ.map(move |summ| (from, (to, summ)))) - .concat(&zero_inputs.enter(&summaries.scope())) + .concat(zero_inputs.enter(&summaries.scope())) .map(|(from, (to, summary))| ((from, to), summary)) .reduce(|_from_to, input, output| { for (summary, _count) in input.iter() { @@ -209,18 +212,20 @@ where } }); let edges = edges.map(|(source, target)| (Location::from(source), Location::from(target))); - let transitions: VecCollection = nodes.concat(&edges); + let transitions: VecCollection = nodes.concat(edges); // Repeatedly restrict to locations with an incoming path. transitions + .clone() .iterate(|locations| { let active = locations + .clone() .map(|(_source, target)| target) .distinct(); transitions .enter(&locations.scope()) - .semijoin(&active) + .semijoin(active) }) .consolidate() } diff --git a/differential-dataflow/examples/projekt.rs b/differential-dataflow/examples/projekt.rs index e611f4773..fb73de71e 100644 --- a/differential-dataflow/examples/projekt.rs +++ b/differential-dataflow/examples/projekt.rs @@ -26,10 +26,11 @@ fn main() { // Report unmet XY goals, and met XY non-goals. let xy_errors = - xyzs.map(|(x,y,_)| (x,y)) + xyzs.clone() + .map(|(x,y,_)| (x,y)) .distinct() .negate() - .concat(&xy_goal) + .concat(xy_goal) .consolidate(); // Report unmet XZ goals, and met XZ non-goals. @@ -37,13 +38,13 @@ fn main() { xyzs.map(|(x,_,z)| (x,z)) .distinct() .negate() - .concat(&xz_goal) + .concat(xz_goal) .consolidate(); let xy_total = xy_errors.distinct().map(|_| ()); let xz_total = xz_errors.distinct().map(|_| ()); xy_total - .concat(&xz_total) + .concat(xz_total) .distinct() .inspect(|x| println!("Not done: {:?}", x)) .probe_with(&mut probe); @@ -58,7 +59,7 @@ fn main() { let xy_xs = xy_goal.map(|(x,_)| (x,())); let xz_xs = xz_goal.map(|(x,_)| (x,())); - xy_xs.join(&xz_xs) + xy_xs.join(xz_xs) .map(|_| ()) .consolidate() .inspect(|x| println!("Maximum solution size: {}", x.2)) @@ -66,7 +67,7 @@ fn main() { // // For each x, produce valid pairs of y and z. // xy_goal - // .join(&xz_goal) + // .join(xz_goal) // .map(|(x,(y,z))| (x,y,z)) // .inspect(|x| println!("Maximum solution: {:?}", x)) // .probe_with(&mut probe); @@ -82,7 +83,7 @@ fn main() { let xy_xs = xy_goal.map(|(x,_)| x).count(); let xz_xs = xz_goal.map(|(x,_)| x).count(); - xy_xs.join(&xz_xs) + xy_xs.join(xz_xs) .explode(|(_,(ys,zs))| Some(((), ::std::cmp::max(ys,zs)))) .consolidate() .inspect(|x| println!("Minimum solution size: {}", x.2)) @@ -96,7 +97,7 @@ fn main() { // out.push((zs.iter().map(|(&z,_)| z).collect::>(), 1)) // ); - // xy_xs.join(&xz_xs) + // xy_xs.join(xz_xs) // .flat_map(|(x,(ys, zs))| { // let max = ::std::cmp::max(ys.len(), zs.len()); // let ys = ys.into_iter().cycle(); diff --git a/differential-dataflow/examples/spines.rs b/differential-dataflow/examples/spines.rs index 5ef6e7bec..a72a6f34b 100644 --- a/differential-dataflow/examples/spines.rs +++ b/differential-dataflow/examples/spines.rs @@ -36,21 +36,21 @@ fn main() { use differential_dataflow::trace::implementations::ord_neu::{ColKeyBatcher, ColKeyBuilder, ColKeySpine}; let data = data.arrange::, ColKeyBuilder<_,_,_>, ColKeySpine<_,_,_>>(); let keys = keys.arrange::, ColKeyBuilder<_,_,_>, ColKeySpine<_,_,_>>(); - keys.join_core(&data, |_k, &(), &()| Option::<()>::None) + keys.join_core(data, |_k, &(), &()| Option::<()>::None) .probe_with(&mut probe); }, "old" => { use differential_dataflow::trace::implementations::ord_neu::{OrdKeyBatcher, RcOrdKeyBuilder, OrdKeySpine}; let data = data.arrange::, RcOrdKeyBuilder<_,_,_>, OrdKeySpine<_,_,_>>(); let keys = keys.arrange::, RcOrdKeyBuilder<_,_,_>, OrdKeySpine<_,_,_>>(); - keys.join_core(&data, |_k, &(), &()| Option::<()>::None) + keys.join_core(data, |_k, &(), &()| Option::<()>::None) .probe_with(&mut probe); }, "rhh" => { use differential_dataflow::trace::implementations::rhh::{HashWrapper, VecBatcher, VecBuilder, VecSpine}; let data = data.map(|x| HashWrapper { inner: x }).arrange::, VecBuilder<_,(),_,_>, VecSpine<_,(),_,_>>(); let keys = keys.map(|x| HashWrapper { inner: x }).arrange::, VecBuilder<_,(),_,_>, VecSpine<_,(),_,_>>(); - keys.join_core(&data, |_k, &(), &()| Option::<()>::None) + keys.join_core(data, |_k, &(), &()| Option::<()>::None) .probe_with(&mut probe); }, _ => { diff --git a/differential-dataflow/examples/stackoverflow.rs b/differential-dataflow/examples/stackoverflow.rs index a3a9bd260..c84ece846 100644 --- a/differential-dataflow/examples/stackoverflow.rs +++ b/differential-dataflow/examples/stackoverflow.rs @@ -40,7 +40,7 @@ fn main() { let roots = roots.to_collection(scope); let graph = graph.to_collection(scope); - bfs(&graph, &roots) + bfs(graph, roots) .filter(move |_| inspect) .map(|(_,l)| l) .consolidate() @@ -105,7 +105,7 @@ fn main() { } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: &VecCollection, roots: &VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where G: Scope, { @@ -113,13 +113,13 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.iterate(|inner| { + nodes.clone().iterate(|inner| { let edges = edges.enter(&inner.scope()); let nodes = nodes.enter(&inner.scope()); - inner.join_map(&edges, |_k,l,d| (*d, l+1)) - .concat(&nodes) + inner.join_map(edges, |_k,l,d| (*d, l+1)) + .concat(nodes) .reduce(|_, s, t| t.push((*s[0].0, 1))) }) } diff --git a/differential-dataflow/src/algorithms/graphs/bfs.rs b/differential-dataflow/src/algorithms/graphs/bfs.rs index baadde341..4ccc89403 100644 --- a/differential-dataflow/src/algorithms/graphs/bfs.rs +++ b/differential-dataflow/src/algorithms/graphs/bfs.rs @@ -9,20 +9,20 @@ use crate::operators::*; use crate::lattice::Lattice; /// Returns pairs (node, dist) indicating distance of each node from a root. -pub fn bfs(edges: &VecCollection, roots: &VecCollection) -> VecCollection +pub fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where G: Scope, N: ExchangeData+Hash, { let edges = edges.arrange_by_key(); - bfs_arranged(&edges, roots) + bfs_arranged(edges, roots) } use crate::trace::TraceReader; use crate::operators::arrange::Arranged; /// Returns pairs (node, dist) indicating distance of each node from a root. -pub fn bfs_arranged(edges: &Arranged, roots: &VecCollection) -> VecCollection +pub fn bfs_arranged(edges: Arranged, roots: VecCollection) -> VecCollection where G: Scope, N: ExchangeData+Hash, @@ -32,13 +32,13 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.iterate(|inner| { + nodes.clone().iterate(|inner| { let edges = edges.enter(&inner.scope()); let nodes = nodes.enter(&inner.scope()); - inner.join_core(&edges, |_k,l,d| Some((d.clone(), l+1))) - .concat(&nodes) + inner.join_core(edges, |_k,l,d| Some((d.clone(), l+1))) + .concat(nodes) .reduce(|_, s, t| t.push((*s[0].0, 1))) }) } diff --git a/differential-dataflow/src/algorithms/graphs/bijkstra.rs b/differential-dataflow/src/algorithms/graphs/bijkstra.rs index 96425dad0..df563c5f7 100644 --- a/differential-dataflow/src/algorithms/graphs/bijkstra.rs +++ b/differential-dataflow/src/algorithms/graphs/bijkstra.rs @@ -19,14 +19,14 @@ use crate::operators::iterate::Variable; /// Goals that cannot reach from the source to the target are relatively expensive, as /// the entire graph must be explored to confirm this. A graph connectivity pre-filter /// could be good insurance here. -pub fn bidijkstra(edges: &VecCollection, goals: &VecCollection) -> VecCollection +pub fn bidijkstra(edges: VecCollection, goals: VecCollection) -> VecCollection where G: Scope, N: ExchangeData+Hash, { - let forward = edges.arrange_by_key(); + let forward = edges.clone().arrange_by_key(); let reverse = edges.map(|(x,y)| (y,x)).arrange_by_key(); - bidijkstra_arranged(&forward, &reverse, goals) + bidijkstra_arranged(forward, reverse, goals) } use crate::trace::TraceReader; @@ -34,9 +34,9 @@ use crate::operators::arrange::Arranged; /// Bi-directional Dijkstra search using arranged forward and reverse edge collections. pub fn bidijkstra_arranged( - forward: &Arranged, - reverse: &Arranged, - goals: &VecCollection + forward: Arranged, + reverse: Arranged, + goals: VecCollection ) -> VecCollection where G: Scope, @@ -55,11 +55,11 @@ where // is a corresponding destination or source that has not yet been reached. // forward and reverse (node, (root, dist)) - let forward = Variable::new_from(goals.map(|(x,_)| (x.clone(),(x.clone(),0))).enter(inner), Product::new(Default::default(), 1)); - let reverse = Variable::new_from(goals.map(|(_,y)| (y.clone(),(y.clone(),0))).enter(inner), Product::new(Default::default(), 1)); + let forward = Variable::new_from(goals.clone().map(|(x,_)| (x.clone(),(x.clone(),0))).enter(inner), Product::new(Default::default(), 1)); + let reverse = Variable::new_from(goals.clone().map(|(_,y)| (y.clone(),(y.clone(),0))).enter(inner), Product::new(Default::default(), 1)); - forward.map(|_| ()).consolidate().inspect(|x| println!("forward: {:?}", x)); - reverse.map(|_| ()).consolidate().inspect(|x| println!("reverse: {:?}", x)); + forward.clone().map(|_| ()).consolidate().inspect(|x| println!("forward: {:?}", x)); + reverse.clone().map(|_| ()).consolidate().inspect(|x| println!("reverse: {:?}", x)); let goals = goals.enter(inner); // let edges = edges.enter(inner); @@ -71,50 +71,54 @@ where // This is a cyclic join, which should scare us a bunch. let reached = forward - .join_map(&reverse, |_, (src,d1), (dst,d2)| ((src.clone(), dst.clone()), *d1 + *d2)) + .clone() + .join_map(reverse.clone(), |_, (src,d1), (dst,d2)| ((src.clone(), dst.clone()), *d1 + *d2)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) - .semijoin(&goals); + .semijoin(goals.clone()); let active = reached + .clone() .negate() .map(|(srcdst,_)| srcdst) - .concat(&goals) + .concat(goals) .consolidate(); // Let's expand out forward queries that are active. - let forward_active = active.map(|(x,_y)| x).distinct(); + let forward_active = active.clone().map(|(x,_y)| x).distinct(); let forward_next = forward + .clone() .map(|(med, (src, dist))| (src, (med, dist))) - .semijoin(&forward_active) + .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) - .join_core(&forward_edges, |_med, (src, dist), next| Some((next.clone(), (src.clone(), *dist+1)))) - .concat(&forward) + .join_core(forward_edges, |_med, (src, dist), next| Some((next.clone(), (src.clone(), *dist+1)))) + .concat(forward.clone()) .map(|(next, (src, dist))| ((next, src), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); - forward_next.map(|_| ()).consolidate().inspect(|x| println!("forward_next: {:?}", x)); + forward_next.clone().map(|_| ()).consolidate().inspect(|x| println!("forward_next: {:?}", x)); - forward.set(&forward_next); + forward.set(forward_next); // Let's expand out reverse queries that are active. let reverse_active = active.map(|(_x,y)| y).distinct(); let reverse_next = reverse + .clone() .map(|(med, (rev, dist))| (rev, (med, dist))) - .semijoin(&reverse_active) + .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) - .join_core(&reverse_edges, |_med, (rev, dist), next| Some((next.clone(), (rev.clone(), *dist+1)))) - .concat(&reverse) + .join_core(reverse_edges, |_med, (rev, dist), next| Some((next.clone(), (rev.clone(), *dist+1)))) + .concat(reverse.clone()) .map(|(next, (rev, dist))| ((next, rev), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); - reverse_next.map(|_| ()).consolidate().inspect(|x| println!("reverse_next: {:?}", x)); + reverse_next.clone().map(|_| ()).consolidate().inspect(|x| println!("reverse_next: {:?}", x)); - reverse.set(&reverse_next); + reverse.set(reverse_next); reached.leave() }) diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 635bb38e9..0755d470c 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -13,7 +13,7 @@ use crate::difference::{Abelian, Multiply}; /// This algorithm naively propagates all labels at once, much like standard label propagation. /// To more carefully control the label propagation, consider `propagate_core` which supports a /// method to limit the introduction of labels. -pub fn propagate(edges: &VecCollection, nodes: &VecCollection) -> VecCollection +pub fn propagate(edges: VecCollection, nodes: VecCollection) -> VecCollection where G: Scope, N: ExchangeData+Hash, @@ -22,7 +22,7 @@ where R: From, L: ExchangeData, { - propagate_core(&edges.arrange_by_key(), nodes, |_label| 0) + propagate_core(edges.arrange_by_key(), nodes, |_label| 0) } /// Propagates labels forward, retaining the minimum label. @@ -30,7 +30,7 @@ where /// This algorithm naively propagates all labels at once, much like standard label propagation. /// To more carefully control the label propagation, consider `propagate_core` which supports a /// method to limit the introduction of labels. -pub fn propagate_at(edges: &VecCollection, nodes: &VecCollection, logic: F) -> VecCollection +pub fn propagate_at(edges: VecCollection, nodes: VecCollection, logic: F) -> VecCollection where G: Scope, N: ExchangeData+Hash, @@ -40,7 +40,7 @@ where L: ExchangeData, F: Fn(&L)->u64+Clone+'static, { - propagate_core(&edges.arrange_by_key(), nodes, logic) + propagate_core(edges.arrange_by_key(), nodes, logic) } use crate::trace::TraceReader; @@ -51,7 +51,7 @@ use crate::operators::arrange::arrangement::Arranged; /// This variant takes a pre-arranged edge collection, to facilitate re-use, and allows /// a method `logic` to specify the rounds in which we introduce various labels. The output /// of `logic should be a number in the interval \[0,64\], -pub fn propagate_core(edges: &Arranged, nodes: &VecCollection, logic: F) -> VecCollection +pub fn propagate_core(edges: Arranged, nodes: VecCollection, logic: F) -> VecCollection where G: Scope, N: ExchangeData+Hash, @@ -72,8 +72,8 @@ where // .iterate(|inner| { // let edges = edges.enter(&inner.scope()); // let nodes = nodes.enter_at(&inner.scope(), move |r| 256 * (64 - (logic(&r.1)).leading_zeros() as u64)); - // inner.join_map(&edges, |_k,l,d| (d.clone(),l.clone())) - // .concat(&nodes) + // inner.join_map(edges, |_k,l,d| (d.clone(),l.clone())) + // .concat(nodes) // .reduce(|_, s, t| t.push((s[0].0.clone(), 1))) // }) @@ -91,14 +91,16 @@ where let labels = proposals - .concat(&nodes) + .clone() + .concat(nodes) .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine<_,_,_,_>>("Propagate", |_, s, t| t.push((s[0].0.clone(), R::from(1_i8)))); let propagate: VecCollection<_, (N, L), R> = labels - .join_core(&edges, |_k, l: &L, d| Some((d.clone(), l.clone()))); + .clone() + .join_core(edges, |_k, l: &L, d| Some((d.clone(), l.clone()))); - proposals.set(&propagate); + proposals.set(propagate); labels .as_collection(|k,v| (k.clone(), v.clone())) diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index c3ad20a81..390174ed4 100644 --- a/differential-dataflow/src/algorithms/graphs/scc.rs +++ b/differential-dataflow/src/algorithms/graphs/scc.rs @@ -13,7 +13,7 @@ use crate::difference::{Abelian, Multiply}; use super::propagate::propagate; /// Iteratively removes nodes with no in-edges. -pub fn trim(graph: &VecCollection) -> VecCollection +pub fn trim(graph: VecCollection) -> VecCollection where G: Scope, N: ExchangeData + Hash, @@ -21,19 +21,19 @@ where R: Multiply, R: From, { - graph.iterate(|edges| { + graph.clone().iterate(|edges| { // keep edges from active edge destinations. + let graph = graph.enter(&edges.scope()); let active = edges.map(|(_src,dst)| dst) .threshold(|_,c| if c.is_zero() { R::from(0_i8) } else { R::from(1_i8) }); - graph.enter(&edges.scope()) - .semijoin(&active) + graph.semijoin(active) }) } /// Returns the subset of edges in the same strongly connected component. -pub fn strongly_connected(graph: &VecCollection) -> VecCollection +pub fn strongly_connected(graph: VecCollection) -> VecCollection where G: Scope, N: ExchangeData + Hash, @@ -41,14 +41,14 @@ where R: Multiply, R: From { - graph.iterate(|inner| { + graph.clone().iterate(|inner| { let edges = graph.enter(&inner.scope()); - let trans = edges.map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); - trim_edges(&trim_edges(inner, &edges), &trans) + let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); + trim_edges(trim_edges(inner, edges), trans) }) } -fn trim_edges(cycle: &VecCollection, edges: &VecCollection) +fn trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection where G: Scope, @@ -57,15 +57,16 @@ where R: Multiply, R: From { - let nodes = edges.map_in_place(|x| x.0 = x.1.clone()) + let nodes = edges.clone() + .map_in_place(|x| x.0 = x.1.clone()) .consolidate(); // NOTE: With a node -> int function, can be improved by: // let labels = propagate_at(&cycle, &nodes, |x| *x as u64); - let labels = propagate(cycle, &nodes); + let labels = propagate(cycle, nodes); - edges.join_map(&labels, |e1,e2,l1| (e2.clone(),(e1.clone(),l1.clone()))) - .join_map(&labels, |e2,(e1,l1),l2| ((e1.clone(),e2.clone()),(l1.clone(),l2.clone()))) + edges.join_map(labels.clone(), |e1,e2,l1| (e2.clone(),(e1.clone(),l1.clone()))) + .join_map(labels, |e2,(e1,l1),l2| ((e1.clone(),e2.clone()),(l1.clone(),l2.clone()))) .filter(|(_,(l1,l2))| l1 == l2) .map(|((x1,x2),_)| (x2,x1)) } diff --git a/differential-dataflow/src/algorithms/graphs/sequential.rs b/differential-dataflow/src/algorithms/graphs/sequential.rs index 3f4bbcf29..a98f8d843 100644 --- a/differential-dataflow/src/algorithms/graphs/sequential.rs +++ b/differential-dataflow/src/algorithms/graphs/sequential.rs @@ -9,17 +9,18 @@ use crate::lattice::Lattice; use crate::operators::*; use crate::hashable::Hashable; -fn _color(edges: &VecCollection) -> VecCollection)> +fn _color(edges: VecCollection) -> VecCollection)> where G: Scope, N: ExchangeData+Hash, { // need some bogus initial values. - let start = edges.map(|(x,_y)| (x,u32::max_value())) + let start = edges.clone() + .map(|(x,_y)| (x,u32::max_value())) .distinct(); // repeatedly apply color-picking logic. - sequence(&start, edges, |_node, vals| { + sequence(start, edges, |_node, vals| { // look for the first absent positive integer. // start at 1 in case we ever use NonZero. @@ -40,8 +41,8 @@ where /// fired, and we apply `logic` to the new state of lower neighbors and /// the old state (input) of higher neighbors. pub fn sequence( - state: &VecCollection, - edges: &VecCollection, + state: VecCollection, + edges: VecCollection, logic: F) -> VecCollection)> where G: Scope, @@ -52,6 +53,7 @@ where // start iteration with None messages for all. state + .clone() .map(|(node, _state)| (node, None)) .iterate(|new_state| { // immutable content: edges and initial state. @@ -60,27 +62,27 @@ where // .map(|x| (x.0, Some(x.1))); // break edges into forward and reverse directions. - let forward = edges.filter(|edge| edge.0 < edge.1); + let forward = edges.clone().filter(|edge| edge.0 < edge.1); let reverse = edges.filter(|edge| edge.0 > edge.1); // new state goes along forward edges, old state along reverse edges - let new_messages = new_state.join_map(&forward, |_k,v,d| (d.clone(),v.clone())); + let new_messages = new_state.join_map(forward, |_k,v,d| (d.clone(),v.clone())); - let incomplete = new_messages.filter(|x| x.1.is_none()).map(|x| x.0).distinct(); + let incomplete = new_messages.clone().filter(|x| x.1.is_none()).map(|x| x.0).distinct(); let new_messages = new_messages.filter(|x| x.1.is_some()).map(|x| (x.0, x.1.unwrap())); - let old_messages = old_state.join_map(&reverse, |_k,v,d| (d.clone(),v.clone())); + let old_messages = old_state.join_map(reverse, |_k,v,d| (d.clone(),v.clone())); - let messages = new_messages.concat(&old_messages).antijoin(&incomplete); + let messages = new_messages.concat(old_messages).antijoin(incomplete.clone()); // // determine who has incoming `None` messages, and suppress all of them. // let incomplete = new_messages.filter(|x| x.1.is_none()).map(|x| x.0).distinct(); // merge messages; suppress computation if not all inputs available yet. messages - // .concat(&old_messages) // /-- possibly too clever: None if any inputs None. - // .antijoin(&incomplete) + // .concat(old_messages) // /-- possibly too clever: None if any inputs None. + // .antijoin(incomplete) .reduce(move |k, vs, t| t.push((Some(logic(k,vs)),1))) - .concat(&incomplete.map(|x| (x, None))) + .concat(incomplete.map(|x| (x, None))) }) } diff --git a/differential-dataflow/src/algorithms/identifiers.rs b/differential-dataflow/src/algorithms/identifiers.rs index e44dbe92c..e38a46ba1 100644 --- a/differential-dataflow/src/algorithms/identifiers.rs +++ b/differential-dataflow/src/algorithms/identifiers.rs @@ -27,7 +27,7 @@ pub trait Identifiers { /// .assert_empty(); /// }); /// ``` - fn identifiers(&self) -> VecCollection; + fn identifiers(self) -> VecCollection; } impl Identifiers for VecCollection @@ -36,7 +36,7 @@ where D: ExchangeData + ::std::hash::Hash, R: ExchangeData + Abelian, { - fn identifiers(&self) -> VecCollection { + fn identifiers(self) -> VecCollection { // The design here is that we iteratively develop a collection // of pairs (round, record), where each pair is a proposal that @@ -58,7 +58,8 @@ where timely::dataflow::operators::generic::operator::empty(&init.scope()) .as_collection() .iterate(|diff| - init.enter(&diff.scope()) + init.clone() + .enter(&diff.scope()) .concat(diff) .map(|pair| (pair.hashed(), pair)) .reduce(|_hash, input, output| { @@ -80,7 +81,7 @@ where }) .map(|(_hash, pair)| pair) ) - .concat(&init) + .concat(init) .map(|pair| { let hash = pair.hashed(); (pair.1, hash) }) } } @@ -109,8 +110,9 @@ mod tests { timely::dataflow::operators::generic::operator::empty(&init.scope()) .as_collection() .iterate(|diff| - init.enter(&diff.scope()) - .concat(&diff) + init.clone() + .enter(&diff.scope()) + .concat(diff) .map(|(round, num)| ((round + num) / 10, (round, num))) .reduce(|_hash, input, output| { println!("Input: {:?}", input); @@ -129,7 +131,7 @@ mod tests { .inspect(|x| println!("{:?}", x)) .map(|(_hash, pair)| pair) ) - .concat(&init) + .concat(init) .map(|(round, num)| { (num, (round + num) / 10) }) .map(|(_data, id)| id) .threshold(|_id,cnt| if cnt > &1 { *cnt } else { 0 }) diff --git a/differential-dataflow/src/algorithms/prefix_sum.rs b/differential-dataflow/src/algorithms/prefix_sum.rs index e479c8229..9a8767f47 100644 --- a/differential-dataflow/src/algorithms/prefix_sum.rs +++ b/differential-dataflow/src/algorithms/prefix_sum.rs @@ -13,10 +13,10 @@ pub trait PrefixSum { /// The prefix sum is data-parallel, in the sense that the sums are computed independently for /// each key of type `K`. For a single prefix sum this type can be `()`, but this permits the /// more general accumulation of multiple independent sequences. - fn prefix_sum(&self, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; + fn prefix_sum(self, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; /// Determine the prefix sum at each element of `location`. - fn prefix_sum_at(&self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; + fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static; } impl PrefixSum for VecCollection @@ -25,11 +25,11 @@ where K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, { - fn prefix_sum(&self, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { - self.prefix_sum_at(self.map(|(x,_)| x), zero, combine) + fn prefix_sum(self, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { + self.clone().prefix_sum_at(self.map(|(x,_)| x), zero, combine) } - fn prefix_sum_at(&self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { + fn prefix_sum_at(self, locations: VecCollection, zero: D, combine: F) -> Self where F: Fn(&K,&D,&D)->D + 'static { let combine1 = ::std::rc::Rc::new(combine); let combine2 = combine1.clone(); @@ -51,13 +51,15 @@ where let unit_ranges = collection.map(|((index, key), data)| ((index, 0, key), data)); unit_ranges - .iterate(|ranges| + .clone() + .iterate(|ranges| { // Each available range, of size less than usize::max_value(), advertises itself as the range // twice as large, aligned to integer multiples of its size. Each range, which may contain at // most two elements, then summarizes itself using the `combine` function. Finally, we re-add // the initial `unit_ranges` intervals, so that the set of ranges grows monotonically. + let unit_ranges = unit_ranges.enter(&ranges.scope()); ranges .filter(|&((_pos, log, _), _)| log < 64) .map(|((pos, log, key), data)| ((pos >> 1, log + 1, key), (pos, data))) @@ -66,8 +68,8 @@ where if input.len() > 1 { result = combine(key, &result, &(input[1].0).1); } output.push((result, 1)); }) - .concat(&unit_ranges.enter(&ranges.scope())) - ) + .concat(unit_ranges) + }) } /// Produces the accumulated values at each of the `usize` locations in `queries`. @@ -105,6 +107,7 @@ where // the loop to pre-suppress duplicate requests. This comes at a complexity cost, though. let requests = queries + .clone() .flat_map(|(idx, key)| (0 .. 64) .filter(move |i| (idx & (1usize << i)) != 0) // set bits require help. @@ -115,34 +118,38 @@ where // Acquire each requested range. let full_ranges = ranges - .semijoin(&requests); + .semijoin(requests.clone()); // Each requested range should exist, even if as a zero range, for correct reconstruction. let zero_ranges = full_ranges + .clone() .map(move |((idx, log, key), _)| ((idx, log, key), zero0.clone())) .negate() - .concat(&requests.map(move |(idx, log, key)| ((idx, log, key), zero1.clone()))); + .concat(requests.map(move |(idx, log, key)| ((idx, log, key), zero1.clone()))); // Merge occupied and empty ranges. - let used_ranges = full_ranges.concat(&zero_ranges); + let used_ranges = full_ranges.concat(zero_ranges); // Each key should initiate a value of `zero` at position `0`. let init_states = queries + .clone() .map(move |(_, key)| ((0, key), zero2.clone())) .distinct(); // Iteratively expand assigned values by joining existing ranges with current assignments. init_states + .clone() .iterate(|states| { + let init_states = init_states.enter(&states.scope()); used_ranges .enter(&states.scope()) .map(|((pos, log, key), data)| ((pos << log, key), (log, data))) .join_map(states, move |&(pos, ref key), &(log, ref data), state| ((pos + (1 << log), key.clone()), combine(key, state, data))) - .concat(&init_states.enter(&states.scope())) + .concat(init_states) .distinct() }) - .semijoin(&queries) + .semijoin(queries) } diff --git a/differential-dataflow/src/capture.rs b/differential-dataflow/src/capture.rs index 23d9ffd9e..ffa3a5a28 100644 --- a/differential-dataflow/src/capture.rs +++ b/differential-dataflow/src/capture.rs @@ -227,7 +227,7 @@ pub mod source { use std::rc::Rc; use std::marker::{Send, Sync}; use std::sync::Arc; - use timely::dataflow::{Scope, Stream, operators::{Capability, CapabilitySet}}; + use timely::dataflow::{Scope, StreamVec, operators::{Capability, CapabilitySet}}; use timely::dataflow::operators::generic::OutputBuilder; use timely::progress::Timestamp; use timely::scheduling::SyncActivator; @@ -253,7 +253,7 @@ pub mod source { pub fn build( scope: G, source_builder: B, - ) -> (Box, Stream) + ) -> (Box, StreamVec) where G: Scope, B: FnOnce(SyncActivator) -> I, @@ -390,7 +390,7 @@ pub mod source { // Step 2: The UPDATES operator. let mut updates_op = OperatorBuilder::new("CDCV2_Updates".to_string(), scope.clone()); - let mut input = updates_op.new_input(&updates, Exchange::new(|x: &(D, T, R)| x.hashed())); + let mut input = updates_op.new_input(updates, Exchange::new(|x: &(D, T, R)| x.hashed())); let (changes_out, changes) = updates_op.new_output(); let mut changes_out = OutputBuilder::from(changes_out); let (counts_out, counts) = updates_op.new_output(); @@ -440,11 +440,11 @@ pub mod source { // Step 3: The PROGRESS operator. let mut progress_op = OperatorBuilder::new("CDCV2_Progress".to_string(), scope.clone()); let mut input = progress_op.new_input( - &progress, + progress, Exchange::new(|x: &(usize, Progress)| x.0 as u64), ); let mut counts = - progress_op.new_input(&counts, Exchange::new(|x: &(T, i64)| (x.0).hashed())); + progress_op.new_input(counts, Exchange::new(|x: &(T, i64)| (x.0).hashed())); let (frontier_out, frontier) = progress_op.new_output(); let mut frontier_out = OutputBuilder::from(frontier_out); progress_op.build(move |_capability| { @@ -468,12 +468,12 @@ pub mod source { // Drain all relevant update counts in to the mutable antichain tracking its frontier. counts.for_each(|cap, counts| { updates_frontier.update_iter(counts.iter().cloned()); - capability = Some(cap.retain()); + capability = Some(cap.retain(0)); }); // Drain all progress statements into the queue out of which we will work. input.for_each(|cap, progress| { progress_queue.extend(progress.iter().map(|x| (x.1).clone())); - capability = Some(cap.retain()); + capability = Some(cap.retain(0)); }); // Extract and act on actionable progress messages. @@ -524,7 +524,7 @@ pub mod source { // Step 4: The FEEDBACK operator. let mut feedback_op = OperatorBuilder::new("CDCV2_Feedback".to_string(), scope.clone()); let mut input = feedback_op.new_input( - &frontier, + frontier, Exchange::new(|x: &(usize, ChangeBatch)| x.0 as u64), ); feedback_op.build(move |_capability| { @@ -560,7 +560,7 @@ pub mod sink { use timely::order::PartialOrder; use timely::progress::{Antichain, ChangeBatch, Timestamp}; - use timely::dataflow::{Scope, Stream}; + use timely::dataflow::{Scope, StreamVec}; use timely::dataflow::channels::pact::{Exchange, Pipeline}; use timely::dataflow::operators::generic::{builder_rc::OperatorBuilder, OutputBuilder}; @@ -574,7 +574,7 @@ pub mod sink { /// performed before calling the method, the recorded output may not be correctly /// reconstructed by readers. pub fn build( - stream: &Stream, + stream: StreamVec, sink_hash: u64, updates_sink: Weak>, progress_sink: Weak>, @@ -590,7 +590,7 @@ pub mod sink { // and so any record we see is in fact guaranteed to happen. let mut builder = OperatorBuilder::new("UpdatesWriter".to_owned(), stream.scope()); let reactivator = stream.scope().activator_for(builder.operator_info().address); - let mut input = builder.new_input(stream, Pipeline); + let mut input = builder.new_input(stream.clone(), Pipeline); let (updates_out, updates) = builder.new_output(); let mut updates_out = OutputBuilder::from(updates_out); @@ -645,7 +645,7 @@ pub mod sink { // We use a lower-level builder here to get access to the operator address, for rescheduling. let mut builder = OperatorBuilder::new("ProgressWriter".to_owned(), stream.scope()); let reactivator = stream.scope().activator_for(builder.operator_info().address); - let mut input = builder.new_input(&updates, Exchange::new(move |_| sink_hash)); + let mut input = builder.new_input(updates, Exchange::new(move |_| sink_hash)); let should_write = stream.scope().index() == (sink_hash as usize) % stream.scope().peers(); // We now record the numbers of updates at each timestamp between lower and upper bounds. @@ -742,7 +742,7 @@ pub mod sink { // use crate::lattice::Lattice; // /// Creates a Kafka source from supplied configuration information. -// pub fn create_source(scope: G, addr: &str, topic: &str, group: &str) -> (Box, Stream) +// pub fn create_source(scope: G, addr: &str, topic: &str, group: &str) -> (Box, StreamVec) // where // G: Scope, // D: ExchangeData + Hash + for<'a> serde::Deserialize<'a>, @@ -757,7 +757,7 @@ pub mod sink { // }) // } -// pub fn create_sink(stream: &Stream, addr: &str, topic: &str) -> Box +// pub fn create_sink(stream: &StreamVec, addr: &str, topic: &str) -> Box // where // G: Scope, // D: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index 6fc091cab..6799d4d4d 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -8,12 +8,12 @@ //! manually. The higher-level of programming allows differential dataflow to provide efficient //! implementations, and to support efficient incremental updates to the collections. -use timely::{Container, Data}; +use timely::Container; use timely::progress::Timestamp; use timely::dataflow::scopes::Child; use timely::dataflow::Scope; use timely::dataflow::operators::*; -use timely::dataflow::StreamCore; +use timely::dataflow::Stream as StreamCore; use crate::difference::Abelian; @@ -23,7 +23,7 @@ use crate::difference::Abelian; /// in order to expose some of this functionality (e.g. negation, timestamp manipulation). Other actions /// on the containers, and streams of containers, are left to the container implementor to describe. #[derive(Clone)] -pub struct Collection { +pub struct Collection { /// The underlying timely dataflow stream. /// /// This field is exposed to support direct timely dataflow manipulation when required, but it is @@ -32,7 +32,7 @@ pub struct Collection { /// The timestamp in the data is required to always be at least the timestamp _of_ the data, in /// the timely-dataflow sense. If this invariant is not upheld, differential operators may behave /// unexpectedly. - pub inner: timely::dataflow::StreamCore, + pub inner: StreamCore, } impl Collection { @@ -63,16 +63,16 @@ impl Collection { /// /// let data = scope.new_collection_from(1 .. 10).1; /// - /// let odds = data.filter(|x| x % 2 == 1); - /// let evens = data.filter(|x| x % 2 == 0); + /// let odds = data.clone().filter(|x| x % 2 == 1); + /// let evens = data.clone().filter(|x| x % 2 == 0); /// - /// odds.concat(&evens) - /// .assert_eq(&data); + /// odds.concat(evens) + /// .assert_eq(data); /// }); /// ``` - pub fn concat(&self, other: &Self) -> Self { + pub fn concat(self, other: Self) -> Self { self.inner - .concat(&other.inner) + .concat(other.inner) .as_collection() } /// Creates a new collection accumulating the contents of the two collections. @@ -90,26 +90,27 @@ impl Collection { /// /// let data = scope.new_collection_from(1 .. 10).1; /// - /// let odds = data.filter(|x| x % 2 == 1); - /// let evens = data.filter(|x| x % 2 == 0); + /// let odds = data.clone().filter(|x| x % 2 == 1); + /// let evens = data.clone().filter(|x| x % 2 == 0); /// /// odds.concatenate(Some(evens)) - /// .assert_eq(&data); + /// .assert_eq(data); /// }); /// ``` - pub fn concatenate(&self, sources: I) -> Self + pub fn concatenate(self, sources: I) -> Self where I: IntoIterator { self.inner - .concatenate(sources.into_iter().map(|x| x.inner)) + .scope() + .concatenate(sources.into_iter().map(|x| x.inner).chain([self.inner])) .as_collection() } // Brings a Collection into a nested region. /// /// This method is a specialization of `enter` to the case where the nested scope is a region. /// It removes the need for an operator that adjusts the timestamp. - pub fn enter_region<'a>(&self, child: &Child<'a, G, ::Timestamp>) -> Collection::Timestamp>, C> { + pub fn enter_region<'a>(self, child: &Child<'a, G, ::Timestamp>) -> Collection::Timestamp>, C> { self.inner .enter(child) .as_collection() @@ -132,7 +133,7 @@ impl Collection { /// .inspect_container(|event| println!("event: {:?}", event)); /// }); /// ``` - pub fn inspect_container(&self, func: F) -> Self + pub fn inspect_container(self, func: F) -> Self where F: FnMut(Result<(&G::Timestamp, &C), &[G::Timestamp]>)+'static, { @@ -144,7 +145,7 @@ impl Collection { /// /// This probe is used to determine when the state of the Collection has stabilized and can /// be read out. - pub fn probe(&self) -> probe::Handle { + pub fn probe(self) -> probe::Handle { self.inner .probe() } @@ -154,7 +155,7 @@ impl Collection { /// In addition, a probe is also often use to limit the number of rounds of input in flight at any moment; a /// computation can wait until the probe has caught up to the input before introducing more rounds of data, to /// avoid swamping the system. - pub fn probe_with(&self, handle: &probe::Handle) -> Self { + pub fn probe_with(self, handle: &probe::Handle) -> Self { Self::new(self.inner.probe_with(handle)) } /// The scope containing the underlying timely dataflow stream. @@ -177,15 +178,15 @@ impl Collection { /// /// let data = scope.new_collection_from(1 .. 10).1; /// - /// let odds = data.filter(|x| x % 2 == 1); - /// let evens = data.filter(|x| x % 2 == 0); + /// let odds = data.clone().filter(|x| x % 2 == 1); + /// let evens = data.clone().filter(|x| x % 2 == 0); /// /// odds.negate() - /// .concat(&data) - /// .assert_eq(&evens); + /// .concat(data) + /// .assert_eq(evens); /// }); /// ``` - pub fn negate(&self) -> Self where C: containers::Negate { + pub fn negate(self) -> Self where C: containers::Negate { use timely::dataflow::channels::pact::Pipeline; self.inner .unary(Pipeline, "Negate", move |_,_| move |input, output| { @@ -207,14 +208,15 @@ impl Collection { /// let data = scope.new_collection_from(1 .. 10).1; /// /// let result = scope.region(|child| { - /// data.enter(child) + /// data.clone() + /// .enter(child) /// .leave() /// }); /// - /// data.assert_eq(&result); + /// data.assert_eq(result); /// }); /// ``` - pub fn enter<'a, T>(&self, child: &Child<'a, G, T>) -> Collection, ::Timestamp, T>>::InnerContainer> + pub fn enter<'a, T>(self, child: &Child<'a, G, T>) -> Collection, ::Timestamp, T>>::InnerContainer> where C: containers::Enter<::Timestamp, T, InnerContainer: Container>, T: Refines<::Timestamp>, @@ -236,7 +238,7 @@ impl Collection { /// # Examples /// ``` /// use timely::dataflow::Scope; - /// use timely::dataflow::operators::{ToStream, Concat, Inspect, BranchWhen}; + /// use timely::dataflow::operators::{ToStream, Concat, Inspect, vec::BranchWhen}; /// /// use differential_dataflow::input::Input; /// @@ -248,7 +250,7 @@ impl Collection { /// data.results_in(summary1); /// }); /// ``` - pub fn results_in(&self, step: ::Summary) -> Self + pub fn results_in(self, step: ::Summary) -> Self where C: containers::ResultsIn<::Summary>, { @@ -283,14 +285,15 @@ where /// let data = scope.new_collection_from(1 .. 10).1; /// /// let result = scope.region(|child| { - /// data.enter(child) + /// data.clone() + /// .enter(child) /// .leave() /// }); /// - /// data.assert_eq(&result); + /// data.assert_eq(result); /// }); /// ``` - pub fn leave(&self) -> Collection>::OuterContainer> { + pub fn leave(self) -> Collection>::OuterContainer> { use timely::dataflow::channels::pact::Pipeline; self.inner .leave() @@ -302,13 +305,13 @@ where } /// Methods requiring a region as the scope. -impl Collection, C> +impl Collection, C> { /// Returns the value of a Collection from a nested region to its containing scope. /// /// This method is a specialization of `leave` to the case that of a nested region. /// It removes the need for an operator that adjusts the timestamp. - pub fn leave_region(&self) -> Collection { + pub fn leave_region(self) -> Collection { self.inner .leave() .as_collection() @@ -321,12 +324,12 @@ pub mod vec { use std::hash::Hash; - use timely::Data; use timely::progress::Timestamp; use timely::order::Product; use timely::dataflow::scopes::child::Iterative; use timely::dataflow::{Scope, ScopeParent}; use timely::dataflow::operators::*; + use timely::dataflow::operators::vec::*; use crate::collection::AsCollection; use crate::difference::{Semigroup, Abelian, Multiply}; @@ -368,9 +371,9 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn map(&self, mut logic: L) -> Collection + pub fn map(self, mut logic: L) -> Collection where - D2: Data, + D2: Clone+'static, L: FnMut(D) -> D2 + 'static, { self.inner @@ -395,7 +398,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn map_in_place(&self, mut logic: L) -> Collection + pub fn map_in_place(self, mut logic: L) -> Collection where L: FnMut(&mut D) + 'static, { @@ -419,10 +422,10 @@ pub mod vec { /// .flat_map(|x| 0 .. x); /// }); /// ``` - pub fn flat_map(&self, mut logic: L) -> Collection + pub fn flat_map(self, mut logic: L) -> Collection where G::Timestamp: Clone, - I: IntoIterator, + I: IntoIterator, L: FnMut(D) -> I + 'static, { self.inner @@ -443,7 +446,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn filter(&self, mut logic: L) -> Collection + pub fn filter(self, mut logic: L) -> Collection where L: FnMut(&D) -> bool + 'static, { @@ -464,16 +467,16 @@ pub mod vec { /// ::timely::example(|scope| { /// /// let nums = scope.new_collection_from(0 .. 10).1; - /// let x1 = nums.flat_map(|x| 0 .. x); + /// let x1 = nums.clone().flat_map(|x| 0 .. x); /// let x2 = nums.map(|x| (x, 9 - x)) /// .explode(|(x,y)| Some((x,y))); /// - /// x1.assert_eq(&x2); + /// x1.assert_eq(x2); /// }); /// ``` - pub fn explode(&self, mut logic: L) -> Collection>::Output> + pub fn explode(self, mut logic: L) -> Collection>::Output> where - D2: Data, + D2: Clone+'static, R2: Semigroup+Multiply, I: IntoIterator, L: FnMut(D)->I+'static, @@ -505,10 +508,10 @@ pub mod vec { /// ); /// }); /// ``` - pub fn join_function(&self, mut logic: L) -> Collection>::Output> + pub fn join_function(self, mut logic: L) -> Collection>::Output> where G::Timestamp: Lattice, - D2: Data, + D2: Clone+'static, R2: Semigroup+Multiply, I: IntoIterator, L: FnMut(D)->I+'static, @@ -533,14 +536,15 @@ pub mod vec { /// let data = scope.new_collection_from(1 .. 10).1; /// /// let result = scope.iterative::(|child| { - /// data.enter_at(child, |x| *x) + /// data.clone() + /// .enter_at(child, |x| *x) /// .leave() /// }); /// - /// data.assert_eq(&result); + /// data.assert_eq(result); /// }); /// ``` - pub fn enter_at<'a, T, F>(&self, child: &Iterative<'a, G, T>, mut initial: F) -> Collection, D, R> + pub fn enter_at<'a, T, F>(self, child: &Iterative<'a, G, T>, mut initial: F) -> Collection, D, R> where T: Timestamp+Hash, F: FnMut(&D) -> T + Clone + 'static, @@ -562,7 +566,7 @@ pub mod vec { /// ordered, they should have the same order or compare equal once `func` is applied to them (this /// is because we advance the timely capability with the same logic, and it must remain `less_equal` /// to all of the data timestamps). - pub fn delay(&self, func: F) -> Collection + pub fn delay(self, func: F) -> Collection where G::Timestamp: Hash, F: FnMut(&G::Timestamp) -> G::Timestamp + Clone + 'static, @@ -599,7 +603,7 @@ pub mod vec { /// .inspect(|x| println!("error: {:?}", x)); /// }); /// ``` - pub fn inspect(&self, func: F) -> Collection + pub fn inspect(self, func: F) -> Collection where F: FnMut(&(D, G::Timestamp, R))+'static, { @@ -625,7 +629,7 @@ pub mod vec { /// .inspect_batch(|t,xs| println!("errors @ {:?}: {:?}", t, xs)); /// }); /// ``` - pub fn inspect_batch(&self, mut func: F) -> Collection + pub fn inspect_batch(self, mut func: F) -> Collection where F: FnMut(&G::Timestamp, &[(D, G::Timestamp, R)])+'static, { @@ -653,7 +657,7 @@ pub mod vec { /// .assert_empty(); /// }); /// ``` - pub fn assert_empty(&self) + pub fn assert_empty(self) where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable + Semigroup, @@ -665,7 +669,7 @@ pub mod vec { } /// Methods requiring an Abelian difference, to support negation. - impl, D: Clone+'static, R: Abelian+'static> Collection { + impl, D: Clone+'static, R: Abelian+'static> Collection { /// Assert if the collections are ever different. /// /// Because this is a dataflow fragment, the test is only applied as the computation is run. If the computation @@ -682,14 +686,14 @@ pub mod vec { /// /// let data = scope.new_collection_from(1 .. 10).1; /// - /// let odds = data.filter(|x| x % 2 == 1); - /// let evens = data.filter(|x| x % 2 == 0); + /// let odds = data.clone().filter(|x| x % 2 == 1); + /// let evens = data.clone().filter(|x| x % 2 == 0); /// - /// odds.concat(&evens) - /// .assert_eq(&data); + /// odds.concat(evens) + /// .assert_eq(data); /// }); /// ``` - pub fn assert_eq(&self, other: &Self) + pub fn assert_eq(self, other: Self) where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable, @@ -738,13 +742,13 @@ pub mod vec { /// }); /// }); /// ``` - pub fn reduce(&self, logic: L) -> Collection + pub fn reduce(self, logic: L) -> Collection where L: FnMut(&K, &[(&V, R)], &mut Vec<(V2, R2)>)+'static { self.reduce_named("Reduce", logic) } /// As `reduce` with the ability to name the operator. - pub fn reduce_named(&self, name: &str, logic: L) -> Collection + pub fn reduce_named(self, name: &str, logic: L) -> Collection where L: FnMut(&K, &[(&V, R)], &mut Vec<(V2, R2)>)+'static { use crate::trace::implementations::{ValBuilder, ValSpine}; @@ -777,7 +781,7 @@ pub mod vec { /// .trace; /// }); /// ``` - pub fn reduce_abelian(&self, name: &str, mut logic: L) -> Arranged> + pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> where T2: for<'a> Trace= &'a K, KeyOwn = K, ValOwn = V, Time=G::Timestamp, Diff: Abelian>+'static, Bu: Builder, Output = T2::Batch>, @@ -795,9 +799,9 @@ pub mod vec { /// Unlike `reduce_arranged`, this method may be called with an empty `input`, /// and it may not be safe to index into the first element. /// At least one of the two collections will be non-empty. - pub fn reduce_core(&self, name: &str, logic: L) -> Arranged> + pub fn reduce_core(self, name: &str, logic: L) -> Arranged> where - V: Data, + V: Clone+'static, T2: for<'a> Trace=&'a K, KeyOwn = K, ValOwn = V, Time=G::Timestamp>+'static, Bu: Builder, Output = T2::Batch>, L: FnMut(&K, &[(&V, R)], &mut Vec<(V,T2::Diff)>, &mut Vec<(V, T2::Diff)>)+'static, @@ -828,7 +832,7 @@ pub mod vec { /// .distinct(); /// }); /// ``` - pub fn distinct(&self) -> Collection { + pub fn distinct(self) -> Collection { self.distinct_core() } @@ -837,7 +841,7 @@ pub mod vec { /// This method allows `distinct` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - pub fn distinct_core>(&self) -> Collection { + pub fn distinct_core>(self) -> Collection { self.threshold_named("Distinct", |_,_| R2::from(1i8)) } @@ -859,12 +863,12 @@ pub mod vec { /// .threshold(|_,c| c % 2); /// }); /// ``` - pub fn thresholdR2+'static>(&self, thresh: F) -> Collection { + pub fn thresholdR2+'static>(self, thresh: F) -> Collection { self.threshold_named("Threshold", thresh) } /// A `threshold` with the ability to name the operator. - pub fn threshold_namedR2+'static>(&self, name: &str, mut thresh: F) -> Collection { + pub fn threshold_namedR2+'static>(self, name: &str, mut thresh: F) -> Collection { use crate::trace::implementations::{KeyBuilder, KeySpine}; self.arrange_by_self_named(&format!("Arrange: {}", name)) @@ -895,14 +899,14 @@ pub mod vec { /// .count(); /// }); /// ``` - pub fn count(&self) -> Collection { self.count_core() } + pub fn count(self) -> Collection { self.count_core() } /// Count for general integer differences. /// /// This method allows `count` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - pub fn count_core + 'static>(&self) -> Collection { + pub fn count_core + 'static>(self) -> Collection { use crate::trace::implementations::{ValBuilder, ValSpine}; self.arrange_by_self_named("Arrange: Count") .reduce_abelian::<_,ValBuilder,ValSpine>("Count", |_k,s,t| t.push((s[0].1.clone(), R2::from(1i8)))) @@ -913,7 +917,7 @@ pub mod vec { /// Methods which require data be arrangeable. impl Collection where - G: Scope, + G: Scope, D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup, { @@ -931,20 +935,21 @@ pub mod vec { /// /// let x = scope.new_collection_from(1 .. 10u32).1; /// - /// x.negate() - /// .concat(&x) + /// x.clone() + /// .negate() + /// .concat(x) /// .consolidate() // <-- ensures cancellation occurs /// .assert_empty(); /// }); /// ``` - pub fn consolidate(&self) -> Self { + pub fn consolidate(self) -> Self { use crate::trace::implementations::{KeyBatcher, KeyBuilder, KeySpine}; self.consolidate_named::,KeyBuilder<_,_,_>, KeySpine<_,_,_>,_>("Consolidate", |key,&()| key.clone()) } /// As `consolidate` but with the ability to name the operator, specify the trace type, /// and provide the function `reify` to produce owned keys and values.. - pub fn consolidate_named(&self, name: &str, reify: F) -> Self + pub fn consolidate_named(self, name: &str, reify: F) -> Self where Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, Tr: for<'a> crate::trace::Trace+'static, @@ -975,12 +980,13 @@ pub mod vec { /// let x = scope.new_collection_from(1 .. 10u32).1; /// /// // nothing to assert, as no particular guarantees. - /// x.negate() - /// .concat(&x) + /// x.clone() + /// .negate() + /// .concat(x) /// .consolidate_stream(); /// }); /// ``` - pub fn consolidate_stream(&self) -> Self { + pub fn consolidate_stream(self) -> Self { use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Operator; @@ -1011,14 +1017,14 @@ pub mod vec { V: crate::ExchangeData, R: crate::ExchangeData + Semigroup, { - fn arrange_named(&self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, Bu: crate::trace::Builder, Tr: crate::trace::Trace + 'static, { let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,V),G::Timestamp,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_, _, Ba, Bu, _>(&self.inner, exchange, name) + crate::operators::arrange::arrangement::arrange_core::<_, _, Ba, Bu, _>(self.inner, exchange, name) } } @@ -1026,14 +1032,14 @@ pub mod vec { where G: Scope, { - fn arrange_named(&self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, Bu: crate::trace::Builder, Tr: crate::trace::Trace + 'static, { let exchange = timely::dataflow::channels::pact::Exchange::new(move |update: &((K,()),G::Timestamp,R)| (update.0).0.hashed().into()); - crate::operators::arrange::arrangement::arrange_core::<_,_,Ba,Bu,_>(&self.map(|k| (k, ())).inner, exchange, name) + crate::operators::arrange::arrangement::arrange_core::<_,_,Ba,Bu,_>(self.map(|k| (k, ())).inner, exchange, name) } } @@ -1047,12 +1053,12 @@ pub mod vec { /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// This trace is current for all times completed by the output stream, which can be used to /// safely identify the stable times and values in the trace. - pub fn arrange_by_key(&self) -> Arranged>> { + pub fn arrange_by_key(self) -> Arranged>> { self.arrange_by_key_named("ArrangeByKey") } /// As `arrange_by_key` but with the ability to name the arrangement. - pub fn arrange_by_key_named(&self, name: &str) -> Arranged>> { + pub fn arrange_by_key_named(self, name: &str) -> Arranged>> { self.arrange_named::,ValBuilder<_,_,_,_>,_>(name) } } @@ -1066,12 +1072,12 @@ pub mod vec { /// This operator arranges a collection of records into a shared trace, whose contents it maintains. /// This trace is current for all times complete in the output stream, which can be used to safely /// identify the stable times and values in the trace. - pub fn arrange_by_self(&self) -> Arranged>> { + pub fn arrange_by_self(self) -> Arranged>> { self.arrange_by_self_named("ArrangeBySelf") } /// As `arrange_by_self` but with the ability to name the arrangement. - pub fn arrange_by_self_named(&self, name: &str) -> Arranged>> { + pub fn arrange_by_self_named(self, name: &str) -> Arranged>> { self.map(|k| (k, ())) .arrange_named::,KeyBuilder<_,_,_>,_>(name) } @@ -1099,11 +1105,11 @@ pub mod vec { /// let y = scope.new_collection_from(vec![(0, 'a'), (1, 'b')]).1; /// let z = scope.new_collection_from(vec![(0, (1, 'a')), (1, (3, 'b'))]).1; /// - /// x.join(&y) - /// .assert_eq(&z); + /// x.join(y) + /// .assert_eq(z); /// }); /// ``` - pub fn join(&self, other: &Collection) -> Collection>::Output> + pub fn join(self, other: Collection) -> Collection>::Output> where K: crate::ExchangeData, V2: crate::ExchangeData, @@ -1126,15 +1132,15 @@ pub mod vec { /// let y = scope.new_collection_from(vec![(0, 'a'), (1, 'b')]).1; /// let z = scope.new_collection_from(vec![(1, 'a'), (3, 'b')]).1; /// - /// x.join_map(&y, |_key, &a, &b| (a,b)) - /// .assert_eq(&z); + /// x.join_map(y, |_key, &a, &b| (a,b)) + /// .assert_eq(z); /// }); /// ``` - pub fn join_map(&self, other: &Collection, mut logic: L) -> Collection>::Output> + pub fn join_map(self, other: Collection, mut logic: L) -> Collection>::Output> where R: Multiply, L: FnMut(&K, &V, &V2)->D+'static { let arranged1 = self.arrange_by_key(); let arranged2 = other.arrange_by_key(); - arranged1.join_core(&arranged2, move |k,v1,v2| Some(logic(k,v1,v2))) + arranged1.join_core(arranged2, move |k,v1,v2| Some(logic(k,v1,v2))) } /// Matches pairs `(key, val)` and `key` based on `key`, producing the former with frequencies multiplied. @@ -1154,15 +1160,15 @@ pub mod vec { /// let y = scope.new_collection_from(vec![0, 2]).1; /// let z = scope.new_collection_from(vec![(0, 1)]).1; /// - /// x.semijoin(&y) - /// .assert_eq(&z); + /// x.semijoin(y) + /// .assert_eq(z); /// }); /// ``` - pub fn semijoin(&self, other: &Collection) -> Collection>::Output> + pub fn semijoin(self, other: Collection) -> Collection>::Output> where R: Multiply { let arranged1 = self.arrange_by_key(); let arranged2 = other.arrange_by_self(); - arranged1.join_core(&arranged2, |k,v,_| Some((k.clone(), v.clone()))) + arranged1.join_core(arranged2, |k,v,_| Some((k.clone(), v.clone()))) } /// Subtracts the semijoin with `other` from `self`. @@ -1186,13 +1192,13 @@ pub mod vec { /// let y = scope.new_collection_from(vec![0, 2]).1; /// let z = scope.new_collection_from(vec![(1, 3)]).1; /// - /// x.antijoin(&y) - /// .assert_eq(&z); + /// x.antijoin(y) + /// .assert_eq(z); /// }); /// ``` - pub fn antijoin(&self, other: &Collection) -> Collection + pub fn antijoin(self, other: Collection) -> Collection where R: Multiply, R: Abelian+'static { - self.concat(&self.semijoin(other).negate()) + self.clone().concat(self.semijoin(other).negate()) } /// Joins two arranged collections with the same key type. @@ -1219,11 +1225,11 @@ pub mod vec { /// /// let z = scope.new_collection_from(vec![(1, 'a'), (3, 'b')]).1; /// - /// x.join_core(&y, |_key, &a, &b| Some((a, b))) - /// .assert_eq(&z); + /// x.join_core(y, |_key, &a, &b| Some((a, b))) + /// .assert_eq(z); /// }); /// ``` - pub fn join_core (&self, stream2: &Arranged, result: L) -> Collection>::Output> + pub fn join_core (self, stream2: Arranged, result: L) -> Collection>::Output> where Tr2: for<'a> crate::trace::TraceReader=&'a K, Time=G::Timestamp>+Clone+'static, R: Multiply, @@ -1239,16 +1245,16 @@ pub mod vec { /// Conversion to a differential dataflow Collection. pub trait AsCollection { /// Converts the type to a differential dataflow collection. - fn as_collection(&self) -> Collection; + fn as_collection(self) -> Collection; } -impl AsCollection for StreamCore { +impl AsCollection for StreamCore { /// Converts the type to a differential dataflow collection. /// /// By calling this method, you guarantee that the timestamp invariant (as documented on /// [Collection]) is upheld. This method will not check it. - fn as_collection(&self) -> Collection { - Collection::::new(self.clone()) + fn as_collection(self) -> Collection { + Collection::::new(self) } } @@ -1266,11 +1272,11 @@ impl AsCollection for StreamCore { /// /// let data = scope.new_collection_from(1 .. 10).1; /// -/// let odds = data.filter(|x| x % 2 == 1); -/// let evens = data.filter(|x| x % 2 == 0); +/// let odds = data.clone().filter(|x| x % 2 == 1); +/// let evens = data.clone().filter(|x| x % 2 == 0); /// /// differential_dataflow::collection::concatenate(scope, vec![odds, evens]) -/// .assert_eq(&data); +/// .assert_eq(data); /// }); /// ``` pub fn concatenate(scope: &mut G, iterator: I) -> Collection diff --git a/differential-dataflow/src/dynamic/mod.rs b/differential-dataflow/src/dynamic/mod.rs index 27217b778..19c5472a6 100644 --- a/differential-dataflow/src/dynamic/mod.rs +++ b/differential-dataflow/src/dynamic/mod.rs @@ -35,16 +35,16 @@ where TOuter: Timestamp, { /// Enters a dynamically created scope which has `level` timestamp coordinates. - pub fn enter_dynamic(&self, _level: usize) -> Self { - (*self).clone() + pub fn enter_dynamic(self, _level: usize) -> Self { + self } /// Leaves a dynamically created scope which has `level` timestamp coordinates. - pub fn leave_dynamic(&self, level: usize) -> Self { + pub fn leave_dynamic(self, level: usize) -> Self { // Create a unary operator that will strip all but `level-1` timestamp coordinates. let mut builder = OperatorBuilder::new("LeaveDynamic".to_string(), self.scope()); let (output, stream) = builder.new_output(); let mut output = OutputBuilder::from(output); - let mut input = builder.new_input_connection(&self.inner, Pipeline, [(0, Antichain::from_elem(Product { outer: Default::default(), inner: PointStampSummary { retain: Some(level - 1), actions: Vec::new() } }))]); + let mut input = builder.new_input_connection(self.inner, Pipeline, [(0, Antichain::from_elem(Product { outer: Default::default(), inner: PointStampSummary { retain: Some(level - 1), actions: Vec::new() } }))]); builder.build(move |_capability| move |_frontier| { let mut output = output.activate(); @@ -53,7 +53,7 @@ where let mut vec = std::mem::take(&mut new_time.inner).into_inner(); vec.truncate(level - 1); new_time.inner = PointStamp::new(vec); - let new_cap = cap.delayed(&new_time); + let new_cap = cap.delayed(&new_time, 0); for (_data, time, _diff) in data.iter_mut() { let mut vec = std::mem::take(&mut time.inner).into_inner(); vec.truncate(level - 1); diff --git a/differential-dataflow/src/input.rs b/differential-dataflow/src/input.rs index ae0f69eb7..df21349cb 100644 --- a/differential-dataflow/src/input.rs +++ b/differential-dataflow/src/input.rs @@ -7,8 +7,8 @@ //! than are evident from the logical times, which appear to execute in sequence. use timely::progress::Timestamp; -use timely::dataflow::operators::Input as TimelyInput; -use timely::dataflow::operators::input::Handle; +use timely::dataflow::operators::vec::Input as TimelyInput; +use timely::dataflow::operators::vec::input::Handle; use timely::dataflow::scopes::ScopeParent; use crate::Data; @@ -121,7 +121,7 @@ impl Input for G where ::Timestamp: Lattice { let (handle, stream) = self.new_input(); let source = data.to_stream(self).as_collection(); - (InputSession::from(handle), stream.as_collection().concat(&source)) + (InputSession::from(handle), stream.as_collection().concat(source)) }} /// An input session wrapping a single timely dataflow capability. diff --git a/differential-dataflow/src/lib.rs b/differential-dataflow/src/lib.rs index a2a265664..615aa5ded 100644 --- a/differential-dataflow/src/lib.rs +++ b/differential-dataflow/src/lib.rs @@ -85,12 +85,12 @@ pub use difference::Abelian as Diff; /// Most differential dataflow operators require the ability to cancel corresponding updates, and the /// way that they do this is by putting the data in a canonical form. The `Ord` trait allows us to sort /// the data, at which point we can consolidate updates for equivalent records. -pub trait Data : timely::Data + Ord + Debug { } -impl Data for T { } +pub trait Data : Ord + Debug + Clone + 'static { } +impl Data for T { } /// Data types exchangeable in differential dataflow. -pub trait ExchangeData : timely::ExchangeData + Ord + Debug { } -impl ExchangeData for T { } +pub trait ExchangeData : timely::ExchangeData + Data { } +impl ExchangeData for T { } pub mod hashable; pub mod operators; diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index 8a1a1fbe3..87517a659 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -17,9 +17,11 @@ //! see ill-defined data at times for which the trace is not complete. (All current implementations //! commit only completed data to the trace). -use timely::dataflow::operators::{Enter, Map}; +use timely::dataflow::operators::{Enter, vec::Map}; use timely::order::PartialOrder; -use timely::dataflow::{Scope, Stream, StreamCore}; +use timely::dataflow::Stream as StreamCore; +use timely::dataflow::StreamVec as Stream; +use timely::dataflow::Scope; use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::progress::Timestamp; @@ -87,14 +89,14 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter<'a, TInner>(&self, child: &Child<'a, G, TInner>) + pub fn enter<'a, TInner>(self, child: &Child<'a, G, TInner>) -> Arranged, TraceEnter> where TInner: Refines+Lattice+Timestamp+Clone, { Arranged { stream: self.stream.enter(child).map(|bw| BatchEnter::make_from(bw)), - trace: TraceEnter::make_from(self.trace.clone()), + trace: TraceEnter::make_from(self.trace), } } @@ -102,11 +104,11 @@ where /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn enter_region<'a>(&self, child: &Child<'a, G, G::Timestamp>) + pub fn enter_region<'a>(self, child: &Child<'a, G, G::Timestamp>) -> Arranged, Tr> { Arranged { stream: self.stream.enter(child), - trace: self.trace.clone(), + trace: self.trace, } } @@ -115,7 +117,7 @@ where /// This method produces a proxy trace handle that uses the same backing data, but acts as if the timestamps /// have all been extended with an additional coordinate with the default value. The resulting collection does /// not vary with the new timestamp coordinate. - pub fn enter_at<'a, TInner, F, P>(&self, child: &Child<'a, G, TInner>, logic: F, prior: P) + pub fn enter_at<'a, TInner, F, P>(self, child: &Child<'a, G, TInner>, logic: F, prior: P) -> Arranged, TraceEnterAt> where TInner: Refines+Lattice+Timestamp+Clone+'static, @@ -125,7 +127,7 @@ where let logic1 = logic.clone(); let logic2 = logic.clone(); Arranged { - trace: TraceEnterAt::make_from(self.trace.clone(), logic1, prior), + trace: TraceEnterAt::make_from(self.trace, logic1, prior), stream: self.stream.enter(child).map(move |bw| BatchEnterAt::make_from(bw, logic2.clone())), } } @@ -135,7 +137,7 @@ where /// The underlying `Stream>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. - pub fn as_collection(&self, mut logic: L) -> VecCollection + pub fn as_collection(self, mut logic: L) -> VecCollection where L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> D+'static, { @@ -147,7 +149,7 @@ where /// The underlying `Stream>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. - pub fn as_vecs(&self) -> VecCollection + pub fn as_vecs(self) -> VecCollection where Tr::KeyOwn: crate::ExchangeData, Tr::ValOwn: crate::ExchangeData, @@ -159,12 +161,12 @@ where /// /// The supplied logic may produce an iterator over output values, allowing either /// filtering or flat mapping as part of the extraction. - pub fn flat_map_ref(&self, logic: L) -> VecCollection + pub fn flat_map_ref(self, logic: L) -> VecCollection where I: IntoIterator, L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, { - Self::flat_map_batches(&self.stream, logic) + Self::flat_map_batches(self.stream, logic) } /// Extracts elements from a stream of batches as a collection. @@ -174,7 +176,7 @@ where /// /// This method exists for streams of batches without the corresponding arrangement. /// If you have the arrangement, its `flat_map_ref` method is equivalent to this. - pub fn flat_map_batches(stream: &Stream, mut logic: L) -> VecCollection + pub fn flat_map_batches(stream: Stream, mut logic: L) -> VecCollection where I: IntoIterator, L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, @@ -214,7 +216,7 @@ where /// A convenience method to join and produce `VecCollection` output. /// /// Avoid this method, as it is likely to evolve into one without the `VecCollection` opinion. - pub fn join_core(&self, other: &Arranged, mut result: L) -> VecCollection>::Output> + pub fn join_core(self, other: Arranged, mut result: L) -> VecCollection>::Output> where T2: for<'a> TraceReader=T1::Key<'a>,Time=T1::Time>+Clone+'static, T1::Diff: Multiply, @@ -249,7 +251,7 @@ where T1: TraceReader + Clone + 'static, { /// A direct implementation of `ReduceCore::reduce_abelian`. - pub fn reduce_abelian(&self, name: &str, mut logic: L) -> Arranged> + pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> where T1: TraceReader, T2: for<'a> Trace< @@ -272,7 +274,7 @@ where } /// A direct implementation of `ReduceCore::reduce_core`. - pub fn reduce_core(&self, name: &str, logic: L) -> Arranged> + pub fn reduce_core(self, name: &str, logic: L) -> Arranged> where T1: TraceReader, T2: for<'a> Trace< @@ -299,22 +301,22 @@ where /// /// This method only applies to *regions*, which are subscopes with the same timestamp /// as their containing scope. In this case, the trace type does not need to change. - pub fn leave_region(&self) -> Arranged { + pub fn leave_region(self) -> Arranged { use timely::dataflow::operators::Leave; Arranged { stream: self.stream.leave(), - trace: self.trace.clone(), + trace: self.trace, } } } /// A type that can be arranged as if a collection of updates. -pub trait Arrange +pub trait Arrange : Sized where G: Scope, { /// Arranges updates into a shared trace. - fn arrange(&self) -> Arranged> + fn arrange(self) -> Arranged> where Ba: Batcher + 'static, Bu: Builder, @@ -324,7 +326,7 @@ where } /// Arranges updates into a shared trace, with a supplied name. - fn arrange_named(&self, name: &str) -> Arranged> + fn arrange_named(self, name: &str) -> Arranged> where Ba: Batcher + 'static, Bu: Builder, @@ -337,7 +339,7 @@ where /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// It uses the supplied parallelization contract to distribute the data, which does not need to /// be consistently by key (though this is the most common). -pub fn arrange_core(stream: &StreamCore, pact: P, name: &str) -> Arranged> +pub fn arrange_core(stream: StreamCore, pact: P, name: &str) -> Arranged> where G: Scope, P: ParallelizationContract, @@ -398,7 +400,7 @@ where // when we realize that time intervals are complete. input.for_each(|cap, data| { - capabilities.insert(cap.retain()); + capabilities.insert(cap.retain(0)); batcher.push_container(data); }); diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index a2aaed772..54f6e980d 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -59,7 +59,7 @@ //! use differential_dataflow::operators::arrange::upsert; //! //! let stream = scope.input_from(&mut input); -//! let arranged = upsert::arrange_from_upsert::<_, ValBuilder, ValSpine>(&stream, &"test"); +//! let arranged = upsert::arrange_from_upsert::<_, ValBuilder, ValSpine>(stream, &"test"); //! //! arranged //! .as_collection(|k,v| (k.clone(), v.clone())) @@ -101,7 +101,7 @@ use std::collections::{BinaryHeap, BTreeMap}; use timely::order::{PartialOrder, TotalOrder}; -use timely::dataflow::{Scope, Stream}; +use timely::dataflow::{Scope, StreamVec as Stream}; use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::Exchange; use timely::progress::Timestamp; @@ -128,7 +128,7 @@ use super::TraceAgent; /// understand what a "sequence" of upserts would mean for partially ordered /// timestamps. pub fn arrange_from_upsert( - stream: &Stream, G::Timestamp)>, + stream: Stream, G::Timestamp)>, name: &str, ) -> Arranged> where @@ -150,18 +150,19 @@ where let exchange = Exchange::new(move |update: &(Tr::KeyOwn,Option,G::Timestamp)| (update.0).hashed().into()); + let scope = stream.scope(); stream.unary_frontier(exchange, name, move |_capability, info| { // Acquire a logger for arrange events. - let logger = stream.scope().logger_for::("differential/arrange").map(Into::into); + let logger = scope.logger_for::("differential/arrange").map(Into::into); // Tracks the lower envelope of times in `priority_queue`. let mut capabilities = Antichain::>::new(); // Form the trace we will both use internally and publish. - let activator = Some(stream.scope().activator_for(info.address.clone())); + let activator = Some(scope.activator_for(info.address.clone())); let mut empty_trace = Tr::new(info.clone(), logger.clone(), activator); - if let Some(exert_logic) = stream.scope().config().get::("differential/default_exert_logic").cloned() { + if let Some(exert_logic) = scope.config().get::("differential/default_exert_logic").cloned() { empty_trace.set_exert_logic(exert_logic); } @@ -180,7 +181,7 @@ where // Stash capabilities and associated data (ordered by time). input.for_each(|cap, data| { - capabilities.insert(cap.retain()); + capabilities.insert(cap.retain(0)); for (key, val, time) in data.drain(..) { priority_queue.push(std::cmp::Reverse((time, key, val))) } diff --git a/differential-dataflow/src/operators/count.rs b/differential-dataflow/src/operators/count.rs index c7455bc3e..713c73c86 100644 --- a/differential-dataflow/src/operators/count.rs +++ b/differential-dataflow/src/operators/count.rs @@ -14,7 +14,7 @@ use crate::operators::arrange::Arranged; use crate::trace::{BatchReader, Cursor, TraceReader}; /// Extension trait for the `count` differential dataflow method. -pub trait CountTotal, K: ExchangeData, R: Semigroup> { +pub trait CountTotal, K: ExchangeData, R: Semigroup> : Sized { /// Counts the number of occurrences of each element. /// /// # Examples @@ -30,7 +30,7 @@ pub trait CountTotal, K: ExchangeDat /// .count_total(); /// }); /// ``` - fn count_total(&self) -> VecCollection { + fn count_total(self) -> VecCollection { self.count_total_core() } @@ -39,14 +39,14 @@ pub trait CountTotal, K: ExchangeDat /// This method allows `count_total` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - fn count_total_core + 'static>(&self) -> VecCollection; + fn count_total_core + 'static>(self) -> VecCollection; } impl CountTotal for VecCollection where G: Scope, { - fn count_total_core + 'static>(&self) -> VecCollection { + fn count_total_core + 'static>(self) -> VecCollection { self.arrange_by_self_named("Arrange: CountTotal") .count_total_core() } @@ -63,7 +63,7 @@ where >+Clone+'static, K: ExchangeData, { - fn count_total_core + 'static>(&self) -> VecCollection { + fn count_total_core + 'static>(self) -> VecCollection { let mut trace = self.trace.clone(); @@ -85,7 +85,7 @@ where let mut cap = None; input.for_each(|capability, batches| { if cap.is_none() { // NB: Assumes batches are in-order - cap = Some(capability.retain()); + cap = Some(capability.retain(0)); } for batch in batches.drain(..) { upper_limit.clone_from(batch.upper()); // NB: Assumes batches are in-order diff --git a/differential-dataflow/src/operators/iterate.rs b/differential-dataflow/src/operators/iterate.rs index 5dc11042a..35d4b6e3b 100644 --- a/differential-dataflow/src/operators/iterate.rs +++ b/differential-dataflow/src/operators/iterate.rs @@ -71,15 +71,15 @@ pub trait Iterate, D: Data, R: Semigroup> { /// }); /// }); /// ``` - fn iterate(&self, logic: F) -> VecCollection + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(&VecCollection, D, R>)->VecCollection, D, R>; + for<'a> F: FnOnce(VecCollection, D, R>)->VecCollection, D, R>; } impl, D: Ord+Data+Debug, R: Abelian+'static> Iterate for VecCollection { - fn iterate(&self, logic: F) -> VecCollection + fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(&VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(VecCollection, D, R>)->VecCollection, D, R>, { self.inner.scope().scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. @@ -89,22 +89,19 @@ impl, D: Ord+Data+Debug, R: Abelian+'static> Iterat // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let variable = Variable::new_from(self.enter(subgraph), Product::new(Default::default(), 1)); - let result = logic(&variable); - variable.set(&result); + let result = logic(variable.clone()); + variable.set(result.clone()); result.leave() }) } } impl, D: Ord+Data+Debug, R: Semigroup+'static> Iterate for G { - fn iterate(&self, logic: F) -> VecCollection + fn iterate(mut self, logic: F) -> VecCollection where - for<'a> F: FnOnce(&VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(VecCollection, D, R>)->VecCollection, D, R>, { - // TODO: This makes me think we have the wrong ownership pattern here. - let mut clone = self.clone(); - clone - .scoped("Iterate", |subgraph| { + self.scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. // // this could be much more succinct if we returned the collection @@ -112,8 +109,8 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let variable = SemigroupVariable::new(subgraph, Product::new(Default::default(), 1)); - let result = logic(&variable); - variable.set(&result); + let result = logic(variable.clone()); + variable.set(result.clone()); result.leave() } ) @@ -144,9 +141,10 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter /// scope.iterative::(|nested| { /// let summary = Product::new(Default::default(), 1); /// let variable = Variable::new_from(numbers.enter(nested), summary); -/// let result = variable.map(|x| if x % 2 == 0 { x/2 } else { x }) +/// let result = variable.clone() +/// .map(|x| if x % 2 == 0 { x/2 } else { x }) /// .consolidate(); -/// variable.set(&result) +/// variable.set(result) /// .leave() /// }); /// }) @@ -165,7 +163,7 @@ where /// A `Variable` specialized to a vector container of update triples (data, time, diff). pub type VecVariable = Variable::Timestamp, R)>>; -impl Variable +impl Variable where G: Scope, C: crate::collection::containers::Negate + crate::collection::containers::ResultsIn<::Summary>, @@ -183,7 +181,7 @@ where /// Creates a new `Variable` from a supplied `source` stream. pub fn new_from(source: Collection, step: ::Summary) -> Self { let (feedback, updates) = source.inner.scope().feedback(step.clone()); - let collection = Collection::::new(updates).concat(&source); + let collection = Collection::::new(updates).concat(source.clone()); Variable { collection, feedback, source: Some(source), step } } @@ -191,12 +189,12 @@ where /// /// This method binds the `Variable` to be equal to the supplied collection, /// which may be recursively defined in terms of the variable itself. - pub fn set(self, result: &Collection) -> Collection { + pub fn set(self, result: Collection) -> Collection where C: Clone { let mut in_result = result.clone(); - if let Some(source) = &self.source { - in_result = in_result.concat(&source.negate()); + if let Some(source) = self.source.clone() { + in_result = in_result.concat(source.negate()); } - self.set_concat(&in_result) + self.set_concat(in_result) } /// Set the definition of the `Variable` to a collection concatenated to `self`. @@ -208,7 +206,7 @@ where /// /// This behavior can also be achieved by using `new` to create an empty initial /// collection, and then using `self.set(self.concat(result))`. - pub fn set_concat(self, result: &Collection) -> Collection { + pub fn set_concat(self, result: Collection) -> Collection { let step = self.step; result .results_in(step) @@ -255,7 +253,7 @@ where } /// Adds a new source of data to `self`. - pub fn set(self, result: &Collection) -> Collection { + pub fn set(self, result: Collection) -> Collection { let step = self.step; result .results_in(step) diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index 65d353d5c..20b42d67e 100644 --- a/differential-dataflow/src/operators/join.rs +++ b/differential-dataflow/src/operators/join.rs @@ -9,7 +9,8 @@ use timely::{Accountable, ContainerBuilder}; use timely::container::PushInto; use timely::order::PartialOrder; use timely::progress::Timestamp; -use timely::dataflow::{Scope, StreamCore}; +use timely::dataflow::Stream as StreamCore; +use timely::dataflow::Scope; use timely::dataflow::operators::generic::{Operator, OutputBuilderSession, Session}; use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Capability; @@ -66,7 +67,7 @@ impl, D> PushInto for EffortBuilder { /// The "correctness" of this method depends heavily on the behavior of the supplied `result` function. /// /// [`AsCollection`]: crate::collection::AsCollection -pub fn join_traces(arranged1: &Arranged, arranged2: &Arranged, mut result: L) -> StreamCore +pub fn join_traces(arranged1: Arranged, arranged2: Arranged, mut result: L) -> StreamCore where G: Scope, T1: TraceReader+Clone+'static, @@ -78,7 +79,7 @@ where let mut trace1 = arranged1.trace.clone(); let mut trace2 = arranged2.trace.clone(); - arranged1.stream.binary_frontier(&arranged2.stream, Pipeline, Pipeline, "Join", move |capability, info| { + arranged1.stream.clone().binary_frontier(arranged2.stream.clone(), Pipeline, Pipeline, "Join", move |capability, info| { // Acquire an activator to reschedule the operator when it has unfinished work. use timely::scheduling::Activator; @@ -165,7 +166,7 @@ where input1.for_each(|capability, data| { // This test *should* always pass, as we only drop a trace in response to the other input emptying. if let Some(ref mut trace2) = trace2_option { - let capability = capability.retain(); + let capability = capability.retain(0); for batch1 in data.drain(..) { // Ignore any pre-loaded data. if PartialOrder::less_equal(&acknowledged1, batch1.lower()) { @@ -192,7 +193,7 @@ where input2.for_each(|capability, data| { // This test *should* always pass, as we only drop a trace in response to the other input emptying. if let Some(ref mut trace1) = trace1_option { - let capability = capability.retain(); + let capability = capability.retain(0); for batch2 in data.drain(..) { // Ignore any pre-loaded data. if PartialOrder::less_equal(&acknowledged2, batch2.lower()) { diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 7423d6e02..33e640def 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -26,7 +26,7 @@ use crate::trace::TraceReader; /// A key-wise reduction of values in an input trace. /// /// This method exists to provide reduce functionality without opinions about qualifying trace types. -pub fn reduce_trace(trace: &Arranged, name: &str, mut logic: L) -> Arranged> +pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L) -> Arranged> where G: Scope, T1: TraceReader + Clone + 'static, @@ -40,15 +40,16 @@ where let stream = { let result_trace = &mut result_trace; + let scope = trace.stream.scope(); trace.stream.unary_frontier(Pipeline, name, move |_capability, operator_info| { // Acquire a logger for arrange events. - let logger = trace.stream.scope().logger_for::("differential/arrange").map(Into::into); + let logger = scope.logger_for::("differential/arrange").map(Into::into); - let activator = Some(trace.stream.scope().activator_for(operator_info.address.clone())); + let activator = Some(scope.activator_for(operator_info.address.clone())); let mut empty = T2::new(operator_info.clone(), logger.clone(), activator); // If there is default exert logic set, install it. - if let Some(exert_logic) = trace.stream.scope().config().get::("differential/default_exert_logic").cloned() { + if let Some(exert_logic) = scope.config().get::("differential/default_exert_logic").cloned() { empty.set_exert_logic(exert_logic); } @@ -80,7 +81,7 @@ where let mut output_upper = Antichain::from_elem(::minimum()); let mut output_lower = Antichain::from_elem(::minimum()); - let id = trace.stream.scope().index(); + let id = scope.index(); move |(input, _frontier), output| { @@ -123,7 +124,7 @@ where // Ensure that `capabilities` covers the capability of the batch. capabilities.retain(|cap| !capability.time().less_than(cap.time())); if !capabilities.iter().any(|cap| cap.time().less_equal(capability.time())) { - capabilities.push(capability.retain()); + capabilities.push(capability.retain(0)); } }); @@ -654,7 +655,7 @@ mod history_replay { // self.output_buffer.push(((*value).clone(), -diff)); // } // else { - // self.temporary.push(next_time.join(&time)); + // self.temporary.push(next_time.join(time)); // } // } diff --git a/differential-dataflow/src/operators/threshold.rs b/differential-dataflow/src/operators/threshold.rs index 39d5b9041..91bbd16fe 100644 --- a/differential-dataflow/src/operators/threshold.rs +++ b/differential-dataflow/src/operators/threshold.rs @@ -17,9 +17,9 @@ use crate::operators::arrange::Arranged; use crate::trace::{BatchReader, Cursor, TraceReader}; /// Extension trait for the `distinct` differential dataflow method. -pub trait ThresholdTotal, K: ExchangeData, R: ExchangeData+Semigroup> { +pub trait ThresholdTotal, K: ExchangeData, R: ExchangeData+Semigroup> : Sized { /// Reduces the collection to one occurrence of each distinct element. - fn threshold_semigroup(&self, thresh: F) -> VecCollection + fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, F: FnMut(&K,&R,Option<&R>)->Option+'static, @@ -39,7 +39,7 @@ pub trait ThresholdTotal, K: Exchang /// .threshold_total(|_,c| c % 2); /// }); /// ``` - fn threshold_totalR2+'static>(&self, mut thresh: F) -> VecCollection { + fn threshold_totalR2+'static>(self, mut thresh: F) -> VecCollection { self.threshold_semigroup(move |key, new, old| { let mut new = thresh(key, new); if let Some(old) = old { @@ -69,7 +69,7 @@ pub trait ThresholdTotal, K: Exchang /// .distinct_total(); /// }); /// ``` - fn distinct_total(&self) -> VecCollection { + fn distinct_total(self) -> VecCollection { self.distinct_total_core() } @@ -78,7 +78,7 @@ pub trait ThresholdTotal, K: Exchang /// This method allows `distinct` to produce collections whose difference /// type is something other than an `isize` integer, for example perhaps an /// `i32`. - fn distinct_total_core+'static>(&self) -> VecCollection { + fn distinct_total_core+'static>(self) -> VecCollection { self.threshold_total(|_,_| R2::from(1i8)) } @@ -88,7 +88,7 @@ impl ThresholdTot where G: Scope, { - fn threshold_semigroup(&self, thresh: F) -> VecCollection + fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, F: FnMut(&K,&R,Option<&R>)->Option+'static, @@ -109,7 +109,7 @@ where >+Clone+'static, K: ExchangeData, { - fn threshold_semigroup(&self, mut thresh: F) -> VecCollection + fn threshold_semigroup(self, mut thresh: F) -> VecCollection where R2: Semigroup+'static, F: for<'a> FnMut(T1::Key<'a>,&T1::Diff,Option<&T1::Diff>)->Option+'static, @@ -135,7 +135,7 @@ where let mut cap = None; input.for_each(|capability, batches| { if cap.is_none() { // NB: Assumes batches are in-order - cap = Some(capability.retain()); + cap = Some(capability.retain(0)); } for batch in batches.drain(..) { upper_limit.clone_from(batch.upper()); // NB: Assumes batches are in-order diff --git a/differential-dataflow/src/trace/implementations/merge_batcher.rs b/differential-dataflow/src/trace/implementations/merge_batcher.rs index 3ef79f2df..5c932af9f 100644 --- a/differential-dataflow/src/trace/implementations/merge_batcher.rs +++ b/differential-dataflow/src/trace/implementations/merge_batcher.rs @@ -239,7 +239,7 @@ pub mod container { use std::marker::PhantomData; use timely::container::{PushInto, SizableContainer}; use timely::progress::frontier::{Antichain, AntichainRef}; - use timely::{Accountable, Data, PartialOrder}; + use timely::{Accountable, PartialOrder}; use timely::container::DrainContainer; use crate::trace::implementations::merge_batcher::Merger; @@ -325,7 +325,7 @@ pub mod container { impl Merger for ContainerMerger where - for<'a> MC: MergerChunk + Clone + PushInto<::Item<'a>> + 'static, + for<'a> MC: MergerChunk + Clone + PushInto<::Item<'a>> + 'static, CQ: ContainerQueue, { type Time = MC::TimeOwned; diff --git a/differential-dataflow/tests/bfs.rs b/differential-dataflow/tests/bfs.rs index 3678823b0..44fb138c0 100644 --- a/differential-dataflow/tests/bfs.rs +++ b/differential-dataflow/tests/bfs.rs @@ -163,7 +163,7 @@ fn bfs_differential( let (root_input, roots) = scope.new_collection(); let (edge_input, edges) = scope.new_collection(); - bfs(&edges, &roots).map(|(_, dist)| dist) + bfs(edges, roots).map(|(_, dist)| dist) .count() .map(|(x,y)| (x, y as usize)) .inner @@ -202,7 +202,7 @@ fn bfs_differential( } // returns pairs (n, s) indicating node n can be reached from a root in s steps. -fn bfs(edges: &VecCollection, roots: &VecCollection) -> VecCollection +fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where G: Scope, { @@ -210,13 +210,13 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.iterate(|inner| { + nodes.clone().iterate(|inner| { let edges = edges.enter(&inner.scope()); let nodes = nodes.enter(&inner.scope()); - inner.join_map(&edges, |_k,l,d| (*d, l+1)) - .concat(&nodes) + inner.join_map(edges, |_k,l,d| (*d, l+1)) + .concat(nodes) .reduce(|_, s, t| t.push((*s[0].0, 1))) }) } diff --git a/differential-dataflow/tests/import.rs b/differential-dataflow/tests/import.rs index fee3d0696..d8d733e84 100644 --- a/differential-dataflow/tests/import.rs +++ b/differential-dataflow/tests/import.rs @@ -138,7 +138,7 @@ fn test_import_completed_dataflow() { .as_collection(|k,v| (k.clone(), v.clone())) .inner .exchange(|_| 0); - let probe = stream.probe(); + let probe = stream.clone().probe(); // <-- ack terrible! let captured = stream.capture(); (probe, captured,) }); diff --git a/differential-dataflow/tests/join.rs b/differential-dataflow/tests/join.rs index 7956518de..b724d6d9a 100644 --- a/differential-dataflow/tests/join.rs +++ b/differential-dataflow/tests/join.rs @@ -1,4 +1,4 @@ -use timely::dataflow::operators::{ToStream, Capture, Map}; +use timely::dataflow::operators::{ToStream, Capture, vec::Map}; use timely::dataflow::operators::capture::Extract; use differential_dataflow::AsCollection; @@ -18,7 +18,7 @@ fn join() { .as_collection(); // should produce triples `(0,0,'a')` and `(1,2,'B')`. - col1.join(&col2).inner.capture() + col1.join(col2).inner.capture() }); let extracted = data.extract(); @@ -35,7 +35,7 @@ fn join_map() { let col2 = vec![((0,'a'), Default::default(),1),((1,'B'), Default::default(),1)].into_iter().to_stream(scope).as_collection(); // should produce records `(0 + 0,'a')` and `(1 + 2,'B')`. - col1.join_map(&col2, |k,v1,v2| (*k + *v1, *v2)).inner.capture() + col1.join_map(col2, |k,v1,v2| (*k + *v1, *v2)).inner.capture() }); let extracted = data.extract(); @@ -50,7 +50,7 @@ fn semijoin() { let col2 = vec![(0, Default::default(),1)].into_iter().to_stream(scope).as_collection(); // should retain record `(0,0)` and discard `(1,2)`. - col1.semijoin(&col2).inner.capture() + col1.semijoin(col2).inner.capture() }); let extracted = data.extract(); @@ -65,7 +65,7 @@ fn antijoin() { let col2 = vec![(0, Default::default(),1)].into_iter().to_stream(scope).as_collection(); // should retain record `(1,2)` and discard `(0,0)`. - col1.antijoin(&col2).consolidate().inner.capture() + col1.antijoin(col2).consolidate().inner.capture() }); let extracted = data.extract(); assert_eq!(extracted.len(), 1); @@ -89,10 +89,10 @@ fn join_scaling(scale: u64) { .as_collection() .count(); - let odds = counts.filter(|x| x.1 % 2 == 1); + let odds = counts.clone().filter(|x| x.1 % 2 == 1); let evens = counts.filter(|x| x.1 % 2 == 0); - odds.join(&evens).inner.capture() + odds.join(evens).inner.capture() }); let extracted = data.extract(); diff --git a/differential-dataflow/tests/reduce.rs b/differential-dataflow/tests/reduce.rs index 6841ff5e8..eac0e217a 100644 --- a/differential-dataflow/tests/reduce.rs +++ b/differential-dataflow/tests/reduce.rs @@ -1,4 +1,4 @@ -use timely::dataflow::operators::{ToStream, Capture, Map}; +use timely::dataflow::operators::{ToStream, Capture, vec::Map}; use timely::dataflow::operators::capture::Extract; use differential_dataflow::AsCollection; diff --git a/differential-dataflow/tests/scc.rs b/differential-dataflow/tests/scc.rs index a0c4fee7d..fe6638ffc 100644 --- a/differential-dataflow/tests/scc.rs +++ b/differential-dataflow/tests/scc.rs @@ -182,7 +182,7 @@ fn scc_differential( let (edge_input, edges) = scope.new_collection(); - _strongly_connected(&edges) + _strongly_connected(edges) .consolidate() .inner .capture_into(send); @@ -215,45 +215,47 @@ fn scc_differential( .collect() } -fn _strongly_connected(graph: &VecCollection) -> VecCollection +fn _strongly_connected(graph: VecCollection) -> VecCollection where G: Scope, { - graph.iterate(|inner| { + graph.clone().iterate(|inner| { let edges = graph.enter(&inner.scope()); - let trans = edges.map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); - _trim_edges(&_trim_edges(inner, &edges), &trans) + let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); + _trim_edges(_trim_edges(inner, edges), trans) }) } -fn _trim_edges(cycle: &VecCollection, edges: &VecCollection) -> VecCollection +fn _trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection where G: Scope, { - let nodes = edges.map_in_place(|x| x.0 = x.1) + let nodes = edges.clone() + .map_in_place(|x| x.0 = x.1) .consolidate(); - let labels = _reachability(&cycle, &nodes); + let labels = _reachability(cycle, nodes); edges.consolidate() // .inspect(|x| println!("pre-join: {:?}", x)) - .join_map(&labels, |&e1,&e2,&l1| (e2,(e1,l1))) - .join_map(&labels, |&e2,&(e1,l1),&l2| ((e1,e2),(l1,l2))) + .join_map(labels.clone(), |&e1,&e2,&l1| (e2,(e1,l1))) + .join_map(labels.clone(), |&e2,&(e1,l1),&l2| ((e1,e2),(l1,l2))) .filter(|&(_,(l1,l2))| l1 == l2) .map(|((x1,x2),_)| (x2,x1)) } -fn _reachability(edges: &VecCollection, nodes: &VecCollection) -> VecCollection +fn _reachability(edges: VecCollection, nodes: VecCollection) -> VecCollection where G: Scope, { - edges.filter(|_| false) + edges.clone() // <-- wth is this. + .filter(|_| false) .iterate(|inner| { let edges = edges.enter(&inner.scope()); let nodes = nodes.enter_at(&inner.scope(), |r| 256 * (64 - (r.0 as u64).leading_zeros() as u64)); - inner.join_map(&edges, |_k,l,d| (*d,*l)) - .concat(&nodes) + inner.join_map(edges, |_k,l,d| (*d,*l)) + .concat(nodes) .reduce(|_, s, t| t.push((*s[0].0, 1))) }) diff --git a/dogsdogsdogs/README.md b/dogsdogsdogs/README.md index e4058b7a1..1e291e5e2 100644 --- a/dogsdogsdogs/README.md +++ b/dogsdogsdogs/README.md @@ -13,9 +13,9 @@ Differential dataflow provides a binary `join` operator that matches records in In the example above, we might write ```rust -edges.join(&edges) +edges.join(edges) .map(|(a,(b,c))| ((b,c),a)) - .semijoin(&edges) + .semijoin(edges) .map(|((b,c),a)| (a,b,c)); ``` diff --git a/dogsdogsdogs/examples/delta_query.rs b/dogsdogsdogs/examples/delta_query.rs index 9e197b5b9..af9eb5bfd 100644 --- a/dogsdogsdogs/examples/delta_query.rs +++ b/dogsdogsdogs/examples/delta_query.rs @@ -28,13 +28,13 @@ fn main() { let (edges_input, edges) = scope.new_collection(); // Graph oriented both ways, indexed by key. - let forward_key = edges.arrange_by_key(); - let reverse_key = edges.map(|(x,y)| (y,x)) + let forward_key = edges.clone().arrange_by_key(); + let reverse_key = edges.clone().map(|(x,y)| (y,x)) .arrange_by_key(); // Graph oriented both ways, indexed by (key, val). - let forward_self = edges.arrange_by_self(); - let reverse_self = edges.map(|(x,y)| (y,x)) + let forward_self = edges.clone().arrange_by_self(); + let reverse_self = edges.clone().map(|(x,y)| (y,x)) .arrange_by_self(); // // Graph oriented both ways, counts of distinct vals for each key. @@ -46,16 +46,16 @@ fn main() { let (triangles_prev, triangles_next) = scope.scoped::,_,_>("DeltaQuery (Triangles)", |inner| { // Grab the stream of changes. - let changes = edges.enter(inner); + let changes = edges.clone().enter(inner); // Each relation we'll need. - let forward_key_alt = forward_key.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1)); + let forward_key_alt = forward_key.clone().enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1)); let reverse_key_alt = reverse_key.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1)); let forward_key_neu = forward_key.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1)); // let reverse_key_neu = reverse_key.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1)); // let forward_self_alt = forward_self.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1)); - let reverse_self_alt = reverse_self.enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1)); + let reverse_self_alt = reverse_self.clone().enter_at(inner, |_,_,t| AltNeu::alt(t.clone()), |t| t.time.saturating_sub(1)); let forward_self_neu = forward_self.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1)); let reverse_self_neu = reverse_self.enter_at(inner, |_,_,t| AltNeu::neu(t.clone()), |t| t.time.saturating_sub(1)); @@ -76,21 +76,21 @@ fn main() { // Prior technology // dQ/dE1 := dE1(a,b), E2(b,c), E3(a,c) - let changes1 = propose(&changes, forward_key_neu.clone(), key2.clone()); - let changes1 = validate(&changes1, forward_self_neu.clone(), key1.clone()); + let changes1 = propose(changes.clone(), forward_key_neu.clone(), key2.clone()); + let changes1 = validate(changes1, forward_self_neu.clone(), key1.clone()); let changes1 = changes1.map(|((a,b),c)| (a,b,c)); // dQ/dE2 := dE2(b,c), E1(a,b), E3(a,c) - let changes2 = propose(&changes, reverse_key_alt.clone(), key1.clone()); - let changes2 = validate(&changes2, reverse_self_neu.clone(), key2.clone()); + let changes2 = propose(changes.clone(), reverse_key_alt.clone(), key1.clone()); + let changes2 = validate(changes2, reverse_self_neu.clone(), key2.clone()); let changes2 = changes2.map(|((b,c),a)| (a,b,c)); // dQ/dE3 := dE3(a,c), E1(a,b), E2(b,c) - let changes3 = propose(&changes, forward_key_alt.clone(), key1.clone()); - let changes3 = validate(&changes3, reverse_self_alt.clone(), key2.clone()); + let changes3 = propose(changes, forward_key_alt.clone(), key1.clone()); + let changes3 = validate(changes3, reverse_self_alt.clone(), key2.clone()); let changes3 = changes3.map(|((a,c),b)| (a,b,c)); - let prev_changes = changes1.concat(&changes2).concat(&changes3).leave(); + let prev_changes = changes1.concat(changes2).concat(changes3).leave(); // New ideas let d_edges = edges.differentiate(inner); @@ -98,29 +98,31 @@ fn main() { // dQ/dE1 := dE1(a,b), E2(b,c), E3(a,c) let changes1 = d_edges + .clone() .map(|(x,y)| (y,x)) - .join_core(&forward_key_neu, |b,a,c| Some(((*a, *c), *b))) - .join_core(&forward_self_neu, |(a,c), b, &()| Some((*a,*b,*c))); + .join_core(forward_key_neu, |b,a,c| Some(((*a, *c), *b))) + .join_core(forward_self_neu.clone(), |(a,c), b, &()| Some((*a,*b,*c))); // dQ/dE2 := dE2(b,c), E1(a,b), E3(a,c) let changes2 = d_edges - .join_core(&reverse_key_alt, |b,c,a| Some(((*a, *c), *b))) - .join_core(&forward_self_neu, |(a,c), b, &()| Some((*a,*b,*c))); + .clone() + .join_core(reverse_key_alt, |b,c,a| Some(((*a, *c), *b))) + .join_core(forward_self_neu, |(a,c), b, &()| Some((*a,*b,*c))); // dQ/dE3 := dE3(a,c), E1(a,b), E2(b,c) let changes3 = d_edges - .join_core(&forward_key_alt, |a,c,b| Some(((*c, *b), *a))) - .join_core(&reverse_self_alt, |(c,b), a, &()| Some((*a,*b,*c))); + .join_core(forward_key_alt, |a,c,b| Some(((*c, *b), *a))) + .join_core(reverse_self_alt, |(c,b), a, &()| Some((*a,*b,*c))); - let next_changes = changes1.concat(&changes2).concat(&changes3).integrate(); + let next_changes = changes1.concat(changes2).concat(changes3).integrate(); (prev_changes, next_changes) }); // Test if our two methods do the same thing. - triangles_prev.assert_eq(&triangles_next); + triangles_prev.clone().assert_eq(triangles_next); triangles_prev .filter(move |_| inspect) diff --git a/dogsdogsdogs/examples/delta_query2.rs b/dogsdogsdogs/examples/delta_query2.rs index 0585f58b8..6b088895e 100644 --- a/dogsdogsdogs/examples/delta_query2.rs +++ b/dogsdogsdogs/examples/delta_query2.rs @@ -1,8 +1,8 @@ use timely::dataflow::Scope; use timely::order::Product; use timely::dataflow::operators::probe::Handle; -use timely::dataflow::operators::UnorderedInput; -use timely::dataflow::operators::Map; +use timely::dataflow::operators::vec::UnorderedInput; +use timely::dataflow::operators::vec::Map; use differential_dataflow::AsCollection; fn main() { @@ -16,7 +16,7 @@ fn main() { // Nested scope as `Product` doesn't refine `()`, because .. coherence. scope.scoped("InnerScope", |inner| { - use timely::dataflow::operators::unordered_input::UnorderedHandle; + use timely::dataflow::operators::vec::unordered_input::UnorderedHandle; let ((input1, capability1), data1): ((UnorderedHandle, ((usize, usize), Product, isize)>, _), _) = inner.new_unordered_input(); let ((input2, capability2), data2): ((UnorderedHandle, ((usize, usize), Product, isize)>, _), _) = inner.new_unordered_input(); @@ -25,8 +25,8 @@ fn main() { let edges2 = data2.as_collection(); // Graph oriented both ways, indexed by key. - let forward1 = edges1.arrange_by_key(); - let forward2 = edges2.arrange_by_key(); + let forward1 = edges1.clone().arrange_by_key(); + let forward2 = edges2.clone().arrange_by_key(); // Grab the stream of changes. Stash the initial time as payload. let changes1 = edges1.inner.map(|((k,v),t,r)| ((k,v,t.clone()),t,r)).as_collection(); @@ -41,7 +41,7 @@ fn main() { let path1 = half_join( - &changes1, + changes1, forward2, closure, |t1,t2| t1.lt(t2), // This one ignores concurrent updates. @@ -50,7 +50,7 @@ fn main() { let path2 = half_join( - &changes2, + changes2, forward1, closure, |t1,t2| t1.le(t2), // This one can "see" concurrent updates. @@ -59,7 +59,7 @@ fn main() { // Delay updates until the worked payload time. // This should be at least the ignored update time. - path1.concat(&path2) + path1.concat(path2) .inner.map(|(((k,v),t),_,r)| ((k,v),t,r)).as_collection() .inspect(|x| println!("{:?}", x)) .probe_with(&mut probe); diff --git a/dogsdogsdogs/examples/delta_query_wcoj.rs b/dogsdogsdogs/examples/delta_query_wcoj.rs index 260c7159f..afceef1ee 100644 --- a/dogsdogsdogs/examples/delta_query_wcoj.rs +++ b/dogsdogsdogs/examples/delta_query_wcoj.rs @@ -38,10 +38,10 @@ fn main() { let reverse = reverse.enter(inner); // Without using wrappers yet, maintain an "old" and a "new" copy of edges. - let alt_forward = CollectionIndex::index(&forward); - let alt_reverse = CollectionIndex::index(&reverse); - let neu_forward = CollectionIndex::index(&forward.delay(|time| AltNeu::neu(time.time.clone()))); - let neu_reverse = CollectionIndex::index(&reverse.delay(|time| AltNeu::neu(time.time.clone()))); + let alt_forward = CollectionIndex::index(forward.clone()); + let alt_reverse = CollectionIndex::index(reverse.clone()); + let neu_forward = CollectionIndex::index(forward.clone().delay(|time| AltNeu::neu(time.time.clone()))); + let neu_reverse = CollectionIndex::index(reverse.clone().delay(|time| AltNeu::neu(time.time.clone()))); // For each relation, we form a delta query driven by changes to that relation. // @@ -55,6 +55,7 @@ fn main() { // dQ/dE1 := dE1(a,b), E2(b,c), E3(a,c) let changes1 = forward + .clone() .extend(&mut [ &mut neu_forward.extend_using(|(_a,b)| *b), &mut neu_forward.extend_using(|(a,_b)| *a), @@ -64,6 +65,7 @@ fn main() { // dQ/dE2 := dE2(b,c), E1(a,b), E3(a,c) let changes2 = forward + .clone() .extend(&mut [ &mut alt_reverse.extend_using(|(b,_c)| *b), &mut neu_reverse.extend_using(|(_b,c)| *c), @@ -78,7 +80,7 @@ fn main() { ]) .map(|((a,c),b)| (a,b,c)); - changes1.concat(&changes2).concat(&changes3).leave() + changes1.concat(changes2).concat(changes3).leave() }); triangles diff --git a/dogsdogsdogs/examples/dogsdogsdogs.rs b/dogsdogsdogs/examples/dogsdogsdogs.rs index 82098a794..be3ce2f90 100644 --- a/dogsdogsdogs/examples/dogsdogsdogs.rs +++ b/dogsdogsdogs/examples/dogsdogsdogs.rs @@ -1,4 +1,4 @@ -use timely::dataflow::operators::{ToStream, Partition, Accumulate, Inspect, Probe}; +use timely::dataflow::operators::{ToStream, vec::{Partition, count::Accumulate}, Inspect, Probe}; use timely::dataflow::operators::probe::Handle; use differential_dataflow::{Collection, AsCollection}; use differential_dataflow::input::Input; @@ -32,7 +32,7 @@ fn main() { println!("loaded {} nodes, {} edges", nodes, edges.len()); let index = worker.dataflow::(|scope| { - CollectionIndex::index(&Collection::new(edges.to_stream(scope))) + CollectionIndex::index(Collection::new(edges.to_stream(scope))) }); let mut index_xz = index.extend_using(|&(ref x, ref _y)| *x); @@ -46,22 +46,22 @@ fn main() { // determine stream of (prefix, count, index) indicating relation with fewest extensions. let counts = edges.map(|p| (p, usize::max_value(), usize::max_value())); - let counts0 = index_xz.count(&counts, 0); - let counts1 = index_yz.count(&counts0, 1); + let counts0 = index_xz.count(counts, 0); + let counts1 = index_yz.count(counts0, 1); // partition by index. let parts = counts1.inner.partition(2, |((p, _c, i),t,d)| (i as u64,(p,t,d))); // propose extensions using relation based on index. - let propose0 = index_xz.propose(&parts[0].as_collection()); - let propose1 = index_yz.propose(&parts[1].as_collection()); + let propose0 = index_xz.propose(parts[0].clone().as_collection()); + let propose1 = index_yz.propose(parts[1].clone().as_collection()); // validate proposals with the other index. - let validate0 = index_yz.validate(&propose0); - let validate1 = index_xz.validate(&propose1); + let validate0 = index_yz.validate(propose0); + let validate1 = index_xz.validate(propose1); validate0 - .concat(&validate1) + .concat(validate1) .inner .count() .inspect(move |x| println!("{:?}", x)) diff --git a/dogsdogsdogs/examples/ngo.rs b/dogsdogsdogs/examples/ngo.rs index d0beb6651..340b6c988 100644 --- a/dogsdogsdogs/examples/ngo.rs +++ b/dogsdogsdogs/examples/ngo.rs @@ -1,6 +1,7 @@ use std::hash::Hash; use timely::dataflow::*; use timely::dataflow::operators::*; +use timely::dataflow::operators::vec::count::Accumulate; use differential_dataflow::VecCollection; use differential_dataflow::lattice::Lattice; @@ -31,13 +32,13 @@ fn main() { println!("loaded {} nodes, {} edges", nodes, edges.len()); worker.dataflow::<(),_,_>(|scope| { - triangles(&VecCollection::new(edges.to_stream(scope))).inner.count().inspect(|x| println!("{:?}", x)); + triangles(VecCollection::new(edges.to_stream(scope))).inner.count().inspect(|x| println!("{:?}", x)); }); }).unwrap(); } -fn triangles(edges: &VecCollection) -> VecCollection +fn triangles(edges: VecCollection) -> VecCollection where G: Scope, { @@ -45,9 +46,9 @@ where let edges = edges.filter(|&(src, dst)| src < dst); // arrange the edge relation three ways. - let as_self = edges.arrange_by_self(); - let forward = edges.arrange_by_key(); - let reverse = edges.map_in_place(|x| ::std::mem::swap(&mut x.0, &mut x.1)) + let as_self = edges.clone().arrange_by_self(); + let forward = edges.clone().arrange_by_key(); + let reverse = edges.clone().map_in_place(|x| ::std::mem::swap(&mut x.0, &mut x.1)) .arrange_by_key(); // arrange the count of extensions from each source. @@ -55,11 +56,11 @@ where .arrange_by_self(); // extract ((src, dst), idx) tuples with weights equal to the number of extensions. - let cand_count1 = forward.join_core(&counts, |&src, &dst, &()| Some(((src, dst), 1))); - let cand_count2 = reverse.join_core(&counts, |&dst, &src, &()| Some(((src, dst), 2))); + let cand_count1 = forward.clone().join_core(counts.clone(), |&src, &dst, &()| Some(((src, dst), 1))); + let cand_count2 = reverse.join_core(counts, |&dst, &src, &()| Some(((src, dst), 2))); // determine for each (src, dst) tuple which index would propose the fewest extensions. - let winners = cand_count1.concat(&cand_count2) + let winners = cand_count1.concat(cand_count2) .reduce(|_srcdst, counts, output| { if counts.len() == 2 { let mut min_cnt = isize::max_value(); @@ -75,17 +76,18 @@ where }); // select tuples with the first relation minimizing the proposals, join, then intersect. - let winners1 = winners.flat_map(|((src, dst), index)| if index == 1 { Some((src, dst)) } else { None }) - .join_core(&forward, |&src, &dst, &ext| Some(((dst, ext), src))) - .join_core(&as_self, |&(dst, ext), &src, &()| Some(((dst, ext), src))) + let winners1 = winners.clone() + .flat_map(|((src, dst), index)| if index == 1 { Some((src, dst)) } else { None }) + .join_core(forward.clone(), |&src, &dst, &ext| Some(((dst, ext), src))) + .join_core(as_self.clone(), |&(dst, ext), &src, &()| Some(((dst, ext), src))) .map(|((dst, ext), src)| (src, dst, ext)); // select tuples with the second relation minimizing the proposals, join, then intersect. let winners2 = winners.flat_map(|((src, dst), index)| if index == 2 { Some((dst, src)) } else { None }) - .join_core(&forward, |&dst, &src, &ext| Some(((src, ext), dst))) - .join_core(&as_self, |&(src, ext), &dst, &()| Some(((src, ext), dst))) + .join_core(forward.clone(), |&dst, &src, &ext| Some(((src, ext), dst))) + .join_core(as_self, |&(src, ext), &dst, &()| Some(((src, ext), dst))) .map(|((src, ext), dst)| (src, dst, ext)); // collect and return results. - winners1.concat(&winners2) + winners1.concat(winners2) } diff --git a/dogsdogsdogs/src/calculus.rs b/dogsdogsdogs/src/calculus.rs index babd9e81a..94b443231 100644 --- a/dogsdogsdogs/src/calculus.rs +++ b/dogsdogsdogs/src/calculus.rs @@ -14,7 +14,7 @@ use timely::dataflow::Scope; use timely::dataflow::scopes::Child; -use timely::dataflow::operators::{Filter, Map}; +use timely::dataflow::operators::vec::{Filter, Map}; use differential_dataflow::{AsCollection, VecCollection, Data}; use differential_dataflow::difference::Abelian; @@ -22,12 +22,12 @@ use crate::altneu::AltNeu; /// Produce a collection containing the changes at the moments they happen. pub trait Differentiate { - fn differentiate<'a>(&self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R>; + fn differentiate<'a>(self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R>; } /// Collect instantaneous changes back in to a collection. pub trait Integrate { - fn integrate(&self) -> VecCollection; + fn integrate(self) -> VecCollection; } impl Differentiate for VecCollection @@ -37,7 +37,7 @@ where R: Abelian + 'static, { // For each (data, Alt(time), diff) we add a (data, Neu(time), -diff). - fn differentiate<'a>(&self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R> { + fn differentiate<'a>(self, child: &Child<'a, G, AltNeu>) -> VecCollection>, D, R> { self.enter(child) .inner .flat_map(|(data, time, diff)| { @@ -58,7 +58,7 @@ where R: Abelian + 'static, { // We discard each `neu` variant and strip off the `alt` wrapper. - fn integrate(&self) -> VecCollection { + fn integrate(self) -> VecCollection { self.inner .filter(|(_d,t,_r)| !t.neu) .as_collection() diff --git a/dogsdogsdogs/src/lib.rs b/dogsdogsdogs/src/lib.rs index 7b63ad202..f704d5988 100644 --- a/dogsdogsdogs/src/lib.rs +++ b/dogsdogsdogs/src/lib.rs @@ -2,7 +2,7 @@ use std::hash::Hash; use timely::dataflow::Scope; use timely::progress::Timestamp; -use timely::dataflow::operators::Partition; +use timely::dataflow::operators::vec::Partition; use timely::dataflow::operators::Concatenate; use differential_dataflow::{ExchangeData, VecCollection, AsCollection}; @@ -26,16 +26,16 @@ pub trait PrefixExtender> { /// The type to be produced as extension. type Extension; /// Annotates prefixes with the number of extensions the relation would propose. - fn count(&mut self, prefixes: &VecCollection, index: usize) -> VecCollection; + fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection; /// Extends each prefix with corresponding extensions. - fn propose(&mut self, prefixes: &VecCollection) -> VecCollection; + fn propose(&mut self, prefixes: VecCollection) -> VecCollection; /// Restricts proposed extensions by those the extender would have proposed. - fn validate(&mut self, extensions: &VecCollection) -> VecCollection; + fn validate(&mut self, extensions: VecCollection) -> VecCollection; } pub trait ProposeExtensionMethod> { - fn propose_using>(&self, extender: &mut PE) -> VecCollection; - fn extend(&self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection; + fn propose_using>(self, extender: &mut PE) -> VecCollection; + fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection; } impl ProposeExtensionMethod for VecCollection @@ -44,33 +44,33 @@ where P: ExchangeData+Ord, R: Monoid+Multiply+'static, { - fn propose_using(&self, extender: &mut PE) -> VecCollection + fn propose_using(self, extender: &mut PE) -> VecCollection where PE: PrefixExtender { extender.propose(self) } - fn extend(&self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection + fn extend(self, extenders: &mut [&mut dyn PrefixExtender]) -> VecCollection where E: ExchangeData+Ord { if extenders.len() == 1 { - extenders[0].propose(&self.clone()) + extenders[0].propose(self) } else { - let mut counts = self.map(|p| (p, 1 << 31, 0)); + let mut counts = self.clone().map(|p| (p, 1 << 31, 0)); for (index,extender) in extenders.iter_mut().enumerate() { - counts = extender.count(&counts, index); + counts = extender.count(counts, index); } let parts = counts.inner.partition(extenders.len() as u64, |((p, _, i),t,d)| (i as u64, (p,t,d))); let mut results = Vec::new(); for (index, nominations) in parts.into_iter().enumerate() { - let mut extensions = extenders[index].propose(&nominations.as_collection()); + let mut extensions = extenders[index].propose(nominations.as_collection()); for other in (0..extenders.len()).filter(|&x| x != index) { - extensions = extenders[other].validate(&extensions); + extensions = extenders[other].validate(extensions); } results.push(extensions.inner); // save extensions @@ -82,11 +82,11 @@ where } pub trait ValidateExtensionMethod, P, E> { - fn validate_using>(&self, extender: &mut PE) -> VecCollection; + fn validate_using>(self, extender: &mut PE) -> VecCollection; } impl, P, E> ValidateExtensionMethod for VecCollection { - fn validate_using>(&self, extender: &mut PE) -> VecCollection { + fn validate_using>(self, extender: &mut PE) -> VecCollection { extender.validate(self) } } @@ -137,12 +137,13 @@ where R: Monoid+Multiply+ExchangeData, { - pub fn index>(collection: &VecCollection) -> Self { + pub fn index>(collection: VecCollection) -> Self { // We need to count the number of (k, v) pairs and not rely on the given Monoid R and its binary addition operation. // counts and validate can share the base arrangement - let arranged = collection.arrange_by_self(); + let arranged = collection.clone().arrange_by_self(); // TODO: This could/should be arrangement to arrangement, via `reduce_abelian`, but the types are a mouthful at the moment. let counts = arranged + .clone() .as_collection(|k,_v| k.clone()) .distinct() .map(|(k, _v)| k) @@ -191,17 +192,17 @@ where type Prefix = P; type Extension = V; - fn count(&mut self, prefixes: &VecCollection, index: usize) -> VecCollection { + fn count(&mut self, prefixes: VecCollection, index: usize) -> VecCollection { let counts = self.indices.count_trace.import(&prefixes.scope()); operators::count::count(prefixes, counts, self.key_selector.clone(), index) } - fn propose(&mut self, prefixes: &VecCollection) -> VecCollection { + fn propose(&mut self, prefixes: VecCollection) -> VecCollection { let propose = self.indices.propose_trace.import(&prefixes.scope()); operators::propose::propose(prefixes, propose, self.key_selector.clone()) } - fn validate(&mut self, extensions: &VecCollection) -> VecCollection { + fn validate(&mut self, extensions: VecCollection) -> VecCollection { let validate = self.indices.validate_trace.import(&extensions.scope()); operators::validate::validate(extensions, validate, self.key_selector.clone()) } diff --git a/dogsdogsdogs/src/operators/count.rs b/dogsdogsdogs/src/operators/count.rs index ae0e87409..0cd363a76 100644 --- a/dogsdogsdogs/src/operators/count.rs +++ b/dogsdogsdogs/src/operators/count.rs @@ -12,7 +12,7 @@ use differential_dataflow::trace::TraceReader; /// associated count in `arrangement`. If the found count is less than `count`, /// the `count` and `index` fields are overwritten with their new values. pub fn count( - prefixes: &VecCollection, + prefixes: VecCollection, arrangement: Arranged, key_selector: F, index: usize, diff --git a/dogsdogsdogs/src/operators/half_join.rs b/dogsdogsdogs/src/operators/half_join.rs index a864a5517..ddb2a5979 100644 --- a/dogsdogsdogs/src/operators/half_join.rs +++ b/dogsdogsdogs/src/operators/half_join.rs @@ -37,7 +37,7 @@ use std::time::Instant; use timely::ContainerBuilder; use timely::container::CapacityContainerBuilder; -use timely::dataflow::{Scope, ScopeParent, StreamCore}; +use timely::dataflow::{Scope, ScopeParent, Stream}; use timely::dataflow::channels::pact::{Pipeline, Exchange}; use timely::dataflow::operators::{Capability, Operator, generic::Session}; use timely::progress::Antichain; @@ -74,7 +74,7 @@ use differential_dataflow::trace::implementations::BatchContainer; /// once out of the "delta flow region", the updates will be `delay`d to the /// times specified in the payloads. pub fn half_join( - stream: &VecCollection, + stream: VecCollection, arrangement: Arranged, frontier_func: FF, comparison: CF, @@ -139,13 +139,13 @@ type SessionFor<'a, 'b, G, CB> = /// records. Note this is not the number of *output* records, owing mainly to /// the number of matched records being easiest to record with low overhead. pub fn half_join_internal_unsafe( - stream: &VecCollection, + stream: VecCollection, mut arrangement: Arranged, frontier_func: FF, comparison: CF, yield_function: Y, mut output_func: S, -) -> StreamCore +) -> Stream where G: Scope, K: Hashable + ExchangeData, @@ -170,16 +170,17 @@ where // Stash for (time, diff) accumulation. let mut output_buffer = Vec::new(); - stream.inner.binary_frontier(&arrangement_stream, exchange, Pipeline, "HalfJoin", move |_,info| { + let scope = stream.scope(); + stream.inner.binary_frontier(arrangement_stream, exchange, Pipeline, "HalfJoin", move |_,info| { // Acquire an activator to reschedule the operator when it has unfinished work. - let activator = stream.scope().activator_for(info.address); + let activator = scope.activator_for(info.address); move |(input1, frontier1), (input2, frontier2), output| { // drain the first input, stashing requests. input1.for_each(|capability, data| { - stash.entry(capability.retain()) + stash.entry(capability.retain(0)) .or_insert(Vec::new()) .append(data) }); diff --git a/dogsdogsdogs/src/operators/lookup_map.rs b/dogsdogsdogs/src/operators/lookup_map.rs index da02e2bd1..7908ce858 100644 --- a/dogsdogsdogs/src/operators/lookup_map.rs +++ b/dogsdogsdogs/src/operators/lookup_map.rs @@ -18,7 +18,7 @@ use differential_dataflow::trace::implementations::BatchContainer; /// key with `key_selector` and then proposes all pair af the prefix /// and values associated with the key in `arrangement`. pub fn lookup_map( - prefixes: &VecCollection, + prefixes: VecCollection, mut arrangement: Arranged, key_selector: F, mut output_func: S, @@ -59,11 +59,11 @@ where let mut key1: K = supplied_key1; let mut key2: K = supplied_key2; - prefixes.inner.binary_frontier(&propose_stream, exchange, Pipeline, "LookupMap", move |_,_| move |(input1, frontier1), (input2, frontier2), output| { + prefixes.inner.binary_frontier(propose_stream, exchange, Pipeline, "LookupMap", move |_,_| move |(input1, frontier1), (input2, frontier2), output| { // drain the first input, stashing requests. input1.for_each(|capability, data| { - stash.entry(capability.retain()) + stash.entry(capability.retain(0)) .or_insert(Vec::new()) .extend(data.drain(..)) }); diff --git a/dogsdogsdogs/src/operators/propose.rs b/dogsdogsdogs/src/operators/propose.rs index 2606285c8..9f2e69f78 100644 --- a/dogsdogsdogs/src/operators/propose.rs +++ b/dogsdogsdogs/src/operators/propose.rs @@ -14,7 +14,7 @@ use differential_dataflow::trace::TraceReader; /// `arrangement` undergoes. More complicated patterns are also appropriate, as in the case /// of delta queries. pub fn propose( - prefixes: &VecCollection, + prefixes: VecCollection, arrangement: Arranged, key_selector: F, ) -> VecCollection @@ -48,7 +48,7 @@ where /// prefixes by the number of matches in `arrangement`. This can be useful to /// avoid the need to prepare an arrangement of distinct extensions. pub fn propose_distinct( - prefixes: &VecCollection, + prefixes: VecCollection, arrangement: Arranged, key_selector: F, ) -> VecCollection diff --git a/dogsdogsdogs/src/operators/validate.rs b/dogsdogsdogs/src/operators/validate.rs index ccb0b34c0..f610d55be 100644 --- a/dogsdogsdogs/src/operators/validate.rs +++ b/dogsdogsdogs/src/operators/validate.rs @@ -13,7 +13,7 @@ use differential_dataflow::trace::TraceReader; /// key with `key_selector` and then proposes all pair af the prefix /// and values associated with the key in `arrangement`. pub fn validate( - extensions: &VecCollection, + extensions: VecCollection, arrangement: Arranged, key_selector: F, ) -> VecCollection diff --git a/doop/src/main.rs b/doop/src/main.rs index 50bc35044..b7f85f499 100644 --- a/doop/src/main.rs +++ b/doop/src/main.rs @@ -64,7 +64,7 @@ impl<'a, G: Scope, D: Data+Hashable> Relation<'a, G, D> where G::Timestamp : Lat Self::new_from(&::timely::dataflow::operators::generic::operator::empty(scope).as_collection()) } /// Creates a new variable initialized with `source`. - pub fn new_from(source: &VecCollection, D, Diff>) -> Self { + pub fn new_from(source: VecCollection, D, Diff>) -> Self { use ::timely::order::Product; let variable = VecVariable::new_from(source.clone(), Product::new(Default::default(), 1)); Relation { @@ -73,7 +73,7 @@ impl<'a, G: Scope, D: Data+Hashable> Relation<'a, G, D> where G::Timestamp : Lat } } /// Concatenates `production` into the definition of the variable. - pub fn add_production(&mut self, production: &VecCollection, D, Diff>) { + pub fn add_production(&mut self, production: VecCollection, D, Diff>) { self.current = self.current.concat(production); } /// Finalizes the variable, connecting its recursive definition. @@ -81,7 +81,7 @@ impl<'a, G: Scope, D: Data+Hashable> Relation<'a, G, D> where G::Timestamp : Lat /// Failure to call `complete` on a variable results in a non-recursively defined /// collection, whose contents are just its initial `source` data. pub fn complete(self) { - self.variable.set(&self.current.threshold(|_,_| 1)); + self.variable.set(self.current.threshold(|_,_| 1)); } } @@ -114,7 +114,7 @@ impl StringInterner { *self.map.get(string).unwrap() } } - pub fn concat(&mut self, id1: Symbol, id2: Symbol) -> Symbol { + pub fn concat(mut self, id1: Symbol, id2: Symbol) -> Symbol { let string = self.vec[id1 as usize].to_owned() + &self.vec[id2 as usize]; self.intern(&string) } @@ -317,16 +317,16 @@ fn main() { // Main schema let isType: VecCollection<_,Type> = ClassType - .concat(&ArrayType) - .concat(&InterfaceType) - .concat(&ApplicationClass) - .concat(&_NormalHeap.map(|(_id,ty)| ty)); + .concat(ArrayType) + .concat(InterfaceType) + .concat(ApplicationClass) + .concat(_NormalHeap.map(|(_id,ty)| ty)); let isReferenceType: VecCollection<_,ReferenceType> = ClassType - .concat(&ArrayType) - .concat(&InterfaceType) - .concat(&ApplicationClass); + .concat(ArrayType) + .concat(InterfaceType) + .concat(ApplicationClass); let Field_DeclaringType = _Field.map(|(sig,dec,_,_)| (sig,dec)); let Method_DeclaringType = _Method.map(|(meth,_,_,dec,_,_,_)| (meth,dec)); @@ -343,9 +343,9 @@ fn main() { let temp4 = interner.borrow_mut().intern("java.lang.String[]"); let HeapAllocation_Type = _NormalHeap - .concat(&_StringConstant.map(move |s| (s, temp1.clone()))) - .concat(&scope.new_collection_from_raw(Some(((temp3.clone(), temp4.clone()), 0, 1))).1) - .concat(&scope.new_collection_from_raw(Some(((temp2.clone(), temp1b.clone()), 0, 1))).1); + .concat(_StringConstant.map(move |s| (s, temp1.clone()))) + .concat(scope.new_collection_from_raw(Some(((temp3.clone(), temp4.clone()), 0, 1))).1) + .concat(scope.new_collection_from_raw(Some(((temp2.clone(), temp1b.clone()), 0, 1))).1); // NOTE: Unused // let MainMethodArgArray: VecCollection<_,HeapAllocation> = scope.new_collection_from_raw(Some(temp3.clone())).1; @@ -354,18 +354,18 @@ fn main() { let Instruction_Method = //: VecCollection<_,(Instruction, Method)> = _AssignHeapAllocation.map(|x| (x.0, x.4)) - .concat(&_AssignLocal.map(|x| (x.0, x.4))) - .concat(&_AssignCast.map(|x| (x.0, x.5))) - .concat(&_StaticMethodInvocation.map(|x| (x.0, x.3))) - .concat(&_SpecialMethodInvocation.map(|x| (x.0, x.4))) - .concat(&_VirtualMethodInvocation.map(|x| (x.0, x.4))) - .concat(&_StoreInstanceField.map(|x| (x.0, x.5))) - .concat(&_LoadInstanceField.map(|x| (x.0, x.5))) - .concat(&_StoreStaticField.map(|x| (x.0, x.4))) - .concat(&_LoadStaticField.map(|x| (x.0, x.4))) - .concat(&_StoreArrayIndex.map(|x| (x.0, x.4))) - .concat(&_LoadArrayIndex.map(|x| (x.0, x.4))) - .concat(&_Return.map(|x| (x.0, x.3))) + .concat(_AssignLocal.map(|x| (x.0, x.4))) + .concat(_AssignCast.map(|x| (x.0, x.5))) + .concat(_StaticMethodInvocation.map(|x| (x.0, x.3))) + .concat(_SpecialMethodInvocation.map(|x| (x.0, x.4))) + .concat(_VirtualMethodInvocation.map(|x| (x.0, x.4))) + .concat(_StoreInstanceField.map(|x| (x.0, x.5))) + .concat(_LoadInstanceField.map(|x| (x.0, x.5))) + .concat(_StoreStaticField.map(|x| (x.0, x.4))) + .concat(_LoadStaticField.map(|x| (x.0, x.4))) + .concat(_StoreArrayIndex.map(|x| (x.0, x.4))) + .concat(_LoadArrayIndex.map(|x| (x.0, x.4))) + .concat(_Return.map(|x| (x.0, x.3))) .distinct() .arrange_by_key(); @@ -374,9 +374,9 @@ fn main() { let FieldInstruction_Signature: VecCollection<_,(FieldInstruction, Field)> = _StoreInstanceField.map(|x| (x.0, x.4)) - .concat(&_LoadInstanceField.map(|x| (x.0, x.4))) - .concat(&_StoreStaticField.map(|x| (x.0, x.3))) - .concat(&_LoadStaticField.map(|x| (x.0, x.3))); + .concat(_LoadInstanceField.map(|x| (x.0, x.4))) + .concat(_StoreStaticField.map(|x| (x.0, x.3))) + .concat(_LoadStaticField.map(|x| (x.0, x.3))); let LoadInstanceField_Base = _LoadInstanceField.map(|x| (x.0, x.3)); let LoadInstanceField_To = _LoadInstanceField.map(|x| (x.0, x.2)); @@ -392,8 +392,8 @@ fn main() { let AssignInstruction_To: VecCollection<_,(AssignInstruction, Var)> = _AssignHeapAllocation.map(|x| (x.0, x.3)) - .concat(&_AssignLocal.map(|x| (x.0, x.3))) - .concat(&_AssignCast.map(|x| (x.0, x.3))); + .concat(_AssignLocal.map(|x| (x.0, x.3))) + .concat(_AssignCast.map(|x| (x.0, x.3))); let AssignCast_From = _AssignCast.map(|x| (x.0, x.2)); let AssignCast_Type = _AssignCast.map(|x| (x.0, x.4)); @@ -403,13 +403,13 @@ fn main() { let ReturnNonvoid_Var = _Return.map(|x| (x.0, x.2)); let MethodInvocation_Method: VecCollection<_,(MethodInvocation, Method)> = _StaticMethodInvocation.map(|x| (x.0, x.2)) - .concat(&_SpecialMethodInvocation.map(|x| (x.0, x.2))) - .concat(&_VirtualMethodInvocation.map(|x| (x.0, x.2))); + .concat(_SpecialMethodInvocation.map(|x| (x.0, x.2))) + .concat(_VirtualMethodInvocation.map(|x| (x.0, x.2))); let VirtualMethodInvocation_Base = _VirtualMethodInvocation.map(|x| (x.0, x.3)); let SpecialMethodInvocation_Base = _SpecialMethodInvocation.map(|x| (x.0, x.3)); // NOTE: Unused - // let MethodInvocation_Base = VirtualMethodInvocation_Base.concat(&SpecialMethodInvocation_Base); + // let MethodInvocation_Base = VirtualMethodInvocation_Base.concat(SpecialMethodInvocation_Base); // Fat schema @@ -421,9 +421,9 @@ fn main() { let LoadInstanceField: VecCollection<_,(Var, Field, Var, Method)> = Instruction_Method .as_vecs() - .join(&LoadInstanceField_Base) - .join(&FieldInstruction_Signature) - .join(&LoadInstanceField_To) + .join(LoadInstanceField_Base) + .join(FieldInstruction_Signature) + .join(LoadInstanceField_To) .map(|(_insn, (((inmethod, base), sig), to))| (base, sig, to, inmethod)); // StoreInstanceField(?from, ?base, ?sig, ?inmethod) :- @@ -434,9 +434,9 @@ fn main() { let StoreInstanceField: VecCollection<_,(Var, Var, Field, Method)> = Instruction_Method .as_vecs() - .join(&StoreInstanceField_From) - .join(&StoreInstanceField_Base) - .join(&FieldInstruction_Signature) + .join(StoreInstanceField_From) + .join(StoreInstanceField_Base) + .join(FieldInstruction_Signature) .map(|(_insn, (((inmethod, from), base), sig))| (from, base, sig, inmethod)); // LoadStaticField(?sig, ?to, ?inmethod) :- @@ -446,8 +446,8 @@ fn main() { let LoadStaticField: VecCollection<_,(Field, Var, Method)> = Instruction_Method .as_vecs() - .join(&FieldInstruction_Signature) - .join(&LoadStaticField_To) + .join(FieldInstruction_Signature) + .join(LoadStaticField_To) .map(|(_insn, ((inmethod, sig), to))| (sig, to, inmethod)); // StoreStaticField(?from, ?sig, ?inmethod) :- @@ -457,8 +457,8 @@ fn main() { let StoreStaticField: VecCollection<_,(Var, Field, Method)> = Instruction_Method .as_vecs() - .join(&StoreStaticField_From) - .join(&FieldInstruction_Signature) + .join(StoreStaticField_From) + .join(FieldInstruction_Signature) .map(|(_insn, ((inmethod, from), sig))| (from, sig, inmethod)); // LoadArrayIndex(?base, ?to, ?inmethod) :- @@ -468,8 +468,8 @@ fn main() { let LoadArrayIndex: VecCollection<_,(Var, Var, Method)> = Instruction_Method .as_vecs() - .join(&LoadArrayIndex_Base) - .join(&LoadArrayIndex_To) + .join(LoadArrayIndex_Base) + .join(LoadArrayIndex_To) .map(|(_insn, ((inmethod, base), to))| (base, to, inmethod)); // StoreArrayIndex(?from, ?base, ?inmethod) :- @@ -479,8 +479,8 @@ fn main() { let StoreArrayIndex: VecCollection<_,(Var, Var, Method)> = Instruction_Method .as_vecs() - .join(&StoreArrayIndex_From) - .join(&StoreArrayIndex_Base) + .join(StoreArrayIndex_From) + .join(StoreArrayIndex_Base) .map(|(_insn, ((inmethod, from), base))| (from, base, inmethod)); // AssignCast(?type, ?from, ?to, ?inmethod) :- @@ -491,9 +491,9 @@ fn main() { let AssignCast: VecCollection<_,(Type, Var, Var, Method)> = Instruction_Method .as_vecs() - .join(&AssignCast_From) - .join(&AssignInstruction_To) - .join(&AssignCast_Type) + .join(AssignCast_From) + .join(AssignInstruction_To) + .join(AssignCast_Type) .map(|(_insn, (((inmethod, from), to), ty))| (ty, from, to, inmethod)); // AssignLocal(?from, ?to, ?inmethod) :- @@ -503,8 +503,8 @@ fn main() { let AssignLocal: VecCollection<_,(Var, Var, Method)> = Instruction_Method .as_vecs() - .join(&AssignInstruction_To) - .join(&AssignLocal_From) + .join(AssignInstruction_To) + .join(AssignLocal_From) .map(|(_insn, ((inmethod, to), from))| (from, to, inmethod)); // AssignHeapAllocation(?heap, ?to, ?inmethod) :- @@ -514,8 +514,8 @@ fn main() { let AssignHeapAllocation: VecCollection<_,(HeapAllocation, Var, Method)> = Instruction_Method .as_vecs() - .join(&AssignHeapAllocation_Heap) - .join(&AssignInstruction_To) + .join(AssignHeapAllocation_Heap) + .join(AssignInstruction_To) .map(|(_insn, ((inmethod, heap), to))| (heap, to, inmethod)); // ReturnVar(?var, ?method) :- @@ -524,7 +524,7 @@ fn main() { let ReturnVar: VecCollection<_,(Var, Method)> = Instruction_Method .as_vecs() - .join(&ReturnNonvoid_Var) + .join(ReturnNonvoid_Var) .map(|(_insn, (inmethod, var))| (var, inmethod)); // StaticMethodInvocation(?invocation, ?signature, ?inmethod) :- @@ -533,8 +533,8 @@ fn main() { // MethodInvocation_Method(?invocation, ?signature). let StaticMethodInvocation: VecCollection<_,(MethodInvocation, Method, Method)> = Instruction_Method - .join_core(&isStaticMethodInvocation_Insn.arrange_by_self(), |k,v,_| [(k.clone(), v.clone())]) - .join(&MethodInvocation_Method) + .join_core(isStaticMethodInvocation_Insn.arrange_by_self(), |k,v,_| [(k.clone(), v.clone())]) + .join(MethodInvocation_Method) .map(|(invocation, (inmethod, sig))| (invocation, sig, inmethod)); // VirtualMethodInvocation_SimpleName(?invocation, ?simplename), @@ -545,14 +545,14 @@ fn main() { // Method_Descriptor(?signature, ?descriptor). let VirtualTemp = MethodInvocation_Method - .semijoin(&isVirtualMethodInvocation_Insn) + .semijoin(isVirtualMethodInvocation_Insn) .map(|(invocation, signature)| (signature, invocation)); let VirtualMethodInvocation_SimpleName = VirtualTemp - .join_map(&Method_SimpleName, |_sig, inv, name| (inv.clone(), name.clone())); + .join_map(Method_SimpleName, |_sig, inv, name| (inv.clone(), name.clone())); let VirtualMethodInvocation_Descriptor = VirtualTemp - .join_map(&Method_Descriptor, |_sig, inv, desc| (inv.clone(), desc.clone())); + .join_map(Method_Descriptor, |_sig, inv, desc| (inv.clone(), desc.clone())); let (MethodImplemented, MainMethodDeclaration, MethodLookup, SupertypeOf) = scope.scoped("Basic", |scope| { @@ -610,9 +610,9 @@ fn main() { let temp = interner.borrow_mut().intern("abstract"); let MethodImplemented = Method_SimpleName - .antijoin(&Method_Modifier.filter(move |x| x.0 == temp).map(|x| x.1).distinct()) - .join(&Method_Descriptor) - .join(&Method_DeclaringType) + .antijoin(Method_Modifier.filter(move |x| x.0 == temp).map(|x| x.1).distinct()) + .join(Method_Descriptor) + .join(Method_DeclaringType) .map(|(method, ((simple, desc), ty))| (simple, desc, ty, method)) .threshold(|_,_| 1 as Diff); // .distinct(); @@ -624,13 +624,13 @@ fn main() { // MethodLookup(?simplename, ?descriptor, ?supertype, ?method), // ! MethodImplemented(?simplename, ?descriptor, ?type, _). let MethodLookupClone = MethodLookup.clone(); - MethodLookup.add_production(&MethodImplemented); + MethodLookup.add_production(MethodImplemented); MethodLookup.add_production( - &DirectSuperclass.concat(&DirectSuperinterface) + DirectSuperclass.concat(DirectSuperinterface) .map(|x| (x.1, x.0)) - .join(&MethodLookupClone.map(|x| (x.2, (x.0, x.1, x.3)))) + .join(MethodLookupClone.map(|x| (x.2, (x.0, x.1, x.3)))) .map(|(_superty, (ty, (name, desc, method)))| ((name, desc, ty), method)) - .antijoin(&MethodImplemented.map(|x| (x.0, x.1, x.2)).threshold(|_,_| 1)) + .antijoin(MethodImplemented.map(|x| (x.0, x.1, x.2)).threshold(|_,_| 1)) .map(|((name, desc, ty), method)| (name, desc, ty, method)) ); @@ -643,8 +643,8 @@ fn main() { // Subclass(?c, ?a) :- DirectSubclass(?a, ?c). // Subclass(?c, ?a) :- Subclass(?b, ?a),DirectSubclass(?b, ?c). let SubclassClone = Subclass.clone(); - Subclass.add_production(&DirectSubclass.map(|x| (x.1, x.0))); - Subclass.add_production(&SubclassClone.join_map(&DirectSubclass, |_b, a, c| (c.clone(), a.clone()))); + Subclass.add_production(DirectSubclass.map(|x| (x.1, x.0))); + Subclass.add_production(SubclassClone.join_map(DirectSubclass, |_b, a, c| (c.clone(), a.clone()))); // Superclass(?c, ?a) :- Subclass(?a, ?c). let _Superclass = Subclass.map(|x| (x.1, x.0)); @@ -653,17 +653,17 @@ fn main() { // Superinterface(?k, ?c) :- DirectSuperinterface(?c, ?j),Superinterface(?k, ?j). // Superinterface(?k, ?c) :- DirectSuperclass(?c, ?super),Superinterface(?k, ?super). let SuperinterfaceClone = Superinterface.clone(); - Superinterface.add_production(&DirectSuperinterface.map(|x| (x.1, x.0))); + Superinterface.add_production(DirectSuperinterface.map(|x| (x.1, x.0))); Superinterface.add_production( - &DirectSuperinterface + DirectSuperinterface .map(|x| (x.1, x.0)) - .join(&SuperinterfaceClone.map(|x| (x.1, x.0))) + .join(SuperinterfaceClone.map(|x| (x.1, x.0))) .map(|(_j, (c, k))| (k, c)) ); Superinterface.add_production( - &DirectSuperclass + DirectSuperclass .map(|x| (x.1, x.0)) - .join(&SuperinterfaceClone.map(|x| (x.1, x.0))) + .join(SuperinterfaceClone.map(|x| (x.1, x.0))) .map(|(_j, (c, k))| (k, c)) ); @@ -672,21 +672,21 @@ fn main() { let SubtypeOfClone = SubtypeOf.clone(); // SubtypeOf(?s, ?s) :- isClassType(?s). - SubtypeOf.add_production(&ClassType.map(|x| (x.clone(), x))); + SubtypeOf.add_production(ClassType.map(|x| (x.clone(), x))); // SubtypeOf(?s, ?t) :- Subclass(?t, ?s). - SubtypeOf.add_production(&Subclass.map(|x| (x.1, x.0))); + SubtypeOf.add_production(Subclass.map(|x| (x.1, x.0))); // SubtypeOf(?s, ?t) :- isClassType(?s),Superinterface(?t, ?s). - SubtypeOf.add_production(&Superinterface.map(|x| (x.1, x.0)).semijoin(&ClassType)); + SubtypeOf.add_production(Superinterface.map(|x| (x.1, x.0)).semijoin(ClassType)); // SubtypeOf(?s, ?t) :- isInterfaceType(?s),isType(?t),?t = "java.lang.Object". let temp = interner.borrow_mut().intern("java.lang.Object"); - SubtypeOf.add_production(&InterfaceType.map(move |x| (x, temp.clone()))); + SubtypeOf.add_production(InterfaceType.map(move |x| (x, temp.clone()))); // SubtypeOf(?s, ?s) :- isInterfaceType(?s). - SubtypeOf.add_production(&InterfaceType.map(|x| (x.clone(), x))); + SubtypeOf.add_production(InterfaceType.map(|x| (x.clone(), x))); // SubtypeOf(?s, ?t) :- isInterfaceType(?s),Superinterface(?t, ?s). - SubtypeOf.add_production(&Superinterface.map(|x| (x.1, x.0)).semijoin(&InterfaceType)); + SubtypeOf.add_production(Superinterface.map(|x| (x.1, x.0)).semijoin(InterfaceType)); // SubtypeOf(?s, ?t) :- isArrayType(?s),isType(?t),?t = "java.lang.Object". let temp = interner.borrow_mut().intern("java.lang.Object"); - SubtypeOf.add_production(&ArrayType.map(move |x| (x, temp.clone()))); + SubtypeOf.add_production(ArrayType.map(move |x| (x, temp.clone()))); // SubtypeOf(?s, ?t) :- // ComponentType(?s, ?sc), // ComponentType(?t, ?tc), @@ -694,20 +694,20 @@ fn main() { // isReferenceType(?tc), // SubtypeOf(?sc, ?tc). SubtypeOf.add_production( - &ComponentType.map(|x| (x.1, x.0)) - .semijoin(&isReferenceType) - .join_map(&SubtypeOfClone, |_sc, s, tc| (tc.clone(), s.clone())) - .semijoin(&isReferenceType) - .join_map(&ComponentType.map(|x| (x.1, x.0)), |_tc, s, t| (s.clone(), t.clone())) + ComponentType.map(|x| (x.1, x.0)) + .semijoin(isReferenceType) + .join_map(SubtypeOfClone, |_sc, s, tc| (tc.clone(), s.clone())) + .semijoin(isReferenceType) + .join_map(ComponentType.map(|x| (x.1, x.0)), |_tc, s, t| (s.clone(), t.clone())) ); // SubtypeOf(?s, ?t) :- isArrayType(?s),isInterfaceType(?t),isType(?t),?t = "java.lang.Cloneable". let temp = interner.borrow_mut().intern("java.lang.Cloneable"); - SubtypeOf.add_production(&ArrayType.map(move |x| (x, temp.clone()))); + SubtypeOf.add_production(ArrayType.map(move |x| (x, temp.clone()))); // SubtypeOf(?s, ?t) :- isArrayType(?s),isInterfaceType(?t),isType(?t),?t = "java.io.Serializable". let temp = interner.borrow_mut().intern("java.io.Serializable"); - SubtypeOf.add_production(&ArrayType.map(move |x| (x, temp.clone()))); + SubtypeOf.add_production(ArrayType.map(move |x| (x, temp.clone()))); // SubtypeOf(?t, ?t) :- isType(?t). - SubtypeOf.add_production(&isType.map(|x| (x.clone(), x))); + SubtypeOf.add_production(isType.map(|x| (x.clone(), x))); // SubtypeOfDifferent(?s, ?t) :- SubtypeOf(?s, ?t),?s != ?t. let _SubtypeOfDifferent = SubtypeOf.filter(|x| x.0 != x.1); @@ -732,13 +732,13 @@ fn main() { let MainMethodDeclaration = Method_DeclaringType .map(|x| (x.1, x.0)) - .semijoin(&MainClass) + .semijoin(MainClass) .map(|x| (x.1, ())) .filter(move |x| x.0 != temp1 && x.0 != temp2 && x.0 != temp3) - .semijoin(&Method_SimpleName.filter(move |x| x.1 == temp4).map(|x| x.0)) - .semijoin(&Method_Descriptor.filter(move |x| x.1 == temp5).map(|x| x.0)) - .semijoin(&Method_Modifier.filter(move |x| x.0 == temp6).map(|x| x.1)) - .semijoin(&Method_Modifier.filter(move |x| x.0 == temp7).map(|x| x.1)) + .semijoin(Method_SimpleName.filter(move |x| x.1 == temp4).map(|x| x.0)) + .semijoin(Method_Descriptor.filter(move |x| x.1 == temp5).map(|x| x.0)) + .semijoin(Method_Modifier.filter(move |x| x.0 == temp6).map(|x| x.1)) + .semijoin(Method_Modifier.filter(move |x| x.0 == temp7).map(|x| x.1)) .map(|x| x.0); let result = (MethodImplemented.leave(), MainMethodDeclaration.leave(), MethodLookup.leave(), SupertypeOf.leave()); @@ -806,7 +806,7 @@ fn main() { Instruction_Method .as_collection(|inv,meth| (meth.clone(), inv.clone())) // .map(|(inv,meth)| (meth,inv)) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_meth,inv)| (inv, ())); // // NOTE: Cheating, but to test what is broken. @@ -826,21 +826,21 @@ fn main() { let InitializedClassClone = InitializedClass.clone(); // InitializedClass(?superclass) :- InitializedClass(?class),DirectSuperclass(?class, ?superclass). - InitializedClass.add_production(&DirectSuperclass.semijoin(&InitializedClassClone).map(|x| x.1)); + InitializedClass.add_production(DirectSuperclass.semijoin(InitializedClassClone).map(|x| x.1)); // InitializedClass(?superinterface) :- InitializedClass(?classOrInterface),DirectSuperinterface(?classOrInterface, ?superinterface). - InitializedClass.add_production(&DirectSuperinterface.semijoin(&InitializedClassClone).map(|x| x.1)); + InitializedClass.add_production(DirectSuperinterface.semijoin(InitializedClassClone).map(|x| x.1)); // InitializedClass(?class) :- basic.MainMethodDeclaration(?method),Method_DeclaringType(?method, ?class). - InitializedClass.add_production(&Method_DeclaringType.semijoin(&MainMethodDeclaration).map(|x| x.1)); + InitializedClass.add_production(Method_DeclaringType.semijoin(MainMethodDeclaration).map(|x| x.1)); // InitializedClass(?class) :- // Reachable(?inmethod), // AssignHeapAllocation(?heap, _, ?inmethod), // HeapAllocation_Type(?heap, ?class). InitializedClass.add_production( - &AssignHeapAllocation + AssignHeapAllocation .map(|(heap,_,inmethod)| (inmethod,heap)) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_inmethod,heap)| (heap, ())) - .join_map(&HeapAllocation_Type, |_,(),class| class.clone()) + .join_map(HeapAllocation_Type, |_,(),class| class.clone()) ); // InitializedClass(?class) :- // Reachable(?inmethod), @@ -849,10 +849,10 @@ fn main() { // MethodInvocation_Method(?invocation, ?signature), // Method_DeclaringType(?signature, ?class). InitializedClass.add_production( - &Reachable_Invocation - .semijoin(&isStaticMethodInvocation_Insn) - .join_map(&MethodInvocation_Method, |_,(),sig| (sig.clone(), ())) - .join_map(&Method_DeclaringType, |_,(),class| class.clone()) + Reachable_Invocation + .semijoin(isStaticMethodInvocation_Insn) + .join_map(MethodInvocation_Method, |_,(),sig| (sig.clone(), ())) + .join_map(Method_DeclaringType, |_,(),class| class.clone()) ); // InitializedClass(?classOrInterface) :- @@ -860,26 +860,26 @@ fn main() { // StoreStaticField(_, ?signature, ?inmethod), // Field_DeclaringType(?signature, ?classOrInterface). InitializedClass.add_production( - &StoreStaticField + StoreStaticField .map(|(_,sig,meth)| (meth,sig)) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_meth,sig)| (sig, ())) - .join_map(&Field_DeclaringType, |_,(),class| class.clone()) + .join_map(Field_DeclaringType, |_,(),class| class.clone()) ); // InitializedClass(?classOrInterface) :- // Reachable(?inmethod), // LoadStaticField(?signature, _, ?inmethod), // Field_DeclaringType(?signature, ?classOrInterface). InitializedClass.add_production( - &LoadStaticField + LoadStaticField .map(|x| (x.2, x.0)) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|x| (x.1, ())) - .join_map(&Field_DeclaringType, |_,_,class| class.clone()) + .join_map(Field_DeclaringType, |_,_,class| class.clone()) ); // Reachable(?clinit) :- InitializedClass(?class),ClassInitializer(?class, ?clinit). - Reachable.add_production(&ClassInitializer.semijoin(&InitializedClass).map(|x| x.1)); + Reachable.add_production(ClassInitializer.semijoin(InitializedClass).map(|x| x.1)); // Main (value-based) analysis let mut Assign = Relation::<_,(Var, Var)>::new(scope); @@ -900,17 +900,17 @@ fn main() { let ArrayIndexPointsTo = StoreArrayIndex .map(|(f,b,m)| (m,(f,b))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_m,(f,b))| (b,f)) - .join_core(&VarPointsToRev, |_b,f,bh| Some((f.clone(), bh.clone()))) - .join_core(&VarPointsToRev, |_f,bh,h| Some((h.clone(), bh.clone()))) - .join(&HeapAllocation_Type) + .join_core(VarPointsToRev, |_b,f,bh| Some((f.clone(), bh.clone()))) + .join_core(VarPointsToRev, |_f,bh,h| Some((h.clone(), bh.clone()))) + .join(HeapAllocation_Type) .map(|(h,(bh,ht))| (bh,(h,ht))) - .join(&HeapAllocation_Type) + .join(HeapAllocation_Type) .map(|(bh,((h,ht),bht))| (bht,(h,ht,bh))) - .join(&ComponentType) + .join(ComponentType) .map(|(_,((h,ht,bh),ct))| ((ct,ht),(bh,h))) - .semijoin(&SupertypeOf) + .semijoin(SupertypeOf) .map(|(_,(bh,h))| (bh,h)); // Assign(?actual, ?formal) :- @@ -918,11 +918,11 @@ fn main() { // FormalParam(?index, ?method, ?formal), // ActualParam(?index, ?invocation, ?actual). Assign.add_production( - &CallGraphEdge + CallGraphEdge .map(|x| (x.1, x.0)) - .join(&FormalParam.map(|x| (x.1, (x.0, x.2)))) + .join(FormalParam.map(|x| (x.1, (x.0, x.2)))) .map(|(_method, (inv, (index, formal)))| ((index, inv), formal)) - .join(&ActualParam.map(|x| ((x.0, x.1), x.2))) + .join(ActualParam.map(|x| ((x.0, x.1), x.2))) .map(|(_ind_inv, (formal, actual))| (actual, formal)) ); // Assign(?return, ?local) :- @@ -930,10 +930,10 @@ fn main() { // ReturnVar(?return, ?method), // AssignReturnValue(?invocation, ?local). Assign.add_production( - &CallGraphEdge - .join(&AssignReturnValue) + CallGraphEdge + .join(AssignReturnValue) .map(|(_inv, (meth, local))| (meth, local)) - .join(&ReturnVar.map(|x| (x.1, x.0))) + .join(ReturnVar.map(|x| (x.1, x.0))) .map(|(_meth, (local, ret))| (ret, local)) ); @@ -945,13 +945,13 @@ fn main() { let InstanceFieldPointsTo = StoreInstanceField .map(|(from, base, fld, meth)| (meth, (from, base, fld))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_, (from, base, fld))| (from, (base, fld))) - .join_core(&VarPointsToRev, |_from,(base,fld),heap| Some((base.clone(), (fld.clone(), heap.clone())))) - .join_core(&VarPointsToRev, |_base,(fld,heap),baseheap| Some((heap.clone(), fld.clone(), baseheap.clone()))); + .join_core(VarPointsToRev, |_from,(base,fld),heap| Some((base.clone(), (fld.clone(), heap.clone())))) + .join_core(VarPointsToRev, |_base,(fld,heap),baseheap| Some((heap.clone(), fld.clone(), baseheap.clone()))); // SIMPLIFICATION: ALL CGE DERIVATIONS PRODUCE REACHABILITY. - Reachable.add_production(&CallGraphEdge.map(|x| x.1)); + Reachable.add_production(CallGraphEdge.map(|x| x.1)); // CallGraphEdge(?invocation, ?toMethod) :- // Reachable(?inMethod), @@ -963,16 +963,16 @@ fn main() { // VirtualMethodInvocation_Descriptor(?invocation, ?descriptor), // basic.MethodLookup(?simplename, ?descriptor, ?heaptype, ?toMethod). CallGraphEdge.add_production( - &Reachable_Invocation - .join(&VirtualMethodInvocation_Base) + Reachable_Invocation + .join(VirtualMethodInvocation_Base) .map(|(inv, ((), base))| (base, inv)) - .join_core(&VarPointsToRev, |_base,inv,heap| Some((heap.clone(), inv.clone()))) - .join(&HeapAllocation_Type) + .join_core(VarPointsToRev, |_base,inv,heap| Some((heap.clone(), inv.clone()))) + .join(HeapAllocation_Type) .map(|(_heap, (inv, heaptype))| (inv, heaptype)) - .join(&VirtualMethodInvocation_SimpleName) - .join(&VirtualMethodInvocation_Descriptor) + .join(VirtualMethodInvocation_SimpleName) + .join(VirtualMethodInvocation_Descriptor) .map(|(inv, ((heaptype, simplename), descriptor))| ((simplename, descriptor, heaptype), inv)) - .join(&MethodLookup.map(|(s,d,h,t)| ((s,d,h),t))) + .join(MethodLookup.map(|(s,d,h,t)| ((s,d,h),t))) .map(|(_, (inv, to))| (inv, to)) ); @@ -980,9 +980,9 @@ fn main() { // Reachable(?inmethod), // StaticMethodInvocation(?invocation, ?tomethod, ?inmethod). CallGraphEdge.add_production( - &StaticMethodInvocation + StaticMethodInvocation .map(|x| (x.2, (x.0, x.1))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_inmethod, (inv, to))| (inv, to)) ); @@ -996,18 +996,18 @@ fn main() { // ThisVar(?tomethod, ?this). let temp = Reachable_Invocation - .join(&SpecialMethodInvocation_Base) - .join(&MethodInvocation_Method) + .join(SpecialMethodInvocation_Base) + .join(MethodInvocation_Method) .map(|(inv, (((), base), tomethod))| (base, (inv,tomethod))) - .join_core(&VarPointsToRev, |_base,(inv,tomethod),heap| Some((tomethod.clone(), (inv.clone(), heap.clone())))) - .join(&ThisVar) + .join_core(VarPointsToRev, |_base,(inv,tomethod),heap| Some((tomethod.clone(), (inv.clone(), heap.clone())))) + .join(ThisVar) .map(|(tomethod, ((inv,heap),this))| (inv, tomethod, heap, this)); - CallGraphEdge.add_production(&temp.map(|(i,t,_,_)| (i,t))); - VarPointsTo.add_production(&temp.map(|(_,_,h,t)| (h,t))); + CallGraphEdge.add_production(temp.map(|(i,t,_,_)| (i,t))); + VarPointsTo.add_production(temp.map(|(_,_,h,t)| (h,t))); // Reachable(?method) :- basic.MainMethodDeclaration(?method). - Reachable.add_production(&MainMethodDeclaration); + Reachable.add_production(MainMethodDeclaration); // StaticFieldPointsTo(?heap, ?fld) :- // Reachable(?inmethod), @@ -1016,26 +1016,26 @@ fn main() { let StaticFieldPointsTo = StoreStaticField .map(|(from, fld, meth)| (meth, (from, fld))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_meth, (from, fld))| (from, fld)) - .join_core(&VarPointsToRev, |_from,fld,heap| Some((heap.clone(), fld.clone()))); + .join_core(VarPointsToRev, |_from,fld,heap| Some((heap.clone(), fld.clone()))); let VarPointsToClone = VarPointsTo.clone(); // VarPointsTo(?heap, ?var) :- // AssignHeapAllocation(?heap, ?var, ?inMethod), // Reachable(?inMethod). VarPointsTo.add_production( - &AssignHeapAllocation + AssignHeapAllocation .map(|x| (x.2, (x.0, x.1))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|x| x.1) ); // VarPointsTo(?heap, ?to) :- Assign(?from, ?to),VarPointsTo(?heap, ?from). VarPointsTo.add_production( - &VarPointsToClone + VarPointsToClone .map(|x| (x.1, x.0)) - .join(&Assign) + .join(Assign) .map(|(_from, (heap, to))| (heap, to)) ); @@ -1044,11 +1044,11 @@ fn main() { // AssignLocal(?from, ?to, ?inmethod), // VarPointsTo(?heap, ?from). VarPointsTo.add_production( - &AssignLocal + AssignLocal .map(|(from, to, meth)| (meth, (from, to))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_, (from, to))| (from, to)) - .join_core(&VarPointsToRev, |_from,to,heap| Some((heap.clone(), to.clone()))) + .join_core(VarPointsToRev, |_from,to,heap| Some((heap.clone(), to.clone()))) ); // VarPointsTo(?heap, ?to) :- @@ -1058,15 +1058,15 @@ fn main() { // HeapAllocation_Type(?heap, ?heaptype), // VarPointsTo(?heap, ?from). VarPointsTo.add_production( - &AssignCast + AssignCast .map(|(ty,f,to,m)| (m, (ty,f,to))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_m, (ty,f,to))| (ty, (f,to))) - .join(&SupertypeOf) + .join(SupertypeOf) .map(|(_ty, ((f,to), heaptype))| (heaptype, (f,to))) - .join(&HeapAllocation_Type.map(|x| (x.1, x.0))) + .join(HeapAllocation_Type.map(|x| (x.1, x.0))) .map(|(_heaptype, ((f,to),heap))| ((heap, f), to)) - .semijoin(&VarPointsToClone) + .semijoin(VarPointsToClone) .map(|((heap, _f), to)| (heap, to)) ); @@ -1080,19 +1080,19 @@ fn main() { // ComponentType(?baseheaptype, ?basecomponenttype), // basic.SupertypeOf(?type, ?basecomponenttype). VarPointsTo.add_production( - &LoadArrayIndex + LoadArrayIndex .map(|(b,t,m)| (m, (b,t))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_m, (b,t))| (b,t)) - .join_core(&VarPointsToRev, |_base,to,baseheap| Some((baseheap.clone(), to.clone()))) - .join(&ArrayIndexPointsTo) - .join(&HeapAllocation_Type) + .join_core(VarPointsToRev, |_base,to,baseheap| Some((baseheap.clone(), to.clone()))) + .join(ArrayIndexPointsTo) + .join(HeapAllocation_Type) .map(|(_baseheap, ((to, heap), baseheaptype))| (to, (heap, baseheaptype))) - .join(&Var_Type) + .join(Var_Type) .map(|(to, ((h,bht),ty))| (bht,(to,h,ty))) - .join(&ComponentType) + .join(ComponentType) .map(|(_bht, ((to,h,ty),bct))| ((ty,bct), (to,h))) - .semijoin(&SupertypeOf) + .semijoin(SupertypeOf) .map(|(_, (to,h))| (h,to)) ); @@ -1102,12 +1102,12 @@ fn main() { // VarPointsTo(?baseheap, ?base), // InstanceFieldPointsTo(?heap, ?signature, ?baseheap). VarPointsTo.add_production( - &LoadInstanceField + LoadInstanceField .map(|(b,s,t,m)| (m, (b,s,t))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_m, (b,s,t))| (b,(s,t))) - .join_core(&VarPointsToRev, |_b,(s,t),bh| Some(((s.clone(),bh.clone()), t.clone()))) - .join(&InstanceFieldPointsTo.map(|(h,s,bh)| ((s,bh),h))) + .join_core(VarPointsToRev, |_b,(s,t),bh| Some(((s.clone(),bh.clone()), t.clone()))) + .join(InstanceFieldPointsTo.map(|(h,s,bh)| ((s,bh),h))) .map(|(_, (t,h))| (h,t)) ); @@ -1116,11 +1116,11 @@ fn main() { // LoadStaticField(?fld, ?to, ?inmethod), // StaticFieldPointsTo(?heap, ?fld). VarPointsTo.add_production( - &LoadStaticField + LoadStaticField .map(|(f,t,m)| (m, (f,t))) - .semijoin(&Reachable) + .semijoin(Reachable) .map(|(_m,(f,t))| (f,t)) - .join(&StaticFieldPointsTo.map(|x| (x.1, x.0))) + .join(StaticFieldPointsTo.map(|x| (x.1, x.0))) .map(|(_f,(t,h))| (h,t)) ); @@ -1135,17 +1135,17 @@ fn main() { // basic.MethodLookup(?simplename, ?descriptor, ?heaptype, ?toMethod), // ThisVar(?toMethod, ?this). VarPointsTo.add_production( - &Reachable_Invocation - .join(&VirtualMethodInvocation_Base) - .join(&VirtualMethodInvocation_SimpleName) - .join(&VirtualMethodInvocation_Descriptor) + Reachable_Invocation + .join(VirtualMethodInvocation_Base) + .join(VirtualMethodInvocation_SimpleName) + .join(VirtualMethodInvocation_Descriptor) .map(|(_inv, ((((), base), simplename), descriptor))| (base, (simplename, descriptor))) - .join_core(&VarPointsToRev, |_base,(name,desc),heap| Some((heap.clone(), (name.clone(), desc.clone())))) - .join(&HeapAllocation_Type) + .join_core(VarPointsToRev, |_base,(name,desc),heap| Some((heap.clone(), (name.clone(), desc.clone())))) + .join(HeapAllocation_Type) .map(|(heap,((name,desc),heaptype))| ((name,desc,heaptype),heap)) - .join(&MethodLookup.map(|(n,d,h,t)| ((n,d,h),t))) + .join(MethodLookup.map(|(n,d,h,t)| ((n,d,h),t))) .map(|((_n,_d,_t),(heap,to_method))| (to_method, heap)) - .join(&ThisVar) + .join(ThisVar) .map(|(_to_method, (heap, this))| (heap,this)) ); diff --git a/experiments/src/bin/attend.rs b/experiments/src/bin/attend.rs index e15c176e8..94bf6c699 100644 --- a/experiments/src/bin/attend.rs +++ b/experiments/src/bin/attend.rs @@ -26,7 +26,7 @@ fn main() { .semijoin(attend) .map(|(_,y)| y) .threshold_total(|_,w| if w >= &3 { 1 } else { 0 }) - .concat(&organizers.enter(&attend.scope())) + .concat(organizers.enter(&attend.scope())) .consolidate() }) .map(|_| ()) diff --git a/experiments/src/bin/deals-interactive.rs b/experiments/src/bin/deals-interactive.rs index 17b053003..02bee9438 100644 --- a/experiments/src/bin/deals-interactive.rs +++ b/experiments/src/bin/deals-interactive.rs @@ -218,8 +218,8 @@ where G::Timestamp: Lattice{ .iterate(|inner| edges .enter(&inner.scope()) - .join_core(&inner.arrange_by_key(), |_,&y,&q| [(y,q)]) - .concat(&tc_1.enter(&inner.scope()).map(|x| (x,x))) + .join_core(inner.arrange_by_key(), |_,&y,&q| [(y,q)]) + .concat(tc_1.enter(&inner.scope()).map(|x| (x,x))) .distinct() ) .map(|(x,q)| (q,x)); @@ -231,8 +231,8 @@ where G::Timestamp: Lattice{ edges .as_collection(|&k,&v| (v,k)) .enter(&inner.scope()) - .join_core(&inner.arrange_by_key(), |_,&y,&q| [(y,q)]) - .concat(&tc_2.enter(&inner.scope()).map(|x| (x,x))) + .join_core(inner.arrange_by_key(), |_,&y,&q| [(y,q)]) + .concat(tc_2.enter(&inner.scope()).map(|x| (x,x))) .distinct() ) .map(|(x,q)| (q,x)); @@ -247,17 +247,17 @@ where G::Timestamp: Lattice{ edges .as_collection(|&k,&v| (v,k)) .enter(&inner.scope()) - .semijoin(&inner) + .semijoin(inner) .map(|(_x,y)| y) - .concat(&sg_x.enter(&inner.scope())) + .concat(sg_x.enter(&inner.scope())) .distinct() ); let magic_edges = edges - .join_core(&magic.arrange_by_self(), |k,v,_| [(k.clone(), v.clone())]) + .join_core(magic.arrange_by_self(), |k,v,_| [(k.clone(), v.clone())]) .map(|(x,y)|(y,x)) - .semijoin(&magic) + .semijoin(magic) .map(|(x,y)|(y,x)); let query3 = @@ -271,15 +271,15 @@ where G::Timestamp: Lattice{ let result = inner - .join_map(&magic_edges, |_x,&y,&cx| (y,cx)) - .join_core(&edges, |_y,&cx,&cy| Some((cx,cy))) - .concat(&magic.map(|x| (x,x))) + .join_map(magic_edges, |_x,&y,&cx| (y,cx)) + .join_core(edges, |_y,&cx,&cy| Some((cx,cy))) + .concat(magic.map(|x| (x,x))) .distinct(); // result.map(|_| ()).consolidate().inspect(|x| println!("\t{:?}", x)); result }) - .semijoin(&sg_x); + .semijoin(sg_x); - query1.concat(&query2).concat(&query3).map(|(q,_)| q) + query1.concat(query2).concat(query3).map(|(q,_)| q) } diff --git a/experiments/src/bin/deals.rs b/experiments/src/bin/deals.rs index 7730a449d..e2457ebbd 100644 --- a/experiments/src/bin/deals.rs +++ b/experiments/src/bin/deals.rs @@ -95,13 +95,13 @@ fn tc>(edges: &EdgeArranged) -> V inner .map(|(x,y)| (y,x)) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&edges, |_y,&x,&z| Some((x, z))) - .concat(&edges.as_collection(|&k,&v| (k,v))) + .join_core(edges, |_y,&x,&z| Some((x, z))) + .concat(edges.as_collection(|&k,&v| (k,v))) .arrange::, KeyBuilder<_,_,_>, KeySpine<_,_,_>>() .threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None }) ; - inner.set(&result); + inner.set(result); result.leave() } ) @@ -110,7 +110,7 @@ fn tc>(edges: &EdgeArranged) -> V // returns pairs (n, s) indicating node n can be reached from a root in s steps. fn sg>(edges: &EdgeArranged) -> VecCollection { - let peers = edges.join_core(&edges, |_,&x,&y| Some((x,y))).filter(|&(x,y)| x != y); + let peers = edges.join_core(edges, |_,&x,&y| Some((x,y))).filter(|&(x,y)| x != y); // repeatedly update minimal distances each node can be reached from each root peers.scope().iterative::(|scope| { @@ -122,15 +122,15 @@ fn sg>(edges: &EdgeArranged) -> V let result = inner .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&edges, |_,&x,&z| Some((x, z))) + .join_core(edges, |_,&x,&z| Some((x, z))) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&edges, |_,&x,&z| Some((x, z))) - .concat(&peers) + .join_core(edges, |_,&x,&z| Some((x, z))) + .concat(peers) .arrange::, KeyBuilder<_,_,_>, KeySpine<_,_,_>>() .threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None }) ; - inner.set(&result); + inner.set(result); result.leave() } ) diff --git a/experiments/src/bin/graphs-interactive-alt.rs b/experiments/src/bin/graphs-interactive-alt.rs index 3a566b384..db84a4e81 100644 --- a/experiments/src/bin/graphs-interactive-alt.rs +++ b/experiments/src/bin/graphs-interactive-alt.rs @@ -60,25 +60,25 @@ fn main() { let state_indexed = state.arrange_by_key(); let graph_indexed = graph.map(|(src, dst)| (dst, src)) - .concat(&graph) + .concat(graph) .arrange_by_key(); // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(&state_indexed, |&query, &(), &state| Some((query, state))) + .join_core(state_indexed, |&query, &(), &state| Some((query, state))) .probe_with(&mut probe); // Q2: One-hop lookups on `state`: q2 .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&state_indexed, |_friend, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(state_indexed, |_friend, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q3: Two-hop lookups on `state`: q3 .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) - .join_core(&state_indexed, |_friend2, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) + .join_core(state_indexed, |_friend2, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q4: Shortest path queries: @@ -93,24 +93,24 @@ fn main() { // let state_indexed = state.arrange_by_key(); let graph = graph.map(|(src, dst)| (dst, src)) - .concat(&graph); + .concat(graph); // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(&state.arrange_by_key(), |&query, &(), &state| Some((query, state))) + .join_core(state.arrange_by_key(), |&query, &(), &state| Some((query, state))) .probe_with(&mut probe); // Q2: One-hop lookups on `state`: q2 .arrange_by_self() - .join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(&state.arrange_by_key(), |_friend, &query, &state| Some((query, state))) + .join_core(graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) + .join_core(state.arrange_by_key(), |_friend, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q3: Two-hop lookups on `state`: q3 .arrange_by_self() - .join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(&graph.arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query))) - .join_core(&state.arrange_by_key(), |_friend2, &query, &state| Some((query, state))) + .join_core(graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) + .join_core(graph.arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query))) + .join_core(state.arrange_by_key(), |_friend2, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q4: Shortest path queries: @@ -264,7 +264,7 @@ type Arrange = Arranged( forward_graph: &Arrange, reverse_graph: &Arrange, - goals: &VecCollection) -> VecCollection + goals: VecCollection) -> VecCollection where G::Timestamp: Lattice+Ord { let sources = goals.map(|(x,_)| x); @@ -272,18 +272,18 @@ where G::Timestamp: Lattice+Ord { // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward2 = forward1.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse2 = reverse1.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let forward = forward0.concat(&forward1).concat(&forward2); - let reverse = reverse0.concat(&reverse1).concat(&reverse2); + let forward = forward0.concat(forward1).concat(forward2); + let reverse = reverse0.concat(reverse1).concat(reverse2); forward - .join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) + .join_map(reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) .reduce(|_st,input,output| output.push((*input[0].0,1))) } @@ -291,7 +291,7 @@ where G::Timestamp: Lattice+Ord { fn _bidijkstra( forward_graph: &Arrange, reverse_graph: &Arrange, - goals: &VecCollection) -> VecCollection + goals: VecCollection) -> VecCollection where G::Timestamp: Lattice+Ord { goals.scope().iterative::(|inner| { @@ -315,15 +315,15 @@ where G::Timestamp: Lattice+Ord { // This is a cyclic join, which should scare us a bunch. let reached = forward - .join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) + .join_map(reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) - .semijoin(&goals); + .semijoin(goals); let active = reached .negate() .map(|(srcdst,_)| srcdst) - .concat(&goals) + .concat(goals) .consolidate(); // Let's expand out forward queries that are active. @@ -331,30 +331,30 @@ where G::Timestamp: Lattice+Ord { let forward_next = forward .map(|(med, (src, dist))| (src, (med, dist))) - .semijoin(&forward_active) + .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) - .join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) - .concat(&forward) + .join_core(forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) + .concat(forward) .map(|(next, (src, dist))| ((next, src), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); - forward.set(&forward_next); + forward.set(forward_next); // Let's expand out reverse queries that are active. let reverse_active = active.map(|(_x,y)| y).distinct(); let reverse_next = reverse .map(|(med, (rev, dist))| (rev, (med, dist))) - .semijoin(&reverse_active) + .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) - .join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) - .concat(&reverse) + .join_core(reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) + .concat(reverse) .map(|(next, (rev, dist))| ((next, rev), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); - reverse.set(&reverse_next); + reverse.set(reverse_next); reached.leave() }) @@ -383,10 +383,10 @@ where G::Timestamp: Lattice + std::hash::Hash { let inner = inner.arrange_by_key(); - let prop = inner.join_core(&graph, |_k,l,d| Some((*d,*l))); + let prop = inner.join_core(graph, |_k,l,d| Some((*d,*l))); nodes - .concat(&prop) + .concat(prop) .reduce(|_, s, t| { t.push((*s[0].0, 1)); }) }) } diff --git a/experiments/src/bin/graphs-interactive-neu-zwei.rs b/experiments/src/bin/graphs-interactive-neu-zwei.rs index 9f365e5d6..8a802ecb5 100644 --- a/experiments/src/bin/graphs-interactive-neu-zwei.rs +++ b/experiments/src/bin/graphs-interactive-neu-zwei.rs @@ -47,7 +47,7 @@ fn main() { let state_indexed = state.arrange_by_key(); let graph_indexed = graph.map(|(src, dst)| (dst, src)) - .concat(&graph) + .concat(graph) .arrange_by_key(); match mode { @@ -57,7 +57,7 @@ fn main() { query .map(|(x,_)| x) .arrange_by_self() - .join_core(&state_indexed, |&query, &(), &state| Some((query, state))) + .join_core(state_indexed, |&query, &(), &state| Some((query, state))) .probe_with(&mut probe); }, 2 => { @@ -65,8 +65,8 @@ fn main() { query .map(|(x,_)| x) .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&state_indexed, |_friend, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(state_indexed, |_friend, &query, &state| Some((query, state))) .probe_with(&mut probe); }, 3 => { @@ -74,9 +74,9 @@ fn main() { query .map(|(x,_)| x) .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) - .join_core(&state_indexed, |_friend2, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) + .join_core(state_indexed, |_friend2, &query, &state| Some((query, state))) .probe_with(&mut probe); }, 4 => { @@ -232,7 +232,7 @@ type Arrange = Arranged( forward_graph: &Arrange, reverse_graph: &Arrange, - goals: &VecCollection) -> VecCollection + goals: VecCollection) -> VecCollection where G::Timestamp: Lattice+Ord { let sources = goals.map(|(x,_)| x); @@ -240,18 +240,18 @@ where G::Timestamp: Lattice+Ord { // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward2 = forward1.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse2 = reverse1.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let forward = forward0.concat(&forward1).concat(&forward2); - let reverse = reverse0.concat(&reverse1).concat(&reverse2); + let forward = forward0.concat(forward1).concat(forward2); + let reverse = reverse0.concat(reverse1).concat(reverse2); forward - .join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) + .join_map(reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) .reduce(|_st,input,output| output.push((*input[0].0,1))) } @@ -259,7 +259,7 @@ where G::Timestamp: Lattice+Ord { // fn bidijkstra( // forward_graph: &Arrange, // reverse_graph: &Arrange, -// goals: &VecCollection, +// goals: VecCollection, // bound: u64) -> VecCollection // where G::Timestamp: Lattice+Ord { @@ -284,15 +284,15 @@ where G::Timestamp: Lattice+Ord { // // This is a cyclic join, which should scare us a bunch. // let reached = // forward -// .join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) +// .join_map(reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) // .reduce(|_key, s, t| t.push((*s[0].0, 1))) -// .semijoin(&goals); +// .semijoin(goals); // let active = // reached // .negate() // .map(|(srcdst,_)| srcdst) -// .concat(&goals) +// .concat(goals) // .consolidate(); // // Let's expand out forward queries that are active. @@ -300,30 +300,30 @@ where G::Timestamp: Lattice+Ord { // let forward_next = // forward // .map(|(med, (src, dist))| (src, (med, dist))) -// .semijoin(&forward_active) +// .semijoin(forward_active) // .map(|(src, (med, dist))| (med, (src, dist))) -// .join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) -// .concat(&forward) +// .join_core(forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) +// .concat(forward) // .map(|(next, (src, dist))| ((next, src), dist)) // .reduce(|_key, s, t| t.push((*s[0].0, 1))) // .map(|((next, src), dist)| (next, (src, dist))); -// forward.set(&forward_next); +// forward.set(forward_next); // // Let's expand out reverse queries that are active. // let reverse_active = active.map(|(_x,y)| y).distinct(); // let reverse_next = // reverse // .map(|(med, (rev, dist))| (rev, (med, dist))) -// .semijoin(&reverse_active) +// .semijoin(reverse_active) // .map(|(rev, (med, dist))| (med, (rev, dist))) -// .join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) -// .concat(&reverse) +// .join_core(reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) +// .concat(reverse) // .map(|(next, (rev, dist))| ((next, rev), dist)) // .reduce(|_key, s, t| t.push((*s[0].0, 1))) // .map(|((next,rev), dist)| (next, (rev, dist))); -// reverse.set(&reverse_next); +// reverse.set(reverse_next); // reached.leave() // }) diff --git a/experiments/src/bin/graphs-interactive-neu.rs b/experiments/src/bin/graphs-interactive-neu.rs index 1aed65bec..1670b3afb 100644 --- a/experiments/src/bin/graphs-interactive-neu.rs +++ b/experiments/src/bin/graphs-interactive-neu.rs @@ -59,25 +59,25 @@ fn main() { let state_indexed = state.arrange_by_key(); let graph_indexed = graph.map(|(src, dst)| (dst, src)) - .concat(&graph) + .concat(graph) .arrange_by_key(); // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(&state_indexed, |&query, &(), &state| Some((query, state))) + .join_core(state_indexed, |&query, &(), &state| Some((query, state))) .probe_with(&mut probe); // Q2: One-hop lookups on `state`: q2 .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&state_indexed, |_friend, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(state_indexed, |_friend, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q3: Two-hop lookups on `state`: q3 .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) - .join_core(&state_indexed, |_friend2, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) + .join_core(state_indexed, |_friend2, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q4: Shortest path queries: @@ -89,24 +89,24 @@ fn main() { // let state_indexed = state.arrange_by_key(); let graph = graph.map(|(src, dst)| (dst, src)) - .concat(&graph); + .concat(graph); // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(&state.arrange_by_key(), |&query, &(), &state| Some((query, state))) + .join_core(state.arrange_by_key(), |&query, &(), &state| Some((query, state))) .probe_with(&mut probe); // Q2: One-hop lookups on `state`: q2 .arrange_by_self() - .join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(&state.arrange_by_key(), |_friend, &query, &state| Some((query, state))) + .join_core(graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) + .join_core(state.arrange_by_key(), |_friend, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q3: Two-hop lookups on `state`: q3 .arrange_by_self() - .join_core(&graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(&graph.arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query))) - .join_core(&state.arrange_by_key(), |_friend2, &query, &state| Some((query, state))) + .join_core(graph.arrange_by_key(), |&query, &(), &friend| Some((friend, query))) + .join_core(graph.arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query))) + .join_core(state.arrange_by_key(), |_friend2, &query, &state| Some((query, state))) .probe_with(&mut probe); // Q4: Shortest path queries: @@ -295,9 +295,9 @@ type Arrange = Arranged( - forward_graph: &Arrange, - reverse_graph: &Arrange, - goals: &VecCollection) -> VecCollection + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection where G::Timestamp: Lattice+Ord { let sources = goals.map(|(x,_)| x); @@ -305,26 +305,26 @@ where G::Timestamp: Lattice+Ord { // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward2 = forward1.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse2 = reverse1.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let forward = forward0.concat(&forward1).concat(&forward2); - let reverse = reverse0.concat(&reverse1).concat(&reverse2); + let forward = forward0.concat(forward1).concat(forward2); + let reverse = reverse0.concat(reverse1).concat(reverse2); forward - .join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) + .join_map(reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) .reduce(|_st,input,output| output.push((*input[0].0,1))) } // returns pairs (n, s) indicating node n can be reached from a root in s steps. fn _bidijkstra( - forward_graph: &Arrange, - reverse_graph: &Arrange, - goals: &VecCollection) -> VecCollection + forward_graph: Arrange, + reverse_graph: Arrange, + goals: VecCollection) -> VecCollection where G::Timestamp: Lattice+Ord { goals.scope().iterative::(|inner| { @@ -348,15 +348,15 @@ where G::Timestamp: Lattice+Ord { // This is a cyclic join, which should scare us a bunch. let reached = forward - .join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) + .join_map(reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) - .semijoin(&goals); + .semijoin(goals); let active = reached .negate() .map(|(srcdst,_)| srcdst) - .concat(&goals) + .concat(goals) .consolidate(); // Let's expand out forward queries that are active. @@ -364,30 +364,30 @@ where G::Timestamp: Lattice+Ord { let forward_next = forward .map(|(med, (src, dist))| (src, (med, dist))) - .semijoin(&forward_active) + .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) - .join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) - .concat(&forward) + .join_core(forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) + .concat(forward) .map(|(next, (src, dist))| ((next, src), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); - forward.set(&forward_next); + forward.set(forward_next); // Let's expand out reverse queries that are active. let reverse_active = active.map(|(_x,y)| y).distinct(); let reverse_next = reverse .map(|(med, (rev, dist))| (rev, (med, dist))) - .semijoin(&reverse_active) + .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) - .join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) - .concat(&reverse) + .join_core(reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) + .concat(reverse) .map(|(next, (rev, dist))| ((next, rev), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); - reverse.set(&reverse_next); + reverse.set(reverse_next); reached.leave() }) diff --git a/experiments/src/bin/graphs-interactive.rs b/experiments/src/bin/graphs-interactive.rs index c7549fe96..928cf8b1a 100644 --- a/experiments/src/bin/graphs-interactive.rs +++ b/experiments/src/bin/graphs-interactive.rs @@ -52,29 +52,29 @@ fn main() { let state_indexed = state.arrange_by_key(); let graph_indexed = graph.map(|(src, dst)| (dst, src)) - .concat(&graph) + .concat(graph) .arrange_by_key(); // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(&state_indexed, |&query, &(), &state| Some((query, state))) + .join_core(state_indexed, |&query, &(), &state| Some((query, state))) // .filter(move |_| inspect) // .inspect(|x| println!("Q1: {:?}", x)) .probe_with(&mut probe); // Q2: One-hop lookups on `state`: q2 .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&state_indexed, |_friend, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(state_indexed, |_friend, &query, &state| Some((query, state))) // .filter(move |_| inspect) // .inspect(|x| println!("Q2: {:?}", x)) .probe_with(&mut probe); // Q3: Two-hop lookups on `state`: q3 .arrange_by_self() - .join_core(&graph_indexed, |&query, &(), &friend| Some((friend, query))) - .join_core(&graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) - .join_core(&state_indexed, |_friend2, &query, &state| Some((query, state))) + .join_core(graph_indexed, |&query, &(), &friend| Some((friend, query))) + .join_core(graph_indexed, |_friend, &query, &friend2| Some((friend2, query))) + .join_core(state_indexed, |_friend2, &query, &state| Some((query, state))) // .filter(move |_| inspect) // .consolidate() // .inspect(|x| println!("Q3: {:?}", x)) @@ -201,7 +201,7 @@ type Arrange = Arranged( forward_graph: &Arrange, reverse_graph: &Arrange, - goals: &VecCollection) -> VecCollection + goals: VecCollection) -> VecCollection where G::Timestamp: Lattice+Ord { let sources = goals.map(|(x,_)| x); @@ -209,18 +209,18 @@ where G::Timestamp: Lattice+Ord { // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.join_core(&forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward2 = forward1.join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.join_core(&reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse2 = reverse1.join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let forward = forward0.concat(&forward1).concat(&forward2); - let reverse = reverse0.concat(&reverse1).concat(&reverse2); + let forward = forward0.concat(forward1).concat(forward2); + let reverse = reverse0.concat(reverse1).concat(reverse2); forward - .join_map(&reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) + .join_map(reverse, |_,&(source, dist1),&(target, dist2)| ((source, target), dist1 + dist2)) .reduce(|_st,input,output| output.push((*input[0].0,1))) } @@ -228,7 +228,7 @@ where G::Timestamp: Lattice+Ord { fn _bidijkstra( forward_graph: &Arrange, reverse_graph: &Arrange, - goals: &VecCollection) -> VecCollection + goals: VecCollection) -> VecCollection where G::Timestamp: Lattice+Ord { goals.scope().iterative::(|inner| { @@ -252,15 +252,15 @@ where G::Timestamp: Lattice+Ord { // This is a cyclic join, which should scare us a bunch. let reached = forward - .join_map(&reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) + .join_map(reverse, |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) - .semijoin(&goals); + .semijoin(goals); let active = reached .negate() .map(|(srcdst,_)| srcdst) - .concat(&goals) + .concat(goals) .consolidate(); // Let's expand out forward queries that are active. @@ -268,30 +268,30 @@ where G::Timestamp: Lattice+Ord { let forward_next = forward .map(|(med, (src, dist))| (src, (med, dist))) - .semijoin(&forward_active) + .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) - .join_core(&forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) - .concat(&forward) + .join_core(forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) + .concat(forward) .map(|(next, (src, dist))| ((next, src), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); - forward.set(&forward_next); + forward.set(forward_next); // Let's expand out reverse queries that are active. let reverse_active = active.map(|(_x,y)| y).distinct(); let reverse_next = reverse .map(|(med, (rev, dist))| (rev, (med, dist))) - .semijoin(&reverse_active) + .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) - .join_core(&reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) - .concat(&reverse) + .join_core(reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) + .concat(reverse) .map(|(next, (rev, dist))| ((next, rev), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); - reverse.set(&reverse_next); + reverse.set(reverse_next); reached.leave() }) diff --git a/experiments/src/bin/graphs-static.rs b/experiments/src/bin/graphs-static.rs index 7255b5a72..3493de8a3 100644 --- a/experiments/src/bin/graphs-static.rs +++ b/experiments/src/bin/graphs-static.rs @@ -121,11 +121,11 @@ fn reach> ( let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1)); let result = - graph.join_core(&inner.arrange_by_self(), |_src,&dst,&()| Some(dst)) - .concat(&roots) + graph.join_core(inner.arrange_by_self(), |_src,&dst,&()| Some(dst)) + .concat(roots) .threshold_total(|_,_| 1); - inner.set(&result); + inner.set(result); result.leave() }) } @@ -146,11 +146,11 @@ fn bfs> ( let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1)); let result = - graph.join_core(&inner.arrange_by_key(), |_src,&dest,&dist| [(dest, dist+1)]) - .concat(&roots) + graph.join_core(inner.arrange_by_key(), |_src,&dest,&dist| [(dest, dist+1)]) + .concat(roots) .reduce(|_key, input, output| output.push((*input[0].0,1))); - inner.set(&result); + inner.set(result); result.leave() }) } @@ -167,7 +167,7 @@ fn connected_components>( // each edge (x,y) means that we need at least a label for the min of x and y. let nodes_f = forward.flat_map_ref(|k,v| if k < v { Some(*k) } else { None }); let nodes_r = reverse.flat_map_ref(|k,v| if k < v { Some(*k) } else { None }); - let nodes = nodes_f.concat(&nodes_r).consolidate().map(|x| (x,x)); + let nodes = nodes_f.concat(nodes_r).consolidate().map(|x| (x,x)); scope.iterative(|scope| { @@ -179,8 +179,8 @@ fn connected_components>( let inner = SemigroupVariable::new(scope, Product::new(Default::default(), 1)); let labels = inner.arrange_by_key(); - let f_prop = labels.join_core(&forward, |_k,l,d| Some((*d,*l))); - let r_prop = labels.join_core(&reverse, |_k,l,d| Some((*d,*l))); + let f_prop = labels.join_core(forward, |_k,l,d| Some((*d,*l))); + let r_prop = labels.join_core(reverse, |_k,l,d| Some((*d,*l))); use timely::dataflow::operators::{Map, Concat, Delay}; @@ -188,14 +188,14 @@ fn connected_components>( nodes .inner .map_in_place(|dtr| (dtr.1).inner = 256 * ((((::std::mem::size_of::() * 8) as u32) - (dtr.0).1.leading_zeros()))) - .concat(&inner.filter(|_| false).inner) + .concat(inner.filter(|_| false).inner) .delay(|dtr,_| dtr.1.clone()) .as_collection() - .concat(&f_prop) - .concat(&r_prop) + .concat(f_prop) + .concat(r_prop) .reduce(|_, s, t| { t.push((*s[0].0, 1)); }); - inner.set(&result); + inner.set(result); result.leave() }) } diff --git a/experiments/src/bin/graphs.rs b/experiments/src/bin/graphs.rs index ee5ed5791..44433a2c8 100644 --- a/experiments/src/bin/graphs.rs +++ b/experiments/src/bin/graphs.rs @@ -100,11 +100,11 @@ fn reach> ( let graph = graph.enter(&inner.scope()); let roots = roots.enter(&inner.scope()); - // let reach = inner.concat(&roots).distinct_total().arrange_by_self(); - // graph.join_core(&reach, |_src,&dst,&()| Some(dst)) + // let reach = inner.concat(roots).distinct_total().arrange_by_self(); + // graph.join_core(reach, |_src,&dst,&()| Some(dst)) - graph.join_core(&inner.arrange_by_self(), |_src,&dst,&()| Some(dst)) - .concat(&roots) + graph.join_core(inner.arrange_by_self(), |_src,&dst,&()| Some(dst)) + .concat(roots) .distinct_total() }) } @@ -123,8 +123,8 @@ fn bfs> ( let graph = graph.enter(&inner.scope()); let roots = roots.enter(&inner.scope()); - graph.join_core(&inner.arrange_by_key(), |_src,&dest,&dist| [(dest, dist+1)]) - .concat(&roots) + graph.join_core(inner.arrange_by_key(), |_src,&dest,&dist| [(dest, dist+1)]) + .concat(roots) .reduce(|_key, input, output| output.push((*input[0].0,1))) }) } @@ -144,7 +144,7 @@ fn bfs> ( // // each edge should exist in both directions. // let edges = edges.map_in_place(|x| mem::swap(&mut x.0, &mut x.1)) -// .concat(&edges); +// .concat(edges); // // don't actually use these labels, just grab the type // nodes.filter(|_| false) @@ -152,8 +152,8 @@ fn bfs> ( // let edges = edges.enter(&inner.scope()); // let nodes = nodes.enter_at(&inner.scope(), |r| 256 * (64 - r.1.leading_zeros() as u64)); -// inner.join_map(&edges, |_k,l,d| (*d,*l)) -// .concat(&nodes) +// inner.join_map(edges, |_k,l,d| (*d,*l)) +// .concat(nodes) // .group(|_, s, t| { t.push((*s[0].0, 1)); } ) // }) // } diff --git a/experiments/src/bin/graspan-interactive.rs b/experiments/src/bin/graspan-interactive.rs index 1ea5002a0..b408a6c01 100644 --- a/experiments/src/bin/graspan-interactive.rs +++ b/experiments/src/bin/graspan-interactive.rs @@ -34,8 +34,8 @@ fn main() { inner .map(|(a,b)| (b,a)) - .join_core(&edges, |_b,&a,&c| Some((a,c))) - .concat(&nodes) + .join_core(edges, |_b,&a,&c| Some((a,c))) + .concat(nodes) .distinct() }) .probe(); diff --git a/experiments/src/bin/graspan1.rs b/experiments/src/bin/graspan1.rs index e93bb5381..be5e5b930 100644 --- a/experiments/src/bin/graspan1.rs +++ b/experiments/src/bin/graspan1.rs @@ -44,13 +44,13 @@ fn main() { let labels = SemigroupVariable::new(inner, Product::new(Default::default(), 1)); let next = - labels.join_core(&edges, |_b, a, c| Some((*c, *a))) - .concat(&nodes) + labels.join_core(edges, |_b, a, c| Some((*c, *a))) + .concat(nodes) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() // .distinct_total_core::(); .threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None }); - labels.set(&next); + labels.set(next); next.leave() }); diff --git a/experiments/src/bin/graspan2.rs b/experiments/src/bin/graspan2.rs index dd5563d7b..7cd9d0dd7 100644 --- a/experiments/src/bin/graspan2.rs +++ b/experiments/src/bin/graspan2.rs @@ -45,7 +45,7 @@ fn unoptimized() { let nodes = assignment .flat_map(|(a,b)| vec![a,b]) - .concat(&dereference.flat_map(|(a,b)| vec![a,b])); + .concat(dereference.flat_map(|(a,b)| vec![a,b])); let dereference = dereference.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); @@ -65,11 +65,11 @@ fn unoptimized() { // VA(a,b) <- VF(x,a),VF(x,b) // VA(a,b) <- VF(x,a),MA(x,y),VF(y,b) - let value_alias_next = value_flow_arranged.join_core(&value_flow_arranged, |_,&a,&b| Some((a,b))); - let value_alias_next = value_flow_arranged.join_core(&memory_alias_arranged, |_,&a,&b| Some((b,a))) + let value_alias_next = value_flow_arranged.join_core(value_flow_arranged, |_,&a,&b| Some((a,b))); + let value_alias_next = value_flow_arranged.join_core(memory_alias_arranged, |_,&a,&b| Some((b,a))) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&value_flow_arranged, |_,&a,&b| Some((a,b))) - .concat(&value_alias_next); + .join_core(value_flow_arranged, |_,&a,&b| Some((a,b))) + .concat(value_alias_next); // VF(a,a) <- // VF(a,b) <- A(a,x),VF(x,b) @@ -78,11 +78,11 @@ fn unoptimized() { assignment .map(|(a,b)| (b,a)) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&memory_alias_arranged, |_,&a,&b| Some((b,a))) - .concat(&assignment.map(|(a,b)| (b,a))) + .join_core(memory_alias_arranged, |_,&a,&b| Some((b,a))) + .concat(assignment.map(|(a,b)| (b,a))) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&value_flow_arranged, |_,&a,&b| Some((a,b))) - .concat(&nodes.map(|n| (n,n))); + .join_core(value_flow_arranged, |_,&a,&b| Some((a,b))) + .concat(nodes.map(|n| (n,n))); let value_flow_next = value_flow_next @@ -94,9 +94,9 @@ fn unoptimized() { // MA(a,b) <- D(x,a),VA(x,y),D(y,b) let memory_alias_next: VecCollection<_,_,Present> = value_alias_next - .join_core(&dereference, |_x,&y,&a| Some((y,a))) + .join_core(dereference, |_x,&y,&a| Some((y,a))) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&dereference, |_y,&a,&b| Some((a,b))); + .join_core(dereference, |_y,&a,&b| Some((a,b))); let memory_alias_next: VecCollection<_,_,Present> = memory_alias_next @@ -105,8 +105,8 @@ fn unoptimized() { .threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None }) ; - value_flow.set(&value_flow_next); - memory_alias.set(&memory_alias_next); + value_flow.set(value_flow_next); + memory_alias.set(memory_alias_next); (value_flow_next.leave(), memory_alias_next.leave(), value_alias_next.leave()) }); @@ -170,7 +170,7 @@ fn optimized() { let nodes = assignment .flat_map(|(a,b)| vec![a,b]) - .concat(&dereference.flat_map(|(a,b)| vec![a,b])); + .concat(dereference.flat_map(|(a,b)| vec![a,b])); let dereference = dereference.arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); @@ -195,11 +195,11 @@ fn optimized() { assignment .map(|(a,b)| (b,a)) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&memory_alias_arranged, |_,&a,&b| Some((b,a))) - .concat(&assignment.map(|(a,b)| (b,a))) + .join_core(memory_alias_arranged, |_,&a,&b| Some((b,a))) + .concat(assignment.map(|(a,b)| (b,a))) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&value_flow_arranged, |_,&a,&b| Some((a,b))) - .concat(&nodes.map(|n| (n,n))) + .join_core(value_flow_arranged, |_,&a,&b| Some((a,b))) + .concat(nodes.map(|n| (n,n))) .arrange::, KeyBuilder<_,_,_>, KeySpine<_,_,_>>() // .distinct_total_core::() .threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None }) @@ -210,28 +210,28 @@ fn optimized() { value_flow .map(|(a,b)| (b,a)) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&dereference, |_x,&a,&b| Some((a,b))) + .join_core(dereference, |_x,&a,&b| Some((a,b))) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>(); // MA(a,b) <- VFD(x,a),VFD(y,b) // MA(a,b) <- VFD(x,a),MA(x,y),VFD(y,b) let memory_alias_next = value_flow_deref - .join_core(&value_flow_deref, |_y,&a,&b| Some((a,b))); + .join_core(value_flow_deref, |_y,&a,&b| Some((a,b))); let memory_alias_next = memory_alias_arranged - .join_core(&value_flow_deref, |_x,&y,&a| Some((y,a))) + .join_core(value_flow_deref, |_x,&y,&a| Some((y,a))) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() - .join_core(&value_flow_deref, |_y,&a,&b| Some((a,b))) - .concat(&memory_alias_next) + .join_core(value_flow_deref, |_y,&a,&b| Some((a,b))) + .concat(memory_alias_next) .arrange::, KeyBuilder<_,_,_>, KeySpine<_,_,_>>() // .distinct_total_core::() .threshold_semigroup(|_,_,x| if x.is_none() { Some(Present) } else { None }) ; - value_flow.set(&value_flow_next); - memory_alias.set(&memory_alias_next); + value_flow.set(value_flow_next); + memory_alias.set(memory_alias_next); (value_flow_next.leave(), memory_alias_next.leave()) }); diff --git a/experiments/src/bin/multitemporal.rs b/experiments/src/bin/multitemporal.rs index e61ef4c5b..b105723a8 100644 --- a/experiments/src/bin/multitemporal.rs +++ b/experiments/src/bin/multitemporal.rs @@ -2,7 +2,7 @@ use rand::{Rng, SeedableRng, StdRng}; use timely::dataflow::ProbeHandle; -use timely::dataflow::operators::unordered_input::UnorderedInput; +use timely::dataflow::operators::vec::unordered_input::UnorderedInput; use differential_dataflow::AsCollection; use differential_dataflow::operators::*; @@ -39,9 +39,9 @@ fn main() { let roots = roots.enter(&inner.scope()); edges - .semijoin(&inner) + .semijoin(inner) .map(|(_s,d)| d) - .concat(&roots) + .concat(roots) .distinct() }) .consolidate() @@ -223,10 +223,10 @@ mod pair { // This extends the `PartialOrder` implementation with additional structure. use differential_dataflow::lattice::Lattice; impl Lattice for Pair { - fn join(&self, other: &Self) -> Self { + fn join(self, other: &Self) -> Self { Pair { - first: self.first.join(&other.first), - second: self.second.join(&other.second), + first: self.first.join(other.first), + second: self.second.join(other.second), } } fn meet(&self, other: &Self) -> Self { @@ -312,12 +312,12 @@ mod vector { // This extends the `PartialOrder` implementation with additional structure. use differential_dataflow::lattice::Lattice; impl Lattice for Vector { - fn join(&self, other: &Self) -> Self { + fn join(self, other: &Self) -> Self { let min_len = ::std::cmp::min(self.vector.len(), other.vector.len()); let max_len = ::std::cmp::max(self.vector.len(), other.vector.len()); let mut vector = Vec::with_capacity(max_len); for index in 0 .. min_len { - vector.push(self.vector[index].join(&other.vector[index])); + vector.push(self.vector[index].join(other.vector[index])); } for time in &self.vector[min_len..] { vector.push(time.clone()); diff --git a/experiments/src/bin/ysb.rs b/experiments/src/bin/ysb.rs index 39ccb1779..0ef676832 100644 --- a/experiments/src/bin/ysb.rs +++ b/experiments/src/bin/ysb.rs @@ -74,7 +74,7 @@ fn main() { let probe = links - .semijoin(&views) + .semijoin(views) .map(|(_ad, campaign)| campaign) .consolidate() .inspect(move |x| if inspect { println!("{:?}:\t{:?}", x.0, x.2); }) diff --git a/interactive/src/plan/join.rs b/interactive/src/plan/join.rs index 5c6596249..956fab419 100644 --- a/interactive/src/plan/join.rs +++ b/interactive/src/plan/join.rs @@ -95,7 +95,7 @@ impl Render for Join { let arrange2 = trace2.import(scope); arrange1 - .join_core(&arrange2, |keys, vals1, vals2| { + .join_core(arrange2, |keys, vals1, vals2| { Some( keys.iter().cloned() .chain(vals1.iter().cloned()) diff --git a/mdbook/src/chapter_0/chapter_0_1.md b/mdbook/src/chapter_0/chapter_0_1.md index eabafe36b..af744d066 100644 --- a/mdbook/src/chapter_0/chapter_0_1.md +++ b/mdbook/src/chapter_0/chapter_0_1.md @@ -28,8 +28,9 @@ fn main() { // if (m2, m1) and (m1, p), then output (m1, (m2, p)) manages + .clone() .map(|(m2, m1)| (m1, m2)) - .join(&manages) + .join(manages) .inspect(|x| println!("{:?}", x)); }); diff --git a/mdbook/src/chapter_0/chapter_0_3.md b/mdbook/src/chapter_0/chapter_0_3.md index 0f16ab4f1..fdb8743eb 100644 --- a/mdbook/src/chapter_0/chapter_0_3.md +++ b/mdbook/src/chapter_0/chapter_0_3.md @@ -77,8 +77,9 @@ Instead of loading all of our changes and only waiting for the result, we can lo // if (m2, m1) and (m1, p), then output (m1, (m2, p)) manages + .clone() .map(|(m2, m1)| (m1, m2)) - .join(&manages) + .join(manages) // .inspect(|x| println!("{:?}", x)) .probe() }); diff --git a/mdbook/src/chapter_1/chapter_1_2.md b/mdbook/src/chapter_1/chapter_1_2.md index 6dcea40f4..06ed31410 100644 --- a/mdbook/src/chapter_1/chapter_1_2.md +++ b/mdbook/src/chapter_1/chapter_1_2.md @@ -43,7 +43,7 @@ The `concat` operator merges two collections together, essentially adding the oc ```rust,ignore collection1 - .concat(&collection2) + .concat(collection2) .count() ``` @@ -58,7 +58,7 @@ Join has a lot of uses, but one common example is to "look up" data. If we have ```rust,ignore let deliver_to = ordered_by - .join(&person_address) + .join(person_address) .map(|(person, package, address)| (package, address)); ``` @@ -68,7 +68,7 @@ Alternately, we can use the same relation to find people living at a given addre let can_sign_for = deliver_to .map(|(package, address)| (address, package)) - .join(&person_address.map(|(p,a)| (a,p))) + .join(person_address.map(|(p,a)| (a,p))) .map(|(address, package, person)| (package, person)); ``` diff --git a/mdbook/src/chapter_1/chapter_1_3.md b/mdbook/src/chapter_1/chapter_1_3.md index 26eed5ae6..60a0ca9a9 100644 --- a/mdbook/src/chapter_1/chapter_1_3.md +++ b/mdbook/src/chapter_1/chapter_1_3.md @@ -16,7 +16,7 @@ Let's describe an iterative computation that starts from manager-employee relati // if x manages y, and y manages z, then x manages z (transitively). manages .map(|(x, y)| (y, x)) - .join(&manages) + .join(manages) .map(|(y, x, z)| (x, z)) }); ``` @@ -70,7 +70,7 @@ To create an iterative computation, we now need to define some variables that ca let step = variable .map(|(x, y)| (y, x)) - .join(&variable) + .join(variable) .map(|(y, x, z)| (x, z)); variable.set(step); @@ -95,7 +95,7 @@ Finally, we probably want to return the final value of the variable, what it con let step = variable .map(|(x, y)| (y, x)) - .join(&variable) + .join(variable) .map(|(y, x, z)| (x, z)); variable diff --git a/mdbook/src/chapter_2/chapter_2_1.md b/mdbook/src/chapter_2/chapter_2_1.md index 3d40b7694..ecc749447 100644 --- a/mdbook/src/chapter_2/chapter_2_1.md +++ b/mdbook/src/chapter_2/chapter_2_1.md @@ -10,12 +10,13 @@ As an example, our example program used `map` to reverse the pairs of identifier # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages + .clone() .map(|(m2, m1)| (m1, m2)) - .join(&manages) + .join(manages) .inspect(|x| println!("{:?}", x)); # } ``` @@ -28,7 +29,7 @@ If instead we had just written # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages diff --git a/mdbook/src/chapter_2/chapter_2_2.md b/mdbook/src/chapter_2/chapter_2_2.md index ee99cfdb9..b30d779ef 100644 --- a/mdbook/src/chapter_2/chapter_2_2.md +++ b/mdbook/src/chapter_2/chapter_2_2.md @@ -10,7 +10,7 @@ As an example, we might select out those management relation where the manager h # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages diff --git a/mdbook/src/chapter_2/chapter_2_3.md b/mdbook/src/chapter_2/chapter_2_3.md index 0f58a16c1..b61a02a57 100644 --- a/mdbook/src/chapter_2/chapter_2_3.md +++ b/mdbook/src/chapter_2/chapter_2_3.md @@ -10,12 +10,13 @@ For example, we might form the symmetric "management relation" by concatenating # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages + .clone() .map(|(m2, m1)| (m1, m2)) - .concat(&manages); + .concat(manages); # } ``` diff --git a/mdbook/src/chapter_2/chapter_2_4.md b/mdbook/src/chapter_2/chapter_2_4.md index 5e3f6a2ca..2c2ad4f4b 100644 --- a/mdbook/src/chapter_2/chapter_2_4.md +++ b/mdbook/src/chapter_2/chapter_2_4.md @@ -12,12 +12,13 @@ As an example, if we were to inspect # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages + .clone() .map(|(m2, m1)| (m1, m2)) - .concat(&manages) + .concat(manages) .inspect(|x| println!("{:?}", x)); # } ``` @@ -37,12 +38,13 @@ However, by introducing `consolidate` # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages + .clone() .map(|(m2, m1)| (m1, m2)) - .concat(&manages) + .concat(manages) .consolidate() .inspect(|x| println!("{:?}", x)); # } diff --git a/mdbook/src/chapter_2/chapter_2_5.md b/mdbook/src/chapter_2/chapter_2_5.md index 965d5d380..8de590eb2 100644 --- a/mdbook/src/chapter_2/chapter_2_5.md +++ b/mdbook/src/chapter_2/chapter_2_5.md @@ -10,12 +10,13 @@ Our example from earlier uses a join to match up pairs `(m2, m1)` and `(m1, p)` # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages + .clone() .map(|(m2, m1)| (m1, m2)) - .join(&manages) + .join(manages) .inspect(|x| println!("{:?}", x)); # } ``` diff --git a/mdbook/src/chapter_2/chapter_2_6.md b/mdbook/src/chapter_2/chapter_2_6.md index 54f103eed..74b5020f4 100644 --- a/mdbook/src/chapter_2/chapter_2_6.md +++ b/mdbook/src/chapter_2/chapter_2_6.md @@ -10,7 +10,7 @@ For example, to produce for each manager their managee with the lowest identifie # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages diff --git a/mdbook/src/chapter_2/chapter_2_7.md b/mdbook/src/chapter_2/chapter_2_7.md index 8f830db79..cb5ac846a 100644 --- a/mdbook/src/chapter_2/chapter_2_7.md +++ b/mdbook/src/chapter_2/chapter_2_7.md @@ -11,16 +11,17 @@ As an example, we can take our `manages` relation and determine for all employee # use differential_dataflow::VecCollection; # use differential_dataflow::operators::Iterate; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages // transitive contains (manager, person) for many hops. .iterate(|transitive| { transitive + .clone() .map(|(mk, m1)| (m1, mk)) - .join(&transitive) + .join(transitive.clone()) .map(|(m1, (mk, p))| (mk, p)) - .concat(&transitive) + .concat(transitive) .distinct() }); # } @@ -47,19 +48,20 @@ In the example above, we could rewrite # use differential_dataflow::VecCollection; # use differential_dataflow::operators::{Iterate, iterate::VecVariable}; # use differential_dataflow::lattice::Lattice; -# fn example(manages: &VecCollection) +# fn example(manages: VecCollection) # where G::Timestamp: Lattice # { manages // transitive contains (manager, person) for many hops. + .clone() .iterate(|transitive| { let manages = manages.enter(&transitive.scope()); transitive .map(|(mk, m1)| (m1, mk)) - .join(&manages) + .join(manages.clone()) .map(|(m1, (mk, p))| (mk, p)) - .concat(&manages) + .concat(manages) .distinct() }); # } @@ -93,13 +95,13 @@ As an example, the implementation of the `iterate` operator looks something like # { # (*variable).clone() # } -# fn example<'a, G: Scope>(collection: &VecCollection) //, logic: impl Fn(&VecVariable, (u64, u64), isize>) -> VecCollection, (u64, u64)>) +# fn example<'a, G: Scope>(collection: VecCollection) //, logic: impl Fn(&VecVariable, (u64, u64), isize>) -> VecCollection, (u64, u64)>) # where G::Timestamp: Lattice # { collection.scope().scoped("inner", |subgraph| { let variable = VecVariable::new_from(collection.enter(subgraph), 1); let result = logic(&variable); - variable.set(&result); + variable.set(result.clone()); result.leave() }); # } diff --git a/mdbook/src/chapter_3/chapter_3_2.md b/mdbook/src/chapter_3/chapter_3_2.md index e61f66664..112ab1aef 100644 --- a/mdbook/src/chapter_3/chapter_3_2.md +++ b/mdbook/src/chapter_3/chapter_3_2.md @@ -15,8 +15,9 @@ For example, recall our example of interacting with our management computation, // if (m2, m1) and (m1, p), then output (m1, (m2, p)) manages + .clone() .map(|(m2, m1)| (m1, m2)) - .join(&manages) + .join(manages) .probe() }); ``` diff --git a/mdbook/src/chapter_4/chapter_4_1.md b/mdbook/src/chapter_4/chapter_4_1.md index d6726b4b1..76e819fbe 100644 --- a/mdbook/src/chapter_4/chapter_4_1.md +++ b/mdbook/src/chapter_4/chapter_4_1.md @@ -17,9 +17,9 @@ Let's write this computation starting from a collection `edges`, using different .iterate(|inner| { let labels = labels.enter(inner.scope()); let edges = edges.enter(inner.scope()); - inner.join(&edges) + inner.join(edges) .map(|(_src,(lbl,dst))| (dst,lbl)) - .concat(&labels) + .concat(labels) .reduce(|_dst, lbls, out| { let min_lbl = lbls.iter() diff --git a/mdbook/src/chapter_5/chapter_5_1.md b/mdbook/src/chapter_5/chapter_5_1.md index 52cbf0d41..a7ca050df 100644 --- a/mdbook/src/chapter_5/chapter_5_1.md +++ b/mdbook/src/chapter_5/chapter_5_1.md @@ -22,8 +22,8 @@ fn main() { let query = query.to_collection(scope); // Hop from x to y, then from y to z. - query.join_map(&knows, |x,q,y| (*y,*q)) - .join_map(&knows, |y,q,z| (*q,*z)) + query.join_map(knows.clone(), |x,q,y| (*y,*q)) + .join_map(knows.clone(), |y,q,z| (*q,*z)) .inspect(|result| println!("result {:?}", result)); }); @@ -60,8 +60,8 @@ fn main() { let knows = knows.arrange_by_key(); // Same logic as before, with a new method name. - query.join_core(&knows, |x,q,y| Some((*y,*q))) - .join_core(&knows, |y,q,z| Some((*q,*z))) + query.join_core(knows.clone(), |x,q,y| Some((*y,*q))) + .join_core(knows.clone(), |y,q,z| Some((*q,*z))) .inspect(|result| println!("result {:?}", result)); }); diff --git a/mdbook/src/chapter_5/chapter_5_2.md b/mdbook/src/chapter_5/chapter_5_2.md index 6dbcc0f46..118607b4a 100644 --- a/mdbook/src/chapter_5/chapter_5_2.md +++ b/mdbook/src/chapter_5/chapter_5_2.md @@ -27,9 +27,9 @@ fn main() { let knows = knows.arrange_by_key(); // Same logic as before, with a new method name. - query.join_core(&knows, |x, q, y| Some((*y, (*x, *q)))) - .join_core(&knows, |y, (x, q), z| Some((*q, (*x, *y, *z)))) - .inspect(|result| println!("result {:?}", result)); + query.join_core(knows.clone(), |x, q, y| Some((*y, (*x, *q)))) + .join_core(knows.clone(), |y, (x, q), z| Some((*q, (*x, *y, *z)))) + .inspect(|result| println!("result {:?}", result)); }); # // to help with type inference ... @@ -63,21 +63,21 @@ fn main() { let query = query.to_collection(scope); // Arrange the data first! (by key and self). - let knows_by_key = knows.arrange_by_key(); - let knows_by_self = knows.arrange_by_self(); + let knows_by_key = knows.clone().arrange_by_key(); + let knows_by_self = knows.clone().arrange_by_self(); // The same outputs as in the previous example. let candidates = - query.join_core(&knows_by_key, |x,q,y| Some((*y,(*x,*q)))) - .join_core(&knows_by_key, |y,(x,q),z| Some((*q,(*x,*y,*z)))); + query.join_core(knows_by_key.clone(), |x,q,y| Some((*y,(*x,*q)))) + .join_core(knows_by_key.clone(), |y,(x,q),z| Some((*q,(*x,*y,*z)))); // Repeatedly put pairs of nodes as keys, and semijoin with knows. candidates .map(|(q,(x,y,z))| ((x,z),(q,y))) - .join_core(&knows_by_self, |&(x,z),&(q,y),&()| Some(((y,z),(q,x)))) - .join_core(&knows_by_self, |&(y,z),&(q,x),&()| Some(((z,x),(q,y)))) - .join_core(&knows_by_self, |&(z,x),&(q,y),&()| Some(((y,x),(q,z)))) - .join_core(&knows_by_self, |&(y,x),&(q,z),&()| Some((q,(x,y,z)))) + .join_core(knows_by_self.clone(), |&(x,z),&(q,y),&()| Some(((y,z),(q,x)))) + .join_core(knows_by_self.clone(), |&(y,z),&(q,x),&()| Some(((z,x),(q,y)))) + .join_core(knows_by_self.clone(), |&(z,x),&(q,y),&()| Some(((y,x),(q,z)))) + .join_core(knows_by_self.clone(), |&(y,x),&(q,z),&()| Some((q,(x,y,z)))) .inspect(|result| println!("result {:?}", result)); }); @@ -120,8 +120,8 @@ fn main() { let knows = knows.as_collection(|k,v| (*k,*v)); // Same logic as before, with a new method name. - query.join_map(&knows, |x,q,y| (*y,(*x,*q))) - .join_map(&knows, |y,(x,q),z| (*q,(*x,*y,*z))) + query.join_map(knows.clone(), |x,q,y| (*y,(*x,*q))) + .join_map(knows.clone(), |y,(x,q),z| (*q,(*x,*y,*z))) .inspect(|result| println!("result {:?}", result)); }); diff --git a/mdbook/src/chapter_5/chapter_5_3.md b/mdbook/src/chapter_5/chapter_5_3.md index 1ba28aff1..0ebe0345c 100644 --- a/mdbook/src/chapter_5/chapter_5_3.md +++ b/mdbook/src/chapter_5/chapter_5_3.md @@ -53,7 +53,7 @@ For example, let's imagine we want to construct many new dataflows each of which // Import arrangement, extract keys from `query`. trace .import(scope) - .semijoin(&query) + .semijoin(query) .consolidate() .inspect(move |x| println!("{:?}\t{:?}", timer.elapsed(), x)) .probe_with(&mut probe); diff --git a/mdbook/src/chapter_5/chapter_5_4.md b/mdbook/src/chapter_5/chapter_5_4.md index 549312cad..25a3e097f 100644 --- a/mdbook/src/chapter_5/chapter_5_4.md +++ b/mdbook/src/chapter_5/chapter_5_4.md @@ -41,13 +41,13 @@ fn main() { let knows = knows.arrange_by_key(); // Reachability queries. - query.iterate(|reach| { + query.clone().iterate(|reach| { let knows = knows.enter(&reach.scope()); let query = query.enter(&reach.scope()); - knows.join_core(&reach.arrange_by_key(), |x,y,q| [(*y,*q)]) - .concat(&query) + knows.join_core(reach.arrange_by_key(), |x,y,q| [(*y,*q)]) + .concat(query) .distinct() }); diff --git a/mdbook/src/chapter_a/chapter_a_3.md b/mdbook/src/chapter_a/chapter_a_3.md index b87d2e55f..864726a3d 100644 --- a/mdbook/src/chapter_a/chapter_a_3.md +++ b/mdbook/src/chapter_a/chapter_a_3.md @@ -11,8 +11,9 @@ Instead of loading all of our changes and only waiting for the result, we can lo // if (m2, m1) and (m1, p), then output (m1, (m2, p)) manages + .clone() .map(|(m2, m1)| (m1, m2)) - .join(&manages) + .join(manages) // .inspect(|x| println!("{:?}", x)) .probe() }); diff --git a/server/dataflows/neighborhood/src/lib.rs b/server/dataflows/neighborhood/src/lib.rs index 2796a11ec..85545ed47 100644 --- a/server/dataflows/neighborhood/src/lib.rs +++ b/server/dataflows/neighborhood/src/lib.rs @@ -23,9 +23,9 @@ pub fn build((dataflow, handles, probe, _timer, args): Environment) -> Result<() query .map(|x| (x, x)) - .join_core(&edges, |_n, &q, &d| Some((d, q))) // one hop - .join_core(&edges, |_n, &q, &d| Some((d, q))) // two hops - .join_core(&edges, |_n, &q, &d| Some((d, q))) // three hops + .join_core(edges, |_n, &q, &d| Some((d, q))) // one hop + .join_core(edges, |_n, &q, &d| Some((d, q))) // two hops + .join_core(edges, |_n, &q, &d| Some((d, q))) // three hops .map(|x| x.1) .consolidate() .inspect(move |x| println!("{:?}:\t{:?}", timer.elapsed(), x)) diff --git a/server/dataflows/reachability/src/lib.rs b/server/dataflows/reachability/src/lib.rs index 2c6bfb10c..3c0f2f15a 100644 --- a/server/dataflows/reachability/src/lib.rs +++ b/server/dataflows/reachability/src/lib.rs @@ -23,8 +23,8 @@ pub fn build((dataflow, handles, probe, _timer, args): Environment) -> Result<() let edges = edges.enter(&dists.scope()); let roots = roots.enter(&dists.scope()); dists.arrange_by_self() - .join_core(&edges, |_src, _, &dst| Some(dst)) - .concat(&roots) + .join_core(edges, |_src, _, &dst| Some(dst)) + .concat(roots) .distinct() }) .probe_with(probe); diff --git a/tpchlike/src/bin/arrange.rs b/tpchlike/src/bin/arrange.rs index 503fb5fdb..a96996bdc 100644 --- a/tpchlike/src/bin/arrange.rs +++ b/tpchlike/src/bin/arrange.rs @@ -43,7 +43,7 @@ fn main() { let (supp_in, supp) = scope.new_input(); if window { - line = line.map(|(x,y,z): (LineItem, usize, isize)| (x,y+1,-z)).concat(&line); + line = line.map(|(x,y,z): (LineItem, usize, isize)| (x,y+1,-z)).concat(line); } let collections = Collections::new( diff --git a/tpchlike/src/bin/just-arrange.rs b/tpchlike/src/bin/just-arrange.rs index ca7bf52fd..60d82de74 100644 --- a/tpchlike/src/bin/just-arrange.rs +++ b/tpchlike/src/bin/just-arrange.rs @@ -43,7 +43,7 @@ fn main() { let (supp_in, supp) = scope.new_input(); if window { - line = line.map(|(x,y,z): (LineItem, usize, isize)| (x,y+1,-z)).concat(&line); + line = line.map(|(x,y,z): (LineItem, usize, isize)| (x,y+1,-z)).concat(line); } let collections = Collections::new( diff --git a/tpchlike/src/lib.rs b/tpchlike/src/lib.rs index c03599b1d..053efd9d0 100644 --- a/tpchlike/src/lib.rs +++ b/tpchlike/src/lib.rs @@ -101,14 +101,14 @@ impl Collections { } } - pub fn customers(&mut self) -> &VecCollection { self.used[0] = true; &self.customers } - pub fn lineitems(&mut self) -> &VecCollection, isize> { self.used[1] = true; &self.lineitems } - pub fn nations(&mut self) -> &VecCollection { self.used[2] = true; &self.nations } - pub fn orders(&mut self) -> &VecCollection { self.used[3] = true; &self.orders } - pub fn parts(&mut self) -> &VecCollection { self.used[4] = true; &self.parts } - pub fn partsupps(&mut self) -> &VecCollection { self.used[5] = true; &self.partsupps } - pub fn regions(&mut self) -> &VecCollection { self.used[6] = true; &self.regions } - pub fn suppliers(&mut self) -> &VecCollection { self.used[7] = true; &self.suppliers } + pub fn customers(&mut self) -> VecCollection { self.used[0] = true; &self.customers } + pub fn lineitems(&mut self) -> VecCollection, isize> { self.used[1] = true; &self.lineitems } + pub fn nations(&mut self) -> VecCollection { self.used[2] = true; &self.nations } + pub fn orders(&mut self) -> VecCollection { self.used[3] = true; &self.orders } + pub fn parts(&mut self) -> VecCollection { self.used[4] = true; &self.parts } + pub fn partsupps(&mut self) -> VecCollection { self.used[5] = true; &self.partsupps } + pub fn regions(&mut self) -> VecCollection { self.used[6] = true; &self.regions } + pub fn suppliers(&mut self) -> VecCollection { self.used[7] = true; &self.suppliers } pub fn used(&self) -> [bool; 8] { self.used } } diff --git a/tpchlike/src/queries/query02.rs b/tpchlike/src/queries/query02.rs index 7fd818962..13fb29550 100644 --- a/tpchlike/src/queries/query02.rs +++ b/tpchlike/src/queries/query02.rs @@ -80,14 +80,14 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .nations() .map(|x| (x.region_key, (x.nation_key, x.name))) - .semijoin(®ions) + .semijoin(regions) .map(|(_region_key, (nation_key, name))| (nation_key, name)); let suppliers = collections .suppliers() .map(|x| (x.nation_key, (x.acctbal, x.name, x.address, x.phone, x.comment, x.supp_key))) - .semijoin(&nations.map(|x| x.0)) + .semijoin(nations.map(|x| x.0)) .map(|(nat, (acc, nam, add, phn, com, key))| (key, (nat, acc, nam, add, phn, com))); let parts = @@ -99,20 +99,20 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .partsupps() .map(|x| (x.supp_key, (x.part_key, x.supplycost))) - .semijoin(&suppliers.map(|x| x.0)) + .semijoin(suppliers.map(|x| x.0)) .map(|(supp, (part, supply_cost))| (part, (supply_cost, supp))) - .semijoin(&parts.map(|x| x.0)) + .semijoin(parts.map(|x| x.0)) .reduce(|_x, s, t| { let minimum = (s[0].0).0; t.extend(s.iter().take_while(|x| (x.0).0 == minimum).map(|&(&x,w)| (x,w))); }); partsupps - .join(&parts) + .join(parts) .map(|(part_key, ((cost, supp), mfgr))| (supp, (cost, part_key, mfgr))) - .join(&suppliers) + .join(suppliers) .map(|(_supp, ((cost, part, mfgr), (nat, acc, nam, add, phn, com)))| (nat, (cost, part, mfgr, acc, nam, add, phn, com))) - .join(&nations) + .join(nations) .probe_with(probe); } @@ -138,9 +138,9 @@ where .inner .map(move |(d,t,r)| (d, ::std::cmp::max(t,round),r)) .as_collection() - .join_core(&arrangements.supplier, |&sk, &(pk,sc), s| Some((s.nation_key, (pk,sc,sk)))) - .join_core(&arrangements.nation, |_nk, &(pk,sc,sk), n| Some((n.region_key, (pk,sc,sk,n.name)))) - .join_core(&arrangements.region, |_rk, &(pk,sc,sk,nm), r| { + .join_core(arrangements.supplier, |&sk, &(pk,sc), s| Some((s.nation_key, (pk,sc,sk)))) + .join_core(arrangements.nation, |_nk, &(pk,sc,sk), n| Some((n.region_key, (pk,sc,sk,n.name)))) + .join_core(arrangements.region, |_rk, &(pk,sc,sk,nm), r| { if starts_with(&r.name[..], b"EUROPE") { Some((pk,(sc,sk,nm))) } else { None } }); @@ -152,7 +152,7 @@ where }); cheapest_suppliers - .join_core(&arrangements.part, |&pk,&(sc,sk,nm),p| { + .join_core(arrangements.part, |&pk,&(sc,sk,nm),p| { if substring(p.typ.as_str().as_bytes(), b"BRASS") && p.size == 15 { Some((sk, (sc,pk,nm,p.mfgr))) } @@ -160,7 +160,7 @@ where None } }) - .join_core(&arrangements.supplier, |_sk,&(sc,pk,nm,pm),s| { + .join_core(arrangements.supplier, |_sk,&(sc,pk,nm,pm),s| { Some((sc,pk,nm,pm,s.acctbal,s.name,s.address,s.phone,s.comment)) }) .probe_with(probe); diff --git a/tpchlike/src/queries/query03.rs b/tpchlike/src/queries/query03.rs index 0342ed930..8941e735a 100644 --- a/tpchlike/src/queries/query03.rs +++ b/tpchlike/src/queries/query03.rs @@ -67,9 +67,9 @@ where G::Timestamp: Lattice+TotalOrder+Ord { .map(|o| (o.cust_key, (o.order_key, o.order_date, o.ship_priority))); orders - .semijoin(&customers) + .semijoin(customers) .map(|(_, (order_key, order_date, ship_priority))| (order_key, (order_date, ship_priority))) - .semijoin(&lineitems) + .semijoin(lineitems) .count_total() // .inspect(|x| println!("{:?}", x)) .probe_with(probe); @@ -94,7 +94,7 @@ where } else { None } ) - .join_core(&arrangements.order, |_k, &(), o| { + .join_core(arrangements.order, |_k, &(), o| { if o.order_date < create_date(1995, 3, 15) { Some((o.cust_key, (o.order_key, o.order_date, o.ship_priority))) } @@ -102,7 +102,7 @@ where None } }) - .join_core(&arrangements.customer, |_k,o,c| { + .join_core(arrangements.customer, |_k,o,c| { if starts_with(&c.mktsegment[..], b"BUILDING") { Some(o.clone()) } diff --git a/tpchlike/src/queries/query04.rs b/tpchlike/src/queries/query04.rs index 7b4d119d5..49c2bd1fc 100644 --- a/tpchlike/src/queries/query04.rs +++ b/tpchlike/src/queries/query04.rs @@ -53,7 +53,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { } else { None } ) - .semijoin(&lineitems) + .semijoin(lineitems) .map(|(_k,v)| v) .count_total() // .inspect(|x| println!("{:?}", x)) @@ -75,7 +75,7 @@ where .lineitem(scope) .flat_map(|l| if l.commit_date < l.receipt_date { Some((l.order_key, ())) } else { None }) .distinct_total() - .join_core(&arrangements.order, |_k,&(),o| { + .join_core(arrangements.order, |_k,&(),o| { if o.order_date >= ::types::create_date(1993, 7, 1) && o.order_date < ::types::create_date(1993, 10, 1) { Some(o.order_priority) } diff --git a/tpchlike/src/queries/query05.rs b/tpchlike/src/queries/query05.rs index a9bf09434..b66eb99fe 100644 --- a/tpchlike/src/queries/query05.rs +++ b/tpchlike/src/queries/query05.rs @@ -57,21 +57,21 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .nations() .map(|x| (x.region_key, x.nation_key)) - .semijoin(®ions) + .semijoin(regions) .map(|(_region_key, nation_key)| nation_key); let suppliers = collections .suppliers() .map(|x| (x.nation_key, x.supp_key)) - .semijoin(&nations) + .semijoin(nations) .map(|(_nat, supp)| (supp, _nat)); let customers = collections .customers() .map(|c| (c.nation_key, c.cust_key)) - .semijoin(&nations) + .semijoin(nations) .map(|c| (c.1, c.0)); let orders = @@ -83,18 +83,18 @@ where G::Timestamp: Lattice+TotalOrder+Ord { } else { None } ) - .join(&customers) + .join(customers) .map(|o| o.1); let lineitems = collections .lineitems() .explode(|l| Some(((l.order_key, l.supp_key), (l.extended_price * (100 - l.discount) / 100) as isize))) - .join(&orders) + .join(orders) .map(|(_order, (supp, nat))| (supp, nat)); suppliers .map(|x| (x, ())) - .semijoin(&lineitems) + .semijoin(lineitems) .map(|((_supp, nat), ())| nat) .count_total() // .inspect(|x| println!("{:?}", x)) @@ -115,21 +115,21 @@ where experiment .lineitem(scope) .explode(|l| Some(((l.order_key, l.supp_key), (l.extended_price * (100 - l.discount) / 100) as isize))) - .join_core(&arrangements.order, |_ok, &sk, o| { + .join_core(arrangements.order, |_ok, &sk, o| { if o.order_date >= create_date(1994, 1, 1) && o.order_date < create_date(1995, 1, 1) { Some((o.cust_key, sk)) } else { None } }) - .join_core(&arrangements.customer, |_ck, &sk, c| Some((sk,c.nation_key))) - .join_core(&arrangements.supplier, |_sk, &nk, s| { + .join_core(arrangements.customer, |_ck, &sk, c| Some((sk,c.nation_key))) + .join_core(arrangements.supplier, |_sk, &nk, s| { if s.nation_key == nk { Some((nk, ())) } else { None } }) - .join_core(&arrangements.nation, |_nk, &(), n| Some((n.region_key, n.name))) - .join_core(&arrangements.region, |_rk, &name, r| { + .join_core(arrangements.nation, |_nk, &(), n| Some((n.region_key, n.name))) + .join_core(arrangements.region, |_rk, &name, r| { if starts_with(&r.name[..], b"ASIA") { Some(name) } else { None } }) .count_total() diff --git a/tpchlike/src/queries/query07.rs b/tpchlike/src/queries/query07.rs index 2e829fc2c..a92c12167 100644 --- a/tpchlike/src/queries/query07.rs +++ b/tpchlike/src/queries/query07.rs @@ -74,19 +74,19 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .customers() .map(|c| (c.nation_key, c.cust_key)) - .join_map(&nations, |_, &cust_key, &name| (cust_key, name)); + .join_map(nations, |_, &cust_key, &name| (cust_key, name)); let orders = collections .orders() .map(|o| (o.cust_key, o.order_key)) - .join_map(&customers, |_, &order_key, &name| (order_key, name)); + .join_map(customers, |_, &order_key, &name| (order_key, name)); let suppliers = collections .suppliers() .map(|s| (s.nation_key, s.supp_key)) - .join_map(&nations, |_, &supp_key, &name| (supp_key, name)); + .join_map(nations, |_, &supp_key, &name| (supp_key, name)); collections .lineitems() @@ -96,8 +96,8 @@ where G::Timestamp: Lattice+TotalOrder+Ord { } else { None } ) - .join_map(&suppliers, |_, &(order_key, ship_date), &name_s| (order_key, (ship_date, name_s))) - .join_map(&orders, |_, &(ship_date, name_s), &name_c| (name_s, name_c, ship_date >> 16)) + .join_map(suppliers, |_, &(order_key, ship_date), &name_s| (order_key, (ship_date, name_s))) + .join_map(orders, |_, &(ship_date, name_s), &name_c| (name_s, name_c, ship_date >> 16)) .filter(|x| x.0 != x.1) .count_total() // .inspect(|x| println!("{:?}", x)) @@ -123,8 +123,8 @@ where } else { None } ) - .join_core(&arrangements.supplier, |_sk,&(ok,sd),s| Some((s.nation_key,(ok,sd)))) - .join_core(&arrangements.nation, |_nk,&(ok,sd),n| { + .join_core(arrangements.supplier, |_sk,&(ok,sd),s| Some((s.nation_key,(ok,sd)))) + .join_core(arrangements.nation, |_nk,&(ok,sd),n| { if starts_with(&n.name, b"FRANCE") || starts_with(&n.name, b"GERMANY") { Some((ok,(sd,n.name))) } @@ -132,9 +132,9 @@ where None } }) - .join_core(&arrangements.order, |_ok,&(sd,n1),o| Some((o.cust_key,(sd,n1)))) - .join_core(&arrangements.customer, |_ck,&(sd,n1),c| Some((c.nation_key,(sd,n1)))) - .join_core(&arrangements.nation, |_nk,&(sd,n1),n| { + .join_core(arrangements.order, |_ok,&(sd,n1),o| Some((o.cust_key,(sd,n1)))) + .join_core(arrangements.customer, |_ck,&(sd,n1),c| Some((c.nation_key,(sd,n1)))) + .join_core(arrangements.nation, |_nk,&(sd,n1),n| { if starts_with(&n.name, b"FRANCE") || starts_with(&n.name, b"GERMANY") { Some((sd,n1,n.name)) } diff --git a/tpchlike/src/queries/query08.rs b/tpchlike/src/queries/query08.rs index 0d2a4c40f..c7e34843b 100644 --- a/tpchlike/src/queries/query08.rs +++ b/tpchlike/src/queries/query08.rs @@ -62,8 +62,8 @@ pub fn query(collections: &mut Collections, probe: &mut ProbeHandle where G::Timestamp: Lattice+TotalOrder+Ord { let regions = collections.regions().filter(|r| starts_with(&r.name, b"AMERICA")).map(|r| r.region_key); - let nations1 = collections.nations().map(|n| (n.region_key, n.nation_key)).semijoin(®ions).map(|x| x.1); - let customers = collections.customers().map(|c| (c.nation_key, c.cust_key)).semijoin(&nations1).map(|x| x.1); + let nations1 = collections.nations().map(|n| (n.region_key, n.nation_key)).semijoin(regions).map(|x| x.1); + let customers = collections.customers().map(|c| (c.nation_key, c.cust_key)).semijoin(nations1).map(|x| x.1); let orders = collections .orders() @@ -73,7 +73,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { } else { None } ) - .semijoin(&customers) + .semijoin(customers) .map(|x| x.1); let nations2 = collections.nations.map(|n| (n.nation_key, starts_with(&n.name, b"BRAZIL"))); @@ -81,7 +81,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .suppliers() .map(|s| (s.nation_key, s.supp_key)) - .join(&nations2) + .join(nations2) .map(|(_, (supp_key, is_name))| (supp_key, is_name)); let parts = collections.parts().filter(|p| p.typ.as_str() == "ECONOMY ANODIZED STEEL").map(|p| p.part_key); @@ -89,11 +89,11 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .lineitems() .explode(|l| Some(((l.part_key, (l.supp_key, l.order_key)), ((l.extended_price * (100 - l.discount)) as isize / 100)))) - .semijoin(&parts) + .semijoin(parts) .map(|(_part_key, (supp_key, order_key))| (order_key, supp_key)) - .join(&orders) + .join(orders) .map(|(_order_key, (supp_key, order_date))| (supp_key, order_date)) - .join(&suppliers) + .join(suppliers) .explode(|(_, (order_date, is_name))| Some((order_date, DiffPair::new(if is_name { 1 } else { 0 }, 1)))) .count_total() // .inspect(|x| println!("{:?}", x)) @@ -114,23 +114,23 @@ where experiment .lineitem(scope) .explode(|l| Some(((l.part_key, (l.order_key, l.supp_key)), ((l.extended_price * (100 - l.discount)) as isize / 100)))) - .join_core(&arrangements.part, |_pk,&(ok,sk),p| { + .join_core(arrangements.part, |_pk,&(ok,sk),p| { if p.typ.as_str() == "ECONOMY ANODIZED STEEL" { Some((ok,sk)) } else { None } }) - .join_core(&arrangements.order, |_ok,&sk,o| { + .join_core(arrangements.order, |_ok,&sk,o| { if create_date(1995,1,1) <= o.order_date && o.order_date <= create_date(1996, 12, 31) { Some((o.cust_key, (sk,o.order_date >> 16))) } else { None } }) - .join_core(&arrangements.customer, |_ck,&(sk,yr),c| Some((c.nation_key, (sk,yr)))) - .join_core(&arrangements.nation, |_nk,&(sk,yr),n| Some((n.region_key, (sk,yr)))) - .join_core(&arrangements.region, |_rk,&(sk,yr),r| { + .join_core(arrangements.customer, |_ck,&(sk,yr),c| Some((c.nation_key, (sk,yr)))) + .join_core(arrangements.nation, |_nk,&(sk,yr),n| Some((n.region_key, (sk,yr)))) + .join_core(arrangements.region, |_rk,&(sk,yr),r| { if starts_with(&r.name, b"AMERICA") { Some((sk,yr,true)) } else { Some((sk,yr,false)) } }) .explode(|(sk,yr,is_name)| Some(((sk,yr), DiffPair::new(if is_name { 1 } else { 0 }, 1)))) - .join_core(&arrangements.supplier, |_sk,&yr,s| Some((s.nation_key, yr))) - .join_core(&arrangements.nation, |_nk,&yr,n| Some((n.name,yr))) + .join_core(arrangements.supplier, |_sk,&yr,s| Some((s.nation_key, yr))) + .join_core(arrangements.nation, |_nk,&yr,n| Some((n.name,yr))) .count_total() .probe_with(probe); } \ No newline at end of file diff --git a/tpchlike/src/queries/query09.rs b/tpchlike/src/queries/query09.rs index f8841c0f1..426a2d8f9 100644 --- a/tpchlike/src/queries/query09.rs +++ b/tpchlike/src/queries/query09.rs @@ -66,15 +66,15 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .lineitems() .map(|l| (l.part_key, (l.supp_key, l.order_key, l.extended_price * (100 - l.discount) / 100, l.quantity))) - .semijoin(&parts) + .semijoin(parts) .map(|(part_key, (supp_key, order_key, revenue, quantity))| ((part_key, supp_key), (order_key, revenue, quantity))) - .join(&collections.partsupps().map(|ps| ((ps.part_key, ps.supp_key), ps.supplycost))) + .join(collections.partsupps().map(|ps| ((ps.part_key, ps.supp_key), ps.supplycost))) .explode(|((_part_key, supp_key), ((order_key, revenue, quantity), supplycost))| Some(((order_key, supp_key), ((revenue - supplycost * quantity) as isize))) ) - .join_map(&collections.orders().map(|o| (o.order_key, o.order_date >> 16)), |_, &supp_key, &order_year| (supp_key, order_year)) - .join_map(&collections.suppliers().map(|s| (s.supp_key, s.nation_key)), |_, &order_year, &nation_key| (nation_key, order_year)) - .join(&collections.nations().map(|n| (n.nation_key, n.name))) + .join_map(collections.orders().map(|o| (o.order_key, o.order_date >> 16)), |_, &supp_key, &order_year| (supp_key, order_year)) + .join_map(collections.suppliers().map(|s| (s.supp_key, s.nation_key)), |_, &order_year, &nation_key| (nation_key, order_year)) + .join(collections.nations().map(|n| (n.nation_key, n.name))) .count_total() .probe_with(probe); } @@ -93,17 +93,17 @@ where experiment .lineitem(scope) .map(|l| (l.part_key, (l.supp_key, l.order_key, l.extended_price * (100 - l.discount) / 100, l.quantity))) - .join_core(&arrangements.part, |&pk,&(sk,ok,ep,qu),p| { + .join_core(arrangements.part, |&pk,&(sk,ok,ep,qu),p| { if substring(&p.name.as_bytes(), b"green") { Some(((pk,sk),(ok,ep,qu))) } else { None } }) - .join_core(&arrangements.partsupp, |&(_pk,sk),&(ok,ep,qu),s| Some((ok, (sk, ep - (qu * s.supplycost))))) + .join_core(arrangements.partsupp, |&(_pk,sk),&(ok,ep,qu),s| Some((ok, (sk, ep - (qu * s.supplycost))))) .explode(|(ok,(sk,am))| Some(((ok,sk), am as isize))) - .join_core(&arrangements.order, |_ok,&sk,o| Some((sk,o.order_date >> 16))) - .join_core(&arrangements.supplier, |_sk,&yr,s| Some((s.nation_key, yr))) - .join_core(&arrangements.nation, |_nk,&yr,n| Some((n.name,yr))) + .join_core(arrangements.order, |_ok,&sk,o| Some((sk,o.order_date >> 16))) + .join_core(arrangements.supplier, |_sk,&yr,s| Some((s.nation_key, yr))) + .join_core(arrangements.nation, |_nk,&yr,n| Some((n.name,yr))) .count_total() .probe_with(probe); } \ No newline at end of file diff --git a/tpchlike/src/queries/query10.rs b/tpchlike/src/queries/query10.rs index 647cb3e9e..b9874d9bf 100644 --- a/tpchlike/src/queries/query10.rs +++ b/tpchlike/src/queries/query10.rs @@ -73,15 +73,15 @@ where G::Timestamp: Lattice+TotalOrder+Ord { } else { None } ) - .semijoin(&lineitems) + .semijoin(lineitems) .map(|(_, cust_key)| cust_key); collections .customers() .map(|c| (c.cust_key, (c.name, c.phone, c.address, c.comment, c.nation_key))) - .semijoin(&orders) + .semijoin(orders) .map(|(cust_key, (name, phn, addr, comm, nation_key))| (nation_key, (cust_key, name, phn, addr, comm))) - .join(&collections.nations().map(|n| (n.nation_key, n.name))) + .join(collections.nations().map(|n| (n.nation_key, n.name))) .count_total() .probe_with(probe); } @@ -106,17 +106,17 @@ where else { None } ) .arrange_by_self() - .join_core(&arrangements.order, |_ok,&(),o| { + .join_core(arrangements.order, |_ok,&(),o| { if create_date(1993,10,1) < o.order_date && o.order_date <= create_date(1994,1,1) { Some(o.cust_key) } else { None } }) .arrange_by_self() - .join_core(&arrangements.customer, |&ck,&(),c| { + .join_core(arrangements.customer, |&ck,&(),c| { Some((c.nation_key, (ck,c.name,c.acctbal,c.address,c.phone,c.comment))) }) - .join_core(&arrangements.nation, |_nk,&data,n| Some((n.name, data))) + .join_core(arrangements.nation, |_nk,&data,n| Some((n.name, data))) .count_total() .probe_with(probe); } \ No newline at end of file diff --git a/tpchlike/src/queries/query11.rs b/tpchlike/src/queries/query11.rs index 1ab40dbea..ff505242a 100644 --- a/tpchlike/src/queries/query11.rs +++ b/tpchlike/src/queries/query11.rs @@ -59,13 +59,13 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .suppliers() .map(|s| (s.nation_key, s.supp_key)) - .semijoin(&nations) + .semijoin(nations) .map(|s| s.1); collections .partsupps() .explode(|x| Some(((x.supp_key, x.part_key), (x.supplycost as isize) * (x.availqty as isize)))) - .semijoin(&suppliers) + .semijoin(suppliers) .map(|(_, part_key)| ((), part_key)) .reduce(|_part_key, s, t| { let threshold: isize = s.iter().map(|x| x.1 as isize).sum::() / 10000; @@ -98,8 +98,8 @@ where .map(move |(d,t,r)| (d, ::std::cmp::max(t,round),r)) .as_collection() .explode(|((sk,pk),prod)| Some(((sk,pk),prod))) - .join_core(&arrangements.supplier, |_sk,&pk,s| Some((s.nation_key, pk))) - .join_core(&arrangements.nation, |_nk,&pk,n| + .join_core(arrangements.supplier, |_sk,&pk,s| Some((s.nation_key, pk))) + .join_core(arrangements.nation, |_nk,&pk,n| if starts_with(&n.name, b"GERMANY") { Some(((), pk)) } else { None } ) .reduce(|_part_key, s, t| { diff --git a/tpchlike/src/queries/query12.rs b/tpchlike/src/queries/query12.rs index efaf74b2b..70e0120d0 100644 --- a/tpchlike/src/queries/query12.rs +++ b/tpchlike/src/queries/query12.rs @@ -81,7 +81,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { .arrange_by_key(); orders - .join_core(&lineitems, |_, _, &ship_mode| Some(ship_mode)) + .join_core(lineitems, |_, _, &ship_mode| Some(ship_mode)) .count_total() // .inspect(|x| println!("{:?}", x)) .probe_with(probe); @@ -108,7 +108,7 @@ where } else { None } ) - .join_core(&arrangements.order, |_ok,&sm,o| { + .join_core(arrangements.order, |_ok,&sm,o| { Some((sm, starts_with(&o.order_priority, b"1-URGENT") || starts_with(&o.order_priority, b"2-HIGH"))) }) .explode(|(sm,priority)| Some((sm, if priority { DiffPair::new(1, 0) } else { DiffPair::new(1, 0) }))) diff --git a/tpchlike/src/queries/query13.rs b/tpchlike/src/queries/query13.rs index 3aa776dfd..d6651e394 100644 --- a/tpchlike/src/queries/query13.rs +++ b/tpchlike/src/queries/query13.rs @@ -50,7 +50,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .customers() .map(|c| c.cust_key) - .concat(&orders) + .concat(orders) .count_total() .map(|(_cust_key, count)| (count-1) as usize) .count_total() @@ -89,7 +89,7 @@ where .inner .map(move |(d,t,r)| (d, ::std::cmp::max(t,round),r)) .as_collection() - .concat(&orders) + .concat(orders) .count_total() .map(|(_cust_key, count)| (count-1) as usize) .count_total() diff --git a/tpchlike/src/queries/query14.rs b/tpchlike/src/queries/query14.rs index 0a8c550e1..85b92cb2c 100644 --- a/tpchlike/src/queries/query14.rs +++ b/tpchlike/src/queries/query14.rs @@ -52,7 +52,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { .parts() .explode(|p| Some((p.part_key, DiffPair::new(1, if starts_with(&p.typ.as_bytes(), b"PROMO") { 1 } else { 0 })))) .arrange_by_self() - .join_core(&lineitems, |&_part_key, _, _| Some(())) + .join_core(lineitems, |&_part_key, _, _| Some(())) .count_total() // .inspect(|x| println!("{:?}", x)) .probe_with(probe); @@ -78,7 +78,7 @@ where else { None } ) .arrange_by_self() - .join_core(&arrangements.part, |_pk,&(),p| Some(DiffPair::new(1, if starts_with(&p.typ.as_bytes(), b"PROMO") { 1 } else { 0 }))) + .join_core(arrangements.part, |_pk,&(),p| Some(DiffPair::new(1, if starts_with(&p.typ.as_bytes(), b"PROMO") { 1 } else { 0 }))) .explode(|dp| Some(((),dp))) .count_total() .probe_with(probe); diff --git a/tpchlike/src/queries/query15.rs b/tpchlike/src/queries/query15.rs index 7cb7a5dea..540592bf8 100644 --- a/tpchlike/src/queries/query15.rs +++ b/tpchlike/src/queries/query15.rs @@ -93,7 +93,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .suppliers() .map(|s| (s.supp_key, (s.name, s.address, s.phone))) - .join(&top_suppliers) + .join(top_suppliers) // .inspect(|x| println!("{:?}", x)) .probe_with(probe); } @@ -148,7 +148,7 @@ where .count_total(); top_suppliers - .join_core(&arrangements.supplier, |_sk,&rev,s| Some((s.supp_key, s.name, s.address, s.phone, rev))) + .join_core(arrangements.supplier, |_sk,&rev,s| Some((s.supp_key, s.name, s.address, s.phone, rev))) .count_total() .probe_with(probe); } \ No newline at end of file diff --git a/tpchlike/src/queries/query16.rs b/tpchlike/src/queries/query16.rs index 932cdf430..c7a61bb9f 100644 --- a/tpchlike/src/queries/query16.rs +++ b/tpchlike/src/queries/query16.rs @@ -64,7 +64,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { let parts = collections .partsupps() .map(|ps| (ps.supp_key, ps.part_key)) - .antijoin(&suppliers) + .antijoin(suppliers) .map(|(_supp_key, part_key)| part_key); collections @@ -75,7 +75,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { } else { None } ) - .semijoin(&parts) + .semijoin(parts) .map(|(_, brand_type_size)| brand_type_size) .count_total() // .inspect(|x| println!("{:?}", x)) @@ -113,14 +113,14 @@ where .inner .map(move |(d,t,r)| (d, ::std::cmp::max(t,round),r)) .as_collection() - .join_core(&arrangements.part, |_pk,&sk,p| { + .join_core(arrangements.part, |_pk,&sk,p| { if !starts_with(&p.brand, b"Brand#45") && !starts_with(&p.typ.as_bytes(), b"MEDIUM POLISHED") && [49, 14, 23, 45, 19, 3, 36, 9].contains(&p.size) { Some((sk, (p.brand, p.typ, p.size))) } else { None } }) - .antijoin(&suppliers) + .antijoin(suppliers) .map(|(_sk, stuff)| stuff) .count_total() .probe_with(probe); diff --git a/tpchlike/src/queries/query17.rs b/tpchlike/src/queries/query17.rs index 07e7df73b..700d6dee1 100644 --- a/tpchlike/src/queries/query17.rs +++ b/tpchlike/src/queries/query17.rs @@ -48,7 +48,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .lineitems() .map(|x| (x.part_key, (x.quantity, x.extended_price))) - .semijoin(&parts) + .semijoin(parts) .reduce(|_k, s, t| { // determine the total and count of quantity. @@ -79,7 +79,7 @@ where experiment .lineitem(scope) .map(|x| (x.part_key, (x.quantity, x.extended_price))) - .join_core(&arrangements.part, |&pk,&(qu,ep),p| { + .join_core(arrangements.part, |&pk,&(qu,ep),p| { if &p.brand[..8] == b"Brand#23" && &p.container[..7] == b"MED BOX" { Some((pk,(qu,ep))) } diff --git a/tpchlike/src/queries/query18.rs b/tpchlike/src/queries/query18.rs index 7df525aba..496a58752 100644 --- a/tpchlike/src/queries/query18.rs +++ b/tpchlike/src/queries/query18.rs @@ -60,8 +60,8 @@ where G::Timestamp: Lattice+TotalOrder+Ord { .explode(|l| Some((l.order_key, l.quantity as isize))) .count_total() .filter(|&(_key, cnt)| cnt > 300) - .join_map(&orders, |&o_key, &quant, &(cust_key, date, price)| (cust_key, (o_key, date, price, quant))) - .join(&collections.customers().map(|c| (c.cust_key, c.name))) + .join_map(orders, |&o_key, &quant, &(cust_key, date, price)| (cust_key, (o_key, date, price, quant))) + .join(collections.customers().map(|c| (c.cust_key, c.name))) .probe_with(probe); } @@ -81,8 +81,8 @@ where .explode(|l| Some((l.order_key, l.quantity as isize))) .count_total() .filter(|&(_key, cnt)| cnt > 300) - .join_core(&arrangements.order, |_ok,&cnt,o| Some((o.cust_key, (o.order_date, o.total_price, cnt)))) - .join_core(&arrangements.customer, |&ck,&(od,tp,cnt),c| Some((ck,c.name,od,tp,cnt))) + .join_core(arrangements.order, |_ok,&cnt,o| Some((o.cust_key, (o.order_date, o.total_price, cnt)))) + .join_core(arrangements.customer, |&ck,&(od,tp,cnt),c| Some((ck,c.name,od,tp,cnt))) .count_total() .probe_with(probe); } \ No newline at end of file diff --git a/tpchlike/src/queries/query19.rs b/tpchlike/src/queries/query19.rs index 755559c82..03e4092dd 100644 --- a/tpchlike/src/queries/query19.rs +++ b/tpchlike/src/queries/query19.rs @@ -78,13 +78,13 @@ where G::Timestamp: Lattice+TotalOrder+Ord { let parts2 = parts.filter(|&(_key, (brand, container, size))| starts_with(&brand, b"Brand#23") && 1 <= size && size <= 10 && (starts_with(&container, b"MED BAG") || starts_with(&container, b"MED BOX") || starts_with(&container, b"MED PKG") || starts_with(&container, b"MED PACK"))).map(|x| x.0).arrange_by_self(); let parts3 = parts.filter(|&(_key, (brand, container, size))| starts_with(&brand, b"Brand#34") && 1 <= size && size <= 15 && (starts_with(&container, b"LG CASE") || starts_with(&container, b"LG BOX") || starts_with(&container, b"LG PACK") || starts_with(&container, b"LG PKG"))).map(|x| x.0).arrange_by_self(); - let result1 = lines1.join_core(&parts1, |_,_,_| Some(())); - let result2 = lines2.join_core(&parts2, |_,_,_| Some(())); - let result3 = lines3.join_core(&parts3, |_,_,_| Some(())); + let result1 = lines1.join_core(parts1, |_,_,_| Some(())); + let result2 = lines2.join_core(parts2, |_,_,_| Some(())); + let result3 = lines3.join_core(parts3, |_,_,_| Some(())); result1 - .concat(&result2) - .concat(&result3) + .concat(result2) + .concat(result3) .count_total() // .inspect(|x| println!("{:?}", x)) .probe_with(probe); @@ -109,7 +109,7 @@ where } else { None } ) - .join_core(&arrangements.part, |_pk,&qu,p| { + .join_core(arrangements.part, |_pk,&qu,p| { if qu >= 1 && qu <= 11 && (starts_with(&p.brand, b"Brand#12") && 1 <= p.size && p.size <= 5 && (starts_with(&p.container, b"SM CASE") || starts_with(&p.container, b"SM BOX") || starts_with(&p.container, b"SM PACK") || starts_with(&p.container, b"MED PKG"))) && qu >= 10 && qu <= 20 && (starts_with(&p.brand, b"Brand#23") && 1 <= p.size && p.size <= 10 && (starts_with(&p.container, b"MED BAG") || starts_with(&p.container, b"MED BOX") || starts_with(&p.container, b"MED PKG") || starts_with(&p.container, b"MED PACK"))) && qu >= 20 && qu <= 30 && (starts_with(&p.brand, b"Brand#12") && 1 <= p.size && p.size <= 15 && (starts_with(&p.container, b"LG CASE") || starts_with(&p.container, b"LG BOX") || starts_with(&p.container, b"LG PACK") || starts_with(&p.container, b"LG PKG"))) diff --git a/tpchlike/src/queries/query20.rs b/tpchlike/src/queries/query20.rs index c127a0396..8c5476fce 100644 --- a/tpchlike/src/queries/query20.rs +++ b/tpchlike/src/queries/query20.rs @@ -75,7 +75,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { } else { None } ) - .semijoin(&partkeys) + .semijoin(partkeys) .explode(|l| Some(((((l.0 as u64) << 32) + (l.1).0 as u64, ()), (l.1).1 as isize))) .reduce_abelian::<_,ValSpine<_,_,_,_>>("Reduce", |_k,s,t| t.push((s[0].1, 1))); @@ -83,9 +83,9 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .partsupps() .map(|ps| (ps.part_key, (ps.supp_key, ps.availqty))) - .semijoin(&partkeys) + .semijoin(partkeys) .map(|(part_key, (supp_key, avail))| (((part_key as u64) << 32) + (supp_key as u64), avail)) - .join_core(&available, |&key, &avail1, &avail2| { + .join_core(available, |&key, &avail1, &avail2| { let key: u64 = key; let avail2: isize = avail2; if avail1 > avail2 as i32 / 2 { @@ -99,9 +99,9 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .suppliers() .map(|s| (s.supp_key, (s.name, s.address, s.nation_key))) - .semijoin(&suppliers) + .semijoin(suppliers) .map(|(_, (name, addr, nation))| (nation, (name, addr))) - .join(&nations) + .join(nations) .probe_with(probe); } @@ -124,16 +124,16 @@ where } else { None } ) - .join_core(&arrangements.part, |&pk,&sk,p| + .join_core(arrangements.part, |&pk,&sk,p| if p.name.as_bytes() == b"forest" { Some((pk,sk)) } else { None } ) .count_total() - .join_core(&arrangements.partsupp, |&(_pk,sk),tot,ps| { + .join_core(arrangements.partsupp, |&(_pk,sk),tot,ps| { if (ps.availqty as isize) > tot / 2 { Some((sk, ())) } else { None } }) .distinct_total() - .join_core(&arrangements.supplier, |_sk,&(),s| Some((s.nation_key, (s.name, s.address)))) - .join_core(&arrangements.nation, |_nk,&(nm,ad),n| + .join_core(arrangements.supplier, |_sk,&(),s| Some((s.nation_key, (s.name, s.address)))) + .join_core(arrangements.nation, |_nk,&(nm,ad),n| if starts_with(&n.name, b"CANADA") { Some((nm,ad)) } else { None } ) .probe_with(probe); diff --git a/tpchlike/src/queries/query21.rs b/tpchlike/src/queries/query21.rs index 095dcd7bc..6a91e6efc 100644 --- a/tpchlike/src/queries/query21.rs +++ b/tpchlike/src/queries/query21.rs @@ -76,7 +76,7 @@ where G::Timestamp: Lattice+TotalOrder+Ord { collections .lineitems() .map(|l| (l.order_key, (l.supp_key, l.receipt_date > l.commit_date))) - .semijoin(&orders); + .semijoin(orders); let lateitems = lineitems.filter(|l| (l.1).1); let lateorders = lateitems.map(|l| l.0).distinct_total(); @@ -84,21 +84,21 @@ where G::Timestamp: Lattice+TotalOrder+Ord { let problems = lineitems .map(|(order_key, (_supp_key, is_late))| (order_key, is_late)) - .semijoin(&lateorders) //- on_time and late, but just one late -\\ + .semijoin(lateorders) //- on_time and late, but just one late -\\ .reduce(|_order_key, s, t| if s.len() == 2 && s[1].1 == 1 { t.push(((), 1)); }) .map(|(order_key, _)| order_key); let latesupps = lateitems - .semijoin(&problems) + .semijoin(problems) .map(|(_order_key, (supp_key, _))| supp_key); collections .suppliers() .map(|s| (s.supp_key, (s.name, s.nation_key))) - .semijoin(&latesupps) + .semijoin(latesupps) .map(|(_, (name, nation))| (nation, name)) - .semijoin(&collections.nations().filter(|n| starts_with(&n.name, b"SAUDI ARABIA")).map(|n| n.nation_key)) + .semijoin(collections.nations().filter(|n| starts_with(&n.name, b"SAUDI ARABIA")).map(|n| n.nation_key)) .count_total() .probe_with(probe); } @@ -127,11 +127,11 @@ where } }) - .join_core(&arrangements.order, |_ok,&sk,o| { + .join_core(arrangements.order, |_ok,&sk,o| { if starts_with(&o.order_status, b"F") { Some((sk, ())) } else { None } }) - .join_core(&arrangements.supplier, |_sk,&(),s| Some((s.nation_key, s.name))) - .join_core(&arrangements.nation, |_nk,&nm,n| + .join_core(arrangements.supplier, |_sk,&(),s| Some((s.nation_key, s.name))) + .join_core(arrangements.nation, |_nk,&nm,n| if starts_with(&n.name, b"SAUDI ARABIA") { Some(nm) } else { None } ) .count_total() diff --git a/tpchlike/src/queries/query22.rs b/tpchlike/src/queries/query22.rs index 407394986..3c24c3b7d 100644 --- a/tpchlike/src/queries/query22.rs +++ b/tpchlike/src/queries/query22.rs @@ -84,9 +84,9 @@ where G::Timestamp: Lattice+TotalOrder+Ord { customers .map(|(cc, acct, key)| (key, (cc, acct))) - .antijoin(&collections.orders().map(|o| o.cust_key).distinct_total()) + .antijoin(collections.orders().map(|o| o.cust_key).distinct_total()) .map(|(_, (cc, acct))| (cc, acct as isize)) - .join_core(&averages, |&cc, &acct, &pair| { + .join_core(averages, |&cc, &acct, &pair| { let acct : isize = acct; let pair : DiffPair = pair; if acct > (pair.element1 / pair.element2) { Some((cc, acct)) } else { None } @@ -146,9 +146,9 @@ where customers .map(|(cc, acct, key)| (key, (cc, acct))) - .antijoin(&orders) + .antijoin(orders) .map(|(_, (cc, acct))| (cc, acct as isize)) - .join_core(&averages, |&cc, &acct, &pair| { + .join_core(averages, |&cc, &acct, &pair| { let acct : isize = acct; let pair : DiffPair = pair; if acct > (pair.element1 / pair.element2) { Some((cc, acct)) } else { None } From 2010052485521582793629e542926d5faa0e45da Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 24 Feb 2026 11:26:36 -0500 Subject: [PATCH 2/6] Add scope as argument to Iterate closure --- README.md | 8 +++--- advent_of_code_2017/src/bin/day_03.rs | 8 +++--- advent_of_code_2017/src/bin/day_06.rs | 8 +++--- advent_of_code_2017/src/bin/day_07.rs | 6 ++--- advent_of_code_2017/src/bin/day_08.rs | 4 +-- advent_of_code_2017/src/bin/day_09.rs | 10 +++---- advent_of_code_2017/src/bin/day_12.rs | 6 ++--- differential-dataflow/examples/arrange.rs | 6 ++--- differential-dataflow/examples/bfs.rs | 6 ++--- differential-dataflow/examples/progress.rs | 21 +++++++-------- .../examples/stackoverflow.rs | 6 ++--- .../src/algorithms/graphs/bfs.rs | 6 ++--- .../src/algorithms/graphs/propagate.rs | 6 ++--- .../src/algorithms/graphs/scc.rs | 8 +++--- .../src/algorithms/graphs/sequential.rs | 6 ++--- .../src/algorithms/identifiers.rs | 8 +++--- .../src/algorithms/prefix_sum.rs | 10 +++---- .../src/operators/iterate.rs | 24 ++++++++++------- differential-dataflow/tests/bfs.rs | 6 ++--- differential-dataflow/tests/scc.rs | 10 +++---- experiments/src/bin/attend.rs | 6 ++--- experiments/src/bin/deals-interactive.rs | 26 +++++++++---------- experiments/src/bin/graphs-interactive-alt.rs | 6 ++--- experiments/src/bin/graphs.rs | 18 ++++++------- experiments/src/bin/graspan-interactive.rs | 6 ++--- experiments/src/bin/multitemporal.rs | 6 ++--- mdbook/src/chapter_1/chapter_1_3.md | 2 +- mdbook/src/chapter_2/chapter_2_7.md | 6 ++--- mdbook/src/chapter_4/chapter_4_1.md | 6 ++--- mdbook/src/chapter_5/chapter_5_4.md | 6 ++--- server/dataflows/reachability/src/lib.rs | 6 ++--- 31 files changed, 135 insertions(+), 132 deletions(-) diff --git a/README.md b/README.md index 6b3c40264..41cfe05f1 100644 --- a/README.md +++ b/README.md @@ -21,8 +21,8 @@ Alternately, here is a fragment that computes the set of nodes reachable from a ```rust let reachable = -roots.iterate(|reach| - edges.enter(&reach.scope()) +roots.iterate(|scope, reach| + edges.enter(&scope) .semijoin(reach) .map(|(src, dst)| dst) .concat(reach) @@ -328,7 +328,7 @@ Here is a direct implementation, in which we repeatedly take determine the set o let k = 5; // iteratively thin edges. -edges.iterate(|inner| { +edges.iterate(|scope, inner| { // determine the active vertices /-- this is a lie --\ let active = inner.flat_map(|(src,dst)| [src,dst].into_iter()) @@ -337,7 +337,7 @@ edges.iterate(|inner| { .map(|(node,_)| node); // keep edges between active vertices - edges.enter(&inner.scope()) + edges.enter(&scope) .semijoin(active) .map(|(src,dst)| (dst,src)) .semijoin(active) diff --git a/advent_of_code_2017/src/bin/day_03.rs b/advent_of_code_2017/src/bin/day_03.rs index 5eefdef1e..d51ab21ab 100644 --- a/advent_of_code_2017/src/bin/day_03.rs +++ b/advent_of_code_2017/src/bin/day_03.rs @@ -16,10 +16,10 @@ fn main() { // // The next odd square is arrived at by following four edges, each of which has length // equal to the even number between the odd numbers. To determine the position of any - // given number, we can subtract the largest odd square from it and then subtract the + // given number, we can subtract the largest odd square from it and then subtract the // multiples of the even length, at which point we have a distance along an edge, from // which we can determine coordinates. - // + // // Similarly, we can determine the sequence number from the coordinates, by determining // which layer the point is in, adding the appropriate squared odd number, and then some // cases to figure out where the point is in the sequence along that layer. @@ -40,7 +40,7 @@ fn main() { values .map(|_val| ((0, 0), (1, 1))) - .iterate(|inner| { + .iterate(|_scope, inner| { inner .filter(move |&(_, (_, val))| val < input) // stop working when we've gotten enough @@ -90,7 +90,7 @@ fn sequence(input: (isize, isize)) -> isize { if -input.1 > layer { layer = -input.1; } // input exists in layer from ((2 * layer - 1)^2, (2 * layer + 1)^2]. - // note: excludes first number, includes last number. + // note: excludes first number, includes last number. let base = (2 * layer - 1) * (2 * layer - 1); diff --git a/advent_of_code_2017/src/bin/day_06.rs b/advent_of_code_2017/src/bin/day_06.rs index a7686ef42..5cae0bef8 100644 --- a/advent_of_code_2017/src/bin/day_06.rs +++ b/advent_of_code_2017/src/bin/day_06.rs @@ -22,9 +22,9 @@ fn main() { let banks = scope.new_collection_from(Some(worker_input)).1; - let stable = banks.iterate(|iter| + let stable = banks.iterate(|scope, iter| iter.map_in_place(|banks| recycle(banks)) - .concat(banks.enter(&iter.scope())) + .concat(banks.enter(&scope)) .distinct() ); @@ -41,9 +41,9 @@ fn main() { // restart iteration from known repeated element. loop_point - .iterate(|iter| + .iterate(|scope, iter| iter.map_in_place(|banks| recycle(banks)) - .concat(loop_point.enter(&iter.scope())) + .concat(loop_point.enter(&scope)) .distinct() ) .map(|_| ((),())) diff --git a/advent_of_code_2017/src/bin/day_07.rs b/advent_of_code_2017/src/bin/day_07.rs index 42a84b854..13b35094e 100644 --- a/advent_of_code_2017/src/bin/day_07.rs +++ b/advent_of_code_2017/src/bin/day_07.rs @@ -1103,11 +1103,11 @@ tvhftq (35)"; let parents = input.flat_map(|(name,_,links)| links.into_iter().map(move |link| (link,name.to_string()))); let total_weights: VecCollection<_,String> = weights - .iterate(|inner| { - parents.enter(&inner.scope()) + .iterate(|scope, inner| { + parents.enter(&scope) .semijoin(inner) .map(|(_, parent)| parent) - .concat(weights.enter(&inner.scope())) + .concat(weights.enter(&scope)) }); parents diff --git a/advent_of_code_2017/src/bin/day_08.rs b/advent_of_code_2017/src/bin/day_08.rs index 62dc6a12b..385bc6489 100644 --- a/advent_of_code_2017/src/bin/day_08.rs +++ b/advent_of_code_2017/src/bin/day_08.rs @@ -1105,9 +1105,9 @@ wui inc -120 if i > -2038"; edits .filter(|_| false) .map(|_| ((0, String::new()), 0)) - .iterate(|valid| { + .iterate(|scope, valid| { - let edits = edits.enter(&valid.scope()); + let edits = edits.enter(&scope); valid .prefix_sum_at(edits.map(|(key,_)| key), 0, |_k,x,y| *x + *y) diff --git a/advent_of_code_2017/src/bin/day_09.rs b/advent_of_code_2017/src/bin/day_09.rs index 9e700e44f..b462993cf 100644 --- a/advent_of_code_2017/src/bin/day_09.rs +++ b/advent_of_code_2017/src/bin/day_09.rs @@ -105,7 +105,7 @@ where let unit_ranges = collection.map(|(index, data)| ((index, 0), data)); unit_ranges - .iterate(|ranges| + .iterate(|scope, ranges| // Each available range, of size less than usize::max_value(), advertises itself as the range // twice as large, aligned to integer multiples of its size. Each range, which may contain at @@ -120,7 +120,7 @@ where if input.len() > 1 { result = combine(result, &(input[1].0).1); } output.push((result, 1)); }) - .concat(unit_ranges.enter(&ranges.scope())) + .concat(unit_ranges.enter(&scope)) ) } @@ -151,13 +151,13 @@ where .as_collection(); init_state - .iterate(|state| { + .iterate(|scope, state| { aggregates .filter(|&((_, log),_)| log < 64) // the log = 64 interval doesn't help us here (overflows). - .enter(&state.scope()) + .enter(&scope) .map(|((pos, log), data)| (pos, (log, data))) .join_map(state, move |&pos, &(log, ref data), state| (pos + (1 << log), combine(state, data))) - .concat(init_state.enter(&state.scope())) + .concat(init_state.enter(&scope)) .distinct() }) .consolidate() diff --git a/advent_of_code_2017/src/bin/day_12.rs b/advent_of_code_2017/src/bin/day_12.rs index 1db76553e..4c92ee174 100644 --- a/advent_of_code_2017/src/bin/day_12.rs +++ b/advent_of_code_2017/src/bin/day_12.rs @@ -2034,9 +2034,9 @@ fn main() { let labels = nodes - .iterate(|label| { - let edges = edges.enter(&label.scope()); - let nodes = nodes.enter(&label.scope()); + .iterate(|scope, label| { + let edges = edges.enter(&scope); + let nodes = nodes.enter(&scope); label .join_map(edges, |_src, &lbl, &tgt| (tgt, lbl)) .concat(nodes) diff --git a/differential-dataflow/examples/arrange.rs b/differential-dataflow/examples/arrange.rs index 201627e2c..d3f5abcec 100644 --- a/differential-dataflow/examples/arrange.rs +++ b/differential-dataflow/examples/arrange.rs @@ -107,10 +107,10 @@ fn main() { let roots = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - roots.clone().iterate(|dists| { + roots.clone().iterate(|scope, dists| { - let edges = edges.enter(&dists.scope()); - let roots = roots.enter(&dists.scope()); + let edges = edges.enter(&scope); + let roots = roots.enter(&scope); dists.arrange_by_key() .join_core(edges, |_k,l,d| Some((*d, l+1))) diff --git a/differential-dataflow/examples/bfs.rs b/differential-dataflow/examples/bfs.rs index 0a35bb4e5..af875d5d5 100644 --- a/differential-dataflow/examples/bfs.rs +++ b/differential-dataflow/examples/bfs.rs @@ -99,10 +99,10 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.clone().iterate(|inner| { + nodes.clone().iterate(|scope, inner| { - let edges = edges.enter(&inner.scope()); - let nodes = nodes.enter(&inner.scope()); + let nodes = nodes.enter(&scope); + let edges = edges.enter(&scope); inner.join_map(edges, |_k,l,d| (*d, l+1)) .concat(nodes) diff --git a/differential-dataflow/examples/progress.rs b/differential-dataflow/examples/progress.rs index 67783e488..634e43d56 100644 --- a/differential-dataflow/examples/progress.rs +++ b/differential-dataflow/examples/progress.rs @@ -130,12 +130,12 @@ where times .clone() - .iterate(|reach| { + .iterate(|scope, reach| { transitions - .enter(&reach.scope()) - .join_map(reach.clone(), |_from, (dest, summ), time| (dest.clone(), summ.results_in(time))) + .enter(&scope) + .join_map(reach, |_from, (dest, summ), time| (dest.clone(), summ.results_in(time))) .flat_map(|(dest, time)| time.map(move |time| (dest, time))) - .concat(times.enter(&reach.scope())) + .concat(times.enter(&scope)) .reduce(|_location, input, output: &mut Vec<(T, isize)>| { // retain the lower envelope of times. for (t1, _count1) in input.iter() { @@ -172,12 +172,12 @@ where zero_inputs .clone() - .iterate(|summaries| { + .iterate(|scope, summaries| { transitions - .enter(&summaries.scope()) - .join_map(summaries.clone(), |_middle, (from, summ1), (to, summ2)| (from.clone(), to.clone(), summ1.followed_by(summ2))) + .enter(&scope) + .join_map(summaries, |_middle, (from, summ1), (to, summ2)| (from.clone(), to.clone(), summ1.followed_by(summ2))) .flat_map(|(from, to, summ)| summ.map(move |summ| (from, (to, summ)))) - .concat(zero_inputs.enter(&summaries.scope())) + .concat(zero_inputs.enter(&scope)) .map(|(from, (to, summary))| ((from, to), summary)) .reduce(|_from_to, input, output| { for (summary, _count) in input.iter() { @@ -217,14 +217,13 @@ where // Repeatedly restrict to locations with an incoming path. transitions .clone() - .iterate(|locations| { + .iterate(|scope, locations| { let active = locations - .clone() .map(|(_source, target)| target) .distinct(); transitions - .enter(&locations.scope()) + .enter(&scope) .semijoin(active) }) .consolidate() diff --git a/differential-dataflow/examples/stackoverflow.rs b/differential-dataflow/examples/stackoverflow.rs index c84ece846..3fe521ee2 100644 --- a/differential-dataflow/examples/stackoverflow.rs +++ b/differential-dataflow/examples/stackoverflow.rs @@ -113,10 +113,10 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.clone().iterate(|inner| { + nodes.clone().iterate(|scope, inner| { - let edges = edges.enter(&inner.scope()); - let nodes = nodes.enter(&inner.scope()); + let edges = edges.enter(&scope); + let nodes = nodes.enter(&scope); inner.join_map(edges, |_k,l,d| (*d, l+1)) .concat(nodes) diff --git a/differential-dataflow/src/algorithms/graphs/bfs.rs b/differential-dataflow/src/algorithms/graphs/bfs.rs index 4ccc89403..1513622bd 100644 --- a/differential-dataflow/src/algorithms/graphs/bfs.rs +++ b/differential-dataflow/src/algorithms/graphs/bfs.rs @@ -32,10 +32,10 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.clone().iterate(|inner| { + nodes.clone().iterate(|scope, inner| { - let edges = edges.enter(&inner.scope()); - let nodes = nodes.enter(&inner.scope()); + let edges = edges.enter(&scope); + let nodes = nodes.enter(&scope); inner.join_core(edges, |_k,l,d| Some((d.clone(), l+1))) .concat(nodes) diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 0755d470c..2eb732bd5 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -69,9 +69,9 @@ where // iterative computation so that the arrangement produced by `reduce` can be re-used. // nodes.filter(|_| false) - // .iterate(|inner| { - // let edges = edges.enter(&inner.scope()); - // let nodes = nodes.enter_at(&inner.scope(), move |r| 256 * (64 - (logic(&r.1)).leading_zeros() as u64)); + // .iterate(|scope, inner| { + // let edges = edges.enter(&scope); + // let nodes = nodes.enter_at(&scope, move |r| 256 * (64 - (logic(&r.1)).leading_zeros() as u64)); // inner.join_map(edges, |_k,l,d| (d.clone(),l.clone())) // .concat(nodes) // .reduce(|_, s, t| t.push((s[0].0.clone(), 1))) diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index 390174ed4..f7bca525f 100644 --- a/differential-dataflow/src/algorithms/graphs/scc.rs +++ b/differential-dataflow/src/algorithms/graphs/scc.rs @@ -21,9 +21,9 @@ where R: Multiply, R: From, { - graph.clone().iterate(|edges| { + graph.clone().iterate(|scope, edges| { // keep edges from active edge destinations. - let graph = graph.enter(&edges.scope()); + let graph = graph.enter(&scope); let active = edges.map(|(_src,dst)| dst) .threshold(|_,c| if c.is_zero() { R::from(0_i8) } else { R::from(1_i8) }); @@ -41,8 +41,8 @@ where R: Multiply, R: From { - graph.clone().iterate(|inner| { - let edges = graph.enter(&inner.scope()); + graph.clone().iterate(|scope, inner| { + let edges = graph.enter(&scope); let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); trim_edges(trim_edges(inner, edges), trans) }) diff --git a/differential-dataflow/src/algorithms/graphs/sequential.rs b/differential-dataflow/src/algorithms/graphs/sequential.rs index a98f8d843..354ad32c2 100644 --- a/differential-dataflow/src/algorithms/graphs/sequential.rs +++ b/differential-dataflow/src/algorithms/graphs/sequential.rs @@ -55,10 +55,10 @@ where state .clone() .map(|(node, _state)| (node, None)) - .iterate(|new_state| { + .iterate(|scope, new_state| { // immutable content: edges and initial state. - let edges = edges.enter(&new_state.scope()); - let old_state = state.enter(&new_state.scope()); + let edges = edges.enter(&scope); + let old_state = state.enter(&scope); // .map(|x| (x.0, Some(x.1))); // break edges into forward and reverse directions. diff --git a/differential-dataflow/src/algorithms/identifiers.rs b/differential-dataflow/src/algorithms/identifiers.rs index e38a46ba1..267e4c453 100644 --- a/differential-dataflow/src/algorithms/identifiers.rs +++ b/differential-dataflow/src/algorithms/identifiers.rs @@ -57,9 +57,9 @@ where let init = self.map(|record| (0, record)); timely::dataflow::operators::generic::operator::empty(&init.scope()) .as_collection() - .iterate(|diff| + .iterate(|scope, diff| init.clone() - .enter(&diff.scope()) + .enter(&scope) .concat(diff) .map(|pair| (pair.hashed(), pair)) .reduce(|_hash, input, output| { @@ -109,9 +109,9 @@ mod tests { let init = input.map(|record| (0, record)); timely::dataflow::operators::generic::operator::empty(&init.scope()) .as_collection() - .iterate(|diff| + .iterate(|scope, diff| init.clone() - .enter(&diff.scope()) + .enter(&scope) .concat(diff) .map(|(round, num)| ((round + num) / 10, (round, num))) .reduce(|_hash, input, output| { diff --git a/differential-dataflow/src/algorithms/prefix_sum.rs b/differential-dataflow/src/algorithms/prefix_sum.rs index 9a8767f47..f76f59d9c 100644 --- a/differential-dataflow/src/algorithms/prefix_sum.rs +++ b/differential-dataflow/src/algorithms/prefix_sum.rs @@ -52,14 +52,14 @@ where unit_ranges .clone() - .iterate(|ranges| { + .iterate(|scope, ranges| { // Each available range, of size less than usize::max_value(), advertises itself as the range // twice as large, aligned to integer multiples of its size. Each range, which may contain at // most two elements, then summarizes itself using the `combine` function. Finally, we re-add // the initial `unit_ranges` intervals, so that the set of ranges grows monotonically. - let unit_ranges = unit_ranges.enter(&ranges.scope()); + let unit_ranges = unit_ranges.enter(&scope); ranges .filter(|&((_pos, log, _), _)| log < 64) .map(|((pos, log, key), data)| ((pos >> 1, log + 1, key), (pos, data))) @@ -141,10 +141,10 @@ where // Iteratively expand assigned values by joining existing ranges with current assignments. init_states .clone() - .iterate(|states| { - let init_states = init_states.enter(&states.scope()); + .iterate(|scope, states| { + let init_states = init_states.enter(&scope); used_ranges - .enter(&states.scope()) + .enter(&scope) .map(|((pos, log, key), data)| ((pos << log, key), (log, data))) .join_map(states, move |&(pos, ref key), &(log, ref data), state| ((pos + (1 << log), key.clone()), combine(key, state, data))) diff --git a/differential-dataflow/src/operators/iterate.rs b/differential-dataflow/src/operators/iterate.rs index 35d4b6e3b..7c8cff4f9 100644 --- a/differential-dataflow/src/operators/iterate.rs +++ b/differential-dataflow/src/operators/iterate.rs @@ -51,10 +51,13 @@ pub trait Iterate, D: Data, R: Semigroup> { /// Iteratively apply `logic` to the source collection until convergence. /// /// Importantly, this method does not automatically consolidate results. - /// It may be important to conclude with `consolidate()` to ensure that - /// logically empty collections that contain cancelling records do not - /// result in non-termination. Operators like `reduce`, `distinct`, and - /// `count` also perform consolidation, and are safe to conclude with. + /// It may be important to conclude the closure you supply with `consolidate()` to ensure that + /// logically empty collections that contain cancelling records do not result in non-termination. + /// Operators like `reduce`, `distinct`, and `count` also perform consolidation, and are safe to conclude with. + /// + /// The closure is also passed a copy of the inner scope, to facilitate importing external collections. + /// It can also be acquired by calling `.scope()` on the closure's collection argument, but the code + /// can be awkward to write fluently. /// /// # Examples /// @@ -65,7 +68,7 @@ pub trait Iterate, D: Data, R: Semigroup> { /// ::timely::example(|scope| { /// /// scope.new_collection_from(1 .. 10u32).1 - /// .iterate(|values| { + /// .iterate(|_scope, values| { /// values.map(|x| if x % 2 == 0 { x/2 } else { x }) /// .consolidate() /// }); @@ -73,13 +76,13 @@ pub trait Iterate, D: Data, R: Semigroup> { /// ``` fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(VecCollection, D, R>)->VecCollection, D, R>; + for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>; } impl, D: Ord+Data+Debug, R: Abelian+'static> Iterate for VecCollection { fn iterate(self, logic: F) -> VecCollection where - for<'a> F: FnOnce(VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, { self.inner.scope().scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. @@ -89,7 +92,8 @@ impl, D: Ord+Data+Debug, R: Abelian+'static> Iterat // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let variable = Variable::new_from(self.enter(subgraph), Product::new(Default::default(), 1)); - let result = logic(variable.clone()); + let inner_scope = variable.scope(); + let result = logic(inner_scope, variable.clone()); variable.set(result.clone()); result.leave() }) @@ -99,7 +103,7 @@ impl, D: Ord+Data+Debug, R: Abelian+'static> Iterat impl, D: Ord+Data+Debug, R: Semigroup+'static> Iterate for G { fn iterate(mut self, logic: F) -> VecCollection where - for<'a> F: FnOnce(VecCollection, D, R>)->VecCollection, D, R>, + for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, { self.scoped("Iterate", |subgraph| { // create a new variable, apply logic, bind variable, return. @@ -109,7 +113,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let variable = SemigroupVariable::new(subgraph, Product::new(Default::default(), 1)); - let result = logic(variable.clone()); + let result = logic(subgraph.clone(), variable.clone()); variable.set(result.clone()); result.leave() } diff --git a/differential-dataflow/tests/bfs.rs b/differential-dataflow/tests/bfs.rs index 44fb138c0..fd6aea26c 100644 --- a/differential-dataflow/tests/bfs.rs +++ b/differential-dataflow/tests/bfs.rs @@ -210,10 +210,10 @@ where let nodes = roots.map(|x| (x, 0)); // repeatedly update minimal distances each node can be reached from each root - nodes.clone().iterate(|inner| { + nodes.clone().iterate(|scope, inner| { - let edges = edges.enter(&inner.scope()); - let nodes = nodes.enter(&inner.scope()); + let edges = edges.enter(&scope); + let nodes = nodes.enter(&scope); inner.join_map(edges, |_k,l,d| (*d, l+1)) .concat(nodes) diff --git a/differential-dataflow/tests/scc.rs b/differential-dataflow/tests/scc.rs index fe6638ffc..52eecdf1d 100644 --- a/differential-dataflow/tests/scc.rs +++ b/differential-dataflow/tests/scc.rs @@ -219,8 +219,8 @@ fn _strongly_connected(graph: VecCollection) -> VecCollection, { - graph.clone().iterate(|inner| { - let edges = graph.enter(&inner.scope()); + graph.clone().iterate(|scope, inner| { + let edges = graph.enter(&scope); let trans = edges.clone().map_in_place(|x| mem::swap(&mut x.0, &mut x.1)); _trim_edges(_trim_edges(inner, edges), trans) }) @@ -250,9 +250,9 @@ where { edges.clone() // <-- wth is this. .filter(|_| false) - .iterate(|inner| { - let edges = edges.enter(&inner.scope()); - let nodes = nodes.enter_at(&inner.scope(), |r| 256 * (64 - (r.0 as u64).leading_zeros() as u64)); + .iterate(|scope, inner| { + let edges = edges.enter(&scope); + let nodes = nodes.enter_at(&scope, |r| 256 * (64 - (r.0 as u64).leading_zeros() as u64)); inner.join_map(edges, |_k,l,d| (*d,*l)) .concat(nodes) diff --git a/experiments/src/bin/attend.rs b/experiments/src/bin/attend.rs index 94bf6c699..0d3d0369f 100644 --- a/experiments/src/bin/attend.rs +++ b/experiments/src/bin/attend.rs @@ -21,12 +21,12 @@ fn main() { .threshold_total(|_,w| if w.1 == 0 { 1 } else { 0 }); organizers - .iterate(|attend| { - graph.enter(&attend.scope()) + .iterate(|scope, attend| { + graph.enter(&scope) .semijoin(attend) .map(|(_,y)| y) .threshold_total(|_,w| if w >= &3 { 1 } else { 0 }) - .concat(organizers.enter(&attend.scope())) + .concat(organizers.enter(&scope)) .consolidate() }) .map(|_| ()) diff --git a/experiments/src/bin/deals-interactive.rs b/experiments/src/bin/deals-interactive.rs index 02bee9438..28495612b 100644 --- a/experiments/src/bin/deals-interactive.rs +++ b/experiments/src/bin/deals-interactive.rs @@ -215,11 +215,11 @@ where G::Timestamp: Lattice{ // descendants of tc_1: let query1 = tc_1.map(|x| (x,x)) - .iterate(|inner| + .iterate(|scope, inner| edges - .enter(&inner.scope()) + .enter(&scope) .join_core(inner.arrange_by_key(), |_,&y,&q| [(y,q)]) - .concat(tc_1.enter(&inner.scope()).map(|x| (x,x))) + .concat(tc_1.enter(&scope).map(|x| (x,x))) .distinct() ) .map(|(x,q)| (q,x)); @@ -227,12 +227,12 @@ where G::Timestamp: Lattice{ // ancestors of tc_2: let query2 = tc_2.map(|x| (x,x)) - .iterate(|inner| + .iterate(|scope, inner| edges .as_collection(|&k,&v| (v,k)) - .enter(&inner.scope()) + .enter(&scope) .join_core(inner.arrange_by_key(), |_,&y,&q| [(y,q)]) - .concat(tc_2.enter(&inner.scope()).map(|x| (x,x))) + .concat(tc_2.enter(&scope).map(|x| (x,x))) .distinct() ) .map(|(x,q)| (q,x)); @@ -243,13 +243,13 @@ where G::Timestamp: Lattice{ // ancestors of sg_x: let magic = - sg_x.iterate(|inner| + sg_x.iterate(|scope, inner| edges .as_collection(|&k,&v| (v,k)) - .enter(&inner.scope()) + .enter(&scope) .semijoin(inner) .map(|(_x,y)| y) - .concat(sg_x.enter(&inner.scope())) + .concat(sg_x.enter(&scope)) .distinct() ); @@ -263,11 +263,11 @@ where G::Timestamp: Lattice{ let query3 = magic .map(|x| (x,x)) // for query q, sg(x,x) - .iterate(|inner| { + .iterate(|scope, inner| { - let edges = edges.enter(&inner.scope()); - let magic = magic.enter(&inner.scope()); - let magic_edges = magic_edges.enter(&inner.scope()); + let edges = edges.enter(&scope); + let magic = magic.enter(&scope); + let magic_edges = magic_edges.enter(&scope); let result = inner diff --git a/experiments/src/bin/graphs-interactive-alt.rs b/experiments/src/bin/graphs-interactive-alt.rs index db84a4e81..c9944f6a0 100644 --- a/experiments/src/bin/graphs-interactive-alt.rs +++ b/experiments/src/bin/graphs-interactive-alt.rs @@ -376,10 +376,10 @@ where G::Timestamp: Lattice + std::hash::Hash { // don't actually use these labels, just grab the type nodes .filter(|_| false) - .iterate(|inner| { + .iterate(|scope, inner| { - let graph = graph.enter(&inner.scope()); - let nodes = nodes.enter_at(&inner.scope(), |r| 256 * (64 - r.1.leading_zeros() as u64)); + let graph = graph.enter(&scope); + let nodes = nodes.enter_at(&scope, |r| 256 * (64 - r.1.leading_zeros() as u64)); let inner = inner.arrange_by_key(); diff --git a/experiments/src/bin/graphs.rs b/experiments/src/bin/graphs.rs index 44433a2c8..99d79ef66 100644 --- a/experiments/src/bin/graphs.rs +++ b/experiments/src/bin/graphs.rs @@ -95,10 +95,10 @@ fn reach> ( let graph = graph.import(&roots.scope()); - roots.iterate(|inner| { + roots.iterate(|scope, inner| { - let graph = graph.enter(&inner.scope()); - let roots = roots.enter(&inner.scope()); + let graph = graph.enter(&scope); + let roots = roots.enter(&scope); // let reach = inner.concat(roots).distinct_total().arrange_by_self(); // graph.join_core(reach, |_src,&dst,&()| Some(dst)) @@ -118,10 +118,10 @@ fn bfs> ( let graph = graph.import(&roots.scope()); let roots = roots.map(|r| (r,0)); - roots.iterate(|inner| { + roots.iterate(|scope, inner| { - let graph = graph.enter(&inner.scope()); - let roots = roots.enter(&inner.scope()); + let graph = graph.enter(&scope); + let roots = roots.enter(&scope); graph.join_core(inner.arrange_by_key(), |_src,&dest,&dist| [(dest, dist+1)]) .concat(roots) @@ -148,9 +148,9 @@ fn bfs> ( // // don't actually use these labels, just grab the type // nodes.filter(|_| false) -// .iterate(|inner| { -// let edges = edges.enter(&inner.scope()); -// let nodes = nodes.enter_at(&inner.scope(), |r| 256 * (64 - r.1.leading_zeros() as u64)); +// .iterate(|scope, inner| { +// let edges = edges.enter(&scope); +// let nodes = nodes.enter_at(&scope, |r| 256 * (64 - r.1.leading_zeros() as u64)); // inner.join_map(edges, |_k,l,d| (*d,*l)) // .concat(nodes) diff --git a/experiments/src/bin/graspan-interactive.rs b/experiments/src/bin/graspan-interactive.rs index b408a6c01..c75df49e8 100644 --- a/experiments/src/bin/graspan-interactive.rs +++ b/experiments/src/bin/graspan-interactive.rs @@ -27,10 +27,10 @@ fn main() { let probe = nodes .filter(|_| false) - .iterate(|inner| { + .iterate(|scope, inner| { - let nodes = nodes.enter(&inner.scope()); - let edges = edges.enter(&inner.scope()); + let nodes = nodes.enter(&scope); + let edges = edges.enter(&scope); inner .map(|(a,b)| (b,a)) diff --git a/experiments/src/bin/multitemporal.rs b/experiments/src/bin/multitemporal.rs index b105723a8..375494fa1 100644 --- a/experiments/src/bin/multitemporal.rs +++ b/experiments/src/bin/multitemporal.rs @@ -33,10 +33,10 @@ fn main() { // .inspect(|x| println!("edge: {:?}", x)) ; - roots.iterate(|inner| { + roots.iterate(|scope, inner| { - let edges = edges.enter(&inner.scope()); - let roots = roots.enter(&inner.scope()); + let edges = edges.enter(&scope); + let roots = roots.enter(&scope); edges .semijoin(inner) diff --git a/mdbook/src/chapter_1/chapter_1_3.md b/mdbook/src/chapter_1/chapter_1_3.md index 60a0ca9a9..70579a147 100644 --- a/mdbook/src/chapter_1/chapter_1_3.md +++ b/mdbook/src/chapter_1/chapter_1_3.md @@ -12,7 +12,7 @@ Let's describe an iterative computation that starts from manager-employee relati ```rust,ignore manager_employee - .iterate(|manages| { + .iterate(|_scope, manages| { // if x manages y, and y manages z, then x manages z (transitively). manages .map(|(x, y)| (y, x)) diff --git a/mdbook/src/chapter_2/chapter_2_7.md b/mdbook/src/chapter_2/chapter_2_7.md index cb5ac846a..1eb240569 100644 --- a/mdbook/src/chapter_2/chapter_2_7.md +++ b/mdbook/src/chapter_2/chapter_2_7.md @@ -15,7 +15,7 @@ As an example, we can take our `manages` relation and determine for all employee # where G::Timestamp: Lattice # { manages // transitive contains (manager, person) for many hops. - .iterate(|transitive| { + .iterate(|_scope, transitive| { transitive .clone() .map(|(mk, m1)| (m1, mk)) @@ -53,9 +53,9 @@ In the example above, we could rewrite # { manages // transitive contains (manager, person) for many hops. .clone() - .iterate(|transitive| { + .iterate(|scope, transitive| { - let manages = manages.enter(&transitive.scope()); + let manages = manages.enter(&scope); transitive .map(|(mk, m1)| (m1, mk)) diff --git a/mdbook/src/chapter_4/chapter_4_1.md b/mdbook/src/chapter_4/chapter_4_1.md index 76e819fbe..268367b06 100644 --- a/mdbook/src/chapter_4/chapter_4_1.md +++ b/mdbook/src/chapter_4/chapter_4_1.md @@ -14,9 +14,9 @@ Let's write this computation starting from a collection `edges`, using different .distinct(); labels - .iterate(|inner| { - let labels = labels.enter(inner.scope()); - let edges = edges.enter(inner.scope()); + .iterate(|scope, inner| { + let labels = labels.enter(&scope); + let edges = edges.enter(&scope); inner.join(edges) .map(|(_src,(lbl,dst))| (dst,lbl)) .concat(labels) diff --git a/mdbook/src/chapter_5/chapter_5_4.md b/mdbook/src/chapter_5/chapter_5_4.md index 25a3e097f..35ebe9ecd 100644 --- a/mdbook/src/chapter_5/chapter_5_4.md +++ b/mdbook/src/chapter_5/chapter_5_4.md @@ -41,10 +41,10 @@ fn main() { let knows = knows.arrange_by_key(); // Reachability queries. - query.clone().iterate(|reach| { + query.clone().iterate(|scope, reach| { - let knows = knows.enter(&reach.scope()); - let query = query.enter(&reach.scope()); + let knows = knows.enter(&scope); + let query = query.enter(&scope); knows.join_core(reach.arrange_by_key(), |x,y,q| [(*y,*q)]) .concat(query) diff --git a/server/dataflows/reachability/src/lib.rs b/server/dataflows/reachability/src/lib.rs index 3c0f2f15a..01012c0a1 100644 --- a/server/dataflows/reachability/src/lib.rs +++ b/server/dataflows/reachability/src/lib.rs @@ -19,9 +19,9 @@ pub fn build((dataflow, handles, probe, _timer, args): Environment) -> Result<() let (_input, roots) = dataflow.new_collection_from(Some(source)); // repeatedly update minimal distances each node can be reached from each root - roots.iterate(|dists| { - let edges = edges.enter(&dists.scope()); - let roots = roots.enter(&dists.scope()); + roots.iterate(|scope, dists| { + let edges = edges.enter(&scope); + let roots = roots.enter(&scope); dists.arrange_by_self() .join_core(edges, |_src, _, &dst| Some(dst)) .concat(roots) From 44e5557cb8d945624f385e0d08f0a9d1667c3372 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 24 Feb 2026 11:31:14 -0500 Subject: [PATCH 3/6] Point at public but unpublished timely version --- Cargo.toml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 3bfb487a4..121a1c550 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,7 +23,8 @@ edition = "2021" differential-dataflow = { path = "differential-dataflow", default-features = false, version = "0.19.1" } #timely = { version = "0.26", default-features = false } columnar = { version = "0.11", default-features = false } -timely = { path = "../timely-dataflow/timely/", default-features = false } +timely = { git = "https://github.com/TimelyDataflow/timely-dataflow" } +#timely = { path = "../timely-dataflow/timely/", default-features = false } [profile.release] opt-level = 3 From 1bbfd171b644b8d95c01e68ab30c9cf8dd353f5b Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 24 Feb 2026 13:38:42 -0500 Subject: [PATCH 4/6] Add Variable::collection() method --- differential-dataflow/examples/dynamic.rs | 2 +- differential-dataflow/examples/graspan.rs | 6 +-- .../examples/iterate_container.rs | 2 +- differential-dataflow/examples/monoid-bfs.rs | 2 +- differential-dataflow/examples/pagerank.rs | 2 +- .../src/algorithms/graphs/bijkstra.rs | 16 ++++---- .../src/algorithms/graphs/propagate.rs | 2 +- .../src/operators/iterate.rs | 41 ++++++++----------- differential-dataflow/src/operators/join.rs | 5 ++- differential-dataflow/src/operators/reduce.rs | 2 +- mdbook/src/chapter_2/chapter_2_7.md | 2 +- 11 files changed, 38 insertions(+), 44 deletions(-) diff --git a/differential-dataflow/examples/dynamic.rs b/differential-dataflow/examples/dynamic.rs index 387571c27..3076efc32 100644 --- a/differential-dataflow/examples/dynamic.rs +++ b/differential-dataflow/examples/dynamic.rs @@ -117,7 +117,7 @@ where let next = label - .clone() + .collection() .join_map(edges, |_k,l,d| (*d, l+1)) .concat(nodes) .reduce(|_, s, t| t.push((*s[0].0, 1))) diff --git a/differential-dataflow/examples/graspan.rs b/differential-dataflow/examples/graspan.rs index b108a0d40..f061f6dfe 100644 --- a/differential-dataflow/examples/graspan.rs +++ b/differential-dataflow/examples/graspan.rs @@ -113,14 +113,14 @@ impl> EdgeVariable { /// The collection arranged in the forward direction. pub fn forward(&mut self) -> &Arrange { if self.forward.is_none() { - self.forward = Some(self.variable.clone().arrange_by_key()); + self.forward = Some(self.variable.collection().arrange_by_key()); } self.forward.as_ref().unwrap() } /// The collection arranged in the reverse direction. pub fn reverse(&mut self) -> &Arrange { if self.reverse.is_none() { - self.reverse = Some(self.variable.clone().map(|(x,y)| (y,x)).arrange_by_key()); + self.reverse = Some(self.variable.collection().map(|(x,y)| (y,x)).arrange_by_key()); } self.reverse.as_ref().unwrap() } @@ -169,7 +169,7 @@ impl Query { // create variables and result handles for each named relation. for (name, (input, collection)) in input_map { let edge_variable = EdgeVariable::from(collection.enter(subscope), Product::new(Default::default(), 1)); - let trace = edge_variable.variable.clone().leave().arrange_by_self().trace; + let trace = edge_variable.variable.collection().leave().arrange_by_self().trace; result_map.insert(name.clone(), RelationHandles { input, trace }); variable_map.insert(name.clone(), edge_variable); } diff --git a/differential-dataflow/examples/iterate_container.rs b/differential-dataflow/examples/iterate_container.rs index dcd330baa..223382bd0 100644 --- a/differential-dataflow/examples/iterate_container.rs +++ b/differential-dataflow/examples/iterate_container.rs @@ -56,7 +56,7 @@ fn main() { scope.iterative::(|nested| { let summary = Product::new(Default::default(), 1); let variable = Variable::new_from(numbers.enter(nested), summary); - let mapped: Collection<_, _> = variable.clone().inner.unary(Pipeline, "Map", |_,_| { + let mapped: Collection<_, _> = variable.collection().inner.unary(Pipeline, "Map", |_,_| { |input, output| { input.for_each(|time, data| { let mut session = output.session(&time); diff --git a/differential-dataflow/examples/monoid-bfs.rs b/differential-dataflow/examples/monoid-bfs.rs index c99f3b24a..8720ff8fb 100644 --- a/differential-dataflow/examples/monoid-bfs.rs +++ b/differential-dataflow/examples/monoid-bfs.rs @@ -140,7 +140,7 @@ where let result = variable - .clone() + .collection() .map(|n| (n,())) .join_map(edges, |_k,&(),d| *d) .concat(roots) diff --git a/differential-dataflow/examples/pagerank.rs b/differential-dataflow/examples/pagerank.rs index cc0cd1444..5c9633191 100644 --- a/differential-dataflow/examples/pagerank.rs +++ b/differential-dataflow/examples/pagerank.rs @@ -109,7 +109,7 @@ where // Match each surfer with the degree, scale numbers down. let to_push = - degrs.semijoin(ranks.clone()) + degrs.semijoin(ranks.collection()) .threshold(|(_node, degr), rank| (5 * rank) / (6 * degr)) .map(|(node, _degr)| node); diff --git a/differential-dataflow/src/algorithms/graphs/bijkstra.rs b/differential-dataflow/src/algorithms/graphs/bijkstra.rs index df563c5f7..e4c556d3a 100644 --- a/differential-dataflow/src/algorithms/graphs/bijkstra.rs +++ b/differential-dataflow/src/algorithms/graphs/bijkstra.rs @@ -58,8 +58,8 @@ where let forward = Variable::new_from(goals.clone().map(|(x,_)| (x.clone(),(x.clone(),0))).enter(inner), Product::new(Default::default(), 1)); let reverse = Variable::new_from(goals.clone().map(|(_,y)| (y.clone(),(y.clone(),0))).enter(inner), Product::new(Default::default(), 1)); - forward.clone().map(|_| ()).consolidate().inspect(|x| println!("forward: {:?}", x)); - reverse.clone().map(|_| ()).consolidate().inspect(|x| println!("reverse: {:?}", x)); + forward.collection().map(|_| ()).consolidate().inspect(|x| println!("forward: {:?}", x)); + reverse.collection().map(|_| ()).consolidate().inspect(|x| println!("reverse: {:?}", x)); let goals = goals.enter(inner); // let edges = edges.enter(inner); @@ -71,8 +71,8 @@ where // This is a cyclic join, which should scare us a bunch. let reached = forward - .clone() - .join_map(reverse.clone(), |_, (src,d1), (dst,d2)| ((src.clone(), dst.clone()), *d1 + *d2)) + .collection() + .join_map(reverse.collection(), |_, (src,d1), (dst,d2)| ((src.clone(), dst.clone()), *d1 + *d2)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .semijoin(goals.clone()); @@ -88,12 +88,12 @@ where let forward_active = active.clone().map(|(x,_y)| x).distinct(); let forward_next = forward - .clone() + .collection() .map(|(med, (src, dist))| (src, (med, dist))) .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) .join_core(forward_edges, |_med, (src, dist), next| Some((next.clone(), (src.clone(), *dist+1)))) - .concat(forward.clone()) + .concat(forward.collection()) .map(|(next, (src, dist))| ((next, src), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); @@ -106,12 +106,12 @@ where let reverse_active = active.map(|(_x,y)| y).distinct(); let reverse_next = reverse - .clone() + .collection() .map(|(med, (rev, dist))| (rev, (med, dist))) .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) .join_core(reverse_edges, |_med, (rev, dist), next| Some((next.clone(), (rev.clone(), *dist+1)))) - .concat(reverse.clone()) + .concat(reverse.collection()) .map(|(next, (rev, dist))| ((next, rev), dist)) .reduce(|_key, s, t| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 2eb732bd5..eaff45322 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -91,7 +91,7 @@ where let labels = proposals - .clone() + .collection() .concat(nodes) .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine<_,_,_,_>>("Propagate", |_, s, t| t.push((s[0].0.clone(), R::from(1_i8)))); diff --git a/differential-dataflow/src/operators/iterate.rs b/differential-dataflow/src/operators/iterate.rs index 7c8cff4f9..cc2c8471f 100644 --- a/differential-dataflow/src/operators/iterate.rs +++ b/differential-dataflow/src/operators/iterate.rs @@ -31,7 +31,6 @@ //! it multiple times. use std::fmt::Debug; -use std::ops::Deref; use timely::Container; use timely::progress::Timestamp; @@ -92,8 +91,7 @@ impl, D: Ord+Data+Debug, R: Abelian+'static> Iterat // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let variable = Variable::new_from(self.enter(subgraph), Product::new(Default::default(), 1)); - let inner_scope = variable.scope(); - let result = logic(inner_scope, variable.clone()); + let result = logic(subgraph.clone(), variable.collection()); variable.set(result.clone()); result.leave() }) @@ -113,7 +111,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter // diffs produced; `result` is post-consolidation, and means fewer // records are yielded out of the loop. let variable = SemigroupVariable::new(subgraph, Product::new(Default::default(), 1)); - let result = logic(subgraph.clone(), variable.clone()); + let result = logic(subgraph.clone(), variable.collection()); variable.set(result.clone()); result.leave() } @@ -145,7 +143,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter /// scope.iterative::(|nested| { /// let summary = Product::new(Default::default(), 1); /// let variable = Variable::new_from(numbers.enter(nested), summary); -/// let result = variable.clone() +/// let result = variable.collection() /// .map(|x| if x % 2 == 0 { x/2 } else { x }) /// .consolidate(); /// variable.set(result) @@ -167,7 +165,7 @@ where /// A `Variable` specialized to a vector container of update triples (data, time, diff). pub type VecVariable = Variable::Timestamp, R)>>; -impl Variable +impl Variable where G: Scope, C: crate::collection::containers::Negate + crate::collection::containers::ResultsIn<::Summary>, @@ -183,7 +181,7 @@ where } /// Creates a new `Variable` from a supplied `source` stream. - pub fn new_from(source: Collection, step: ::Summary) -> Self { + pub fn new_from(source: Collection, step: ::Summary) -> Self where C: Clone { let (feedback, updates) = source.inner.scope().feedback(step.clone()); let collection = Collection::::new(updates).concat(source.clone()); Variable { collection, feedback, source: Some(source), step } @@ -193,12 +191,11 @@ where /// /// This method binds the `Variable` to be equal to the supplied collection, /// which may be recursively defined in terms of the variable itself. - pub fn set(self, result: Collection) -> Collection where C: Clone { - let mut in_result = result.clone(); - if let Some(source) = self.source.clone() { - in_result = in_result.concat(source.negate()); + pub fn set(mut self, mut result: Collection) -> Collection where C: Clone { + if let Some(source) = self.source.take() { + result = result.concat(source.negate()); } - self.set_concat(in_result) + self.set_concat(result) } /// Set the definition of the `Variable` to a collection concatenated to `self`. @@ -219,13 +216,11 @@ where self.collection } -} -impl, C: Container> Deref for Variable { - type Target = Collection; - fn deref(&self) -> &Self::Target { - &self.collection - } + /// Acquires a copy of the collection the variable presents as. + /// + /// This is the intended way to use the contents of the `Variable`. + pub fn collection(&self) -> Collection where C: Clone { self.collection.clone() } } /// A recursively defined collection that only "grows". @@ -266,11 +261,9 @@ where self.collection } -} -impl Deref for SemigroupVariable where G::Timestamp: Lattice { - type Target = Collection; - fn deref(&self) -> &Self::Target { - &self.collection - } + /// Acquires a copy of the collection the variable presents as. + /// + /// This is the intended way to use the contents of the `SemigroupVariable`. + pub fn collection(&self) -> Collection where C: Clone { self.collection.clone() } } diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index 20b42d67e..aa79a6d6e 100644 --- a/differential-dataflow/src/operators/join.rs +++ b/differential-dataflow/src/operators/join.rs @@ -79,11 +79,12 @@ where let mut trace1 = arranged1.trace.clone(); let mut trace2 = arranged2.trace.clone(); - arranged1.stream.clone().binary_frontier(arranged2.stream.clone(), Pipeline, Pipeline, "Join", move |capability, info| { + let scope = arranged1.stream.scope(); + arranged1.stream.binary_frontier(arranged2.stream, Pipeline, Pipeline, "Join", move |capability, info| { // Acquire an activator to reschedule the operator when it has unfinished work. use timely::scheduling::Activator; - let activations = arranged1.stream.scope().activations().clone(); + let activations = scope.activations().clone(); let activator = Activator::new(info.address, activations); // Our initial invariants are that for each trace, physical compaction is less or equal the trace's upper bound. diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index 33e640def..f0302989f 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -655,7 +655,7 @@ mod history_replay { // self.output_buffer.push(((*value).clone(), -diff)); // } // else { - // self.temporary.push(next_time.join(time)); + // self.temporary.push(next_time.join(&time)); // } // } diff --git a/mdbook/src/chapter_2/chapter_2_7.md b/mdbook/src/chapter_2/chapter_2_7.md index 1eb240569..afa7f8437 100644 --- a/mdbook/src/chapter_2/chapter_2_7.md +++ b/mdbook/src/chapter_2/chapter_2_7.md @@ -93,7 +93,7 @@ As an example, the implementation of the `iterate` operator looks something like # fn logic<'a, G: Scope>(variable: &VecVariable, (u64, u64), isize>) -> VecCollection, (u64, u64)> # where G::Timestamp: Lattice # { -# (*variable).clone() +# (*variable).collection() # } # fn example<'a, G: Scope>(collection: VecCollection) //, logic: impl Fn(&VecVariable, (u64, u64), isize>) -> VecCollection, (u64, u64)>) # where G::Timestamp: Lattice From 225df6c809cb3fd67d2c7d105e26103c6b03e202 Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 24 Feb 2026 13:39:32 -0500 Subject: [PATCH 5/6] Update MSRV to 1.86 --- .github/workflows/test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 936f5a1cd..e557a1f51 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -15,7 +15,7 @@ jobs: # - windows toolchain: - stable - - 1.79 + - 1.86 name: cargo test on ${{ matrix.os }}, rust ${{ matrix.toolchain }} runs-on: ${{ matrix.os }}-latest steps: From f7ea77565d249fdbaf8d95cca1a4aacddcbb4b6f Mon Sep 17 00:00:00 2001 From: Frank McSherry Date: Tue, 24 Feb 2026 14:49:41 -0500 Subject: [PATCH 6/6] Remove all instances of StreamCore and StreamVec --- differential-dataflow/src/capture.rs | 12 ++++++------ differential-dataflow/src/collection.rs | 9 ++++----- .../src/operators/arrange/arrangement.rs | 14 ++++++-------- .../src/operators/arrange/upsert.rs | 4 ++-- differential-dataflow/src/operators/join.rs | 5 ++--- 5 files changed, 20 insertions(+), 24 deletions(-) diff --git a/differential-dataflow/src/capture.rs b/differential-dataflow/src/capture.rs index ffa3a5a28..74cced0ca 100644 --- a/differential-dataflow/src/capture.rs +++ b/differential-dataflow/src/capture.rs @@ -227,7 +227,7 @@ pub mod source { use std::rc::Rc; use std::marker::{Send, Sync}; use std::sync::Arc; - use timely::dataflow::{Scope, StreamVec, operators::{Capability, CapabilitySet}}; + use timely::dataflow::{Scope, Stream, operators::{Capability, CapabilitySet}}; use timely::dataflow::operators::generic::OutputBuilder; use timely::progress::Timestamp; use timely::scheduling::SyncActivator; @@ -253,7 +253,7 @@ pub mod source { pub fn build( scope: G, source_builder: B, - ) -> (Box, StreamVec) + ) -> (Box, Stream>) where G: Scope, B: FnOnce(SyncActivator) -> I, @@ -560,7 +560,7 @@ pub mod sink { use timely::order::PartialOrder; use timely::progress::{Antichain, ChangeBatch, Timestamp}; - use timely::dataflow::{Scope, StreamVec}; + use timely::dataflow::{Scope, Stream}; use timely::dataflow::channels::pact::{Exchange, Pipeline}; use timely::dataflow::operators::generic::{builder_rc::OperatorBuilder, OutputBuilder}; @@ -574,7 +574,7 @@ pub mod sink { /// performed before calling the method, the recorded output may not be correctly /// reconstructed by readers. pub fn build( - stream: StreamVec, + stream: Stream>, sink_hash: u64, updates_sink: Weak>, progress_sink: Weak>, @@ -742,7 +742,7 @@ pub mod sink { // use crate::lattice::Lattice; // /// Creates a Kafka source from supplied configuration information. -// pub fn create_source(scope: G, addr: &str, topic: &str, group: &str) -> (Box, StreamVec) +// pub fn create_source(scope: G, addr: &str, topic: &str, group: &str) -> (Box, Stream>) // where // G: Scope, // D: ExchangeData + Hash + for<'a> serde::Deserialize<'a>, @@ -757,7 +757,7 @@ pub mod sink { // }) // } -// pub fn create_sink(stream: &StreamVec, addr: &str, topic: &str) -> Box +// pub fn create_sink(stream: &Stream>, addr: &str, topic: &str) -> Box // where // G: Scope, // D: ExchangeData + Hash + Serialize + for<'a> Deserialize<'a>, diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index 6799d4d4d..6b38f834a 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -11,9 +11,8 @@ use timely::Container; use timely::progress::Timestamp; use timely::dataflow::scopes::Child; -use timely::dataflow::Scope; +use timely::dataflow::{Scope, Stream}; use timely::dataflow::operators::*; -use timely::dataflow::Stream as StreamCore; use crate::difference::Abelian; @@ -32,7 +31,7 @@ pub struct Collection { /// The timestamp in the data is required to always be at least the timestamp _of_ the data, in /// the timely-dataflow sense. If this invariant is not upheld, differential operators may behave /// unexpectedly. - pub inner: StreamCore, + pub inner: Stream, } impl Collection { @@ -45,7 +44,7 @@ impl Collection { /// /// This stream should satisfy the timestamp invariant as documented on [Collection]; this /// method does not check it. - pub fn new(stream: StreamCore) -> Self { Self { inner: stream } } + pub fn new(stream: Stream) -> Self { Self { inner: stream } } } impl Collection { /// Creates a new collection accumulating the contents of the two collections. @@ -1248,7 +1247,7 @@ pub trait AsCollection { fn as_collection(self) -> Collection; } -impl AsCollection for StreamCore { +impl AsCollection for Stream { /// Converts the type to a differential dataflow collection. /// /// By calling this method, you guarantee that the timestamp invariant (as documented on diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index 87517a659..5aaf8bb33 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -19,9 +19,7 @@ use timely::dataflow::operators::{Enter, vec::Map}; use timely::order::PartialOrder; -use timely::dataflow::Stream as StreamCore; -use timely::dataflow::StreamVec as Stream; -use timely::dataflow::Scope; +use timely::dataflow::{Scope, Stream}; use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::{ParallelizationContract, Pipeline}; use timely::progress::Timestamp; @@ -54,7 +52,7 @@ where /// This stream contains the same batches of updates the trace itself accepts, so there should /// be no additional overhead to receiving these records. The batches can be navigated just as /// the batches in the trace, by key and by value. - pub stream: Stream, + pub stream: Stream>, /// A shared trace, updated by the `Arrange` operator and readable by others. pub trace: Tr, // TODO : We might have an `Option>` here, which `as_collection` sets and @@ -134,7 +132,7 @@ where /// Flattens the stream into a `Collection`. /// - /// The underlying `Stream>` is a much more efficient way to access the data, + /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. pub fn as_collection(self, mut logic: L) -> VecCollection @@ -146,7 +144,7 @@ where /// Flattens the stream into a `Collection`. /// - /// The underlying `Stream>` is a much more efficient way to access the data, + /// The underlying `Stream>>` is a much more efficient way to access the data, /// and this method should only be used when the data need to be transformed or exchanged, rather than /// supplied as arguments to an operator using the same key-value structure. pub fn as_vecs(self) -> VecCollection @@ -176,7 +174,7 @@ where /// /// This method exists for streams of batches without the corresponding arrangement. /// If you have the arrangement, its `flat_map_ref` method is equivalent to this. - pub fn flat_map_batches(stream: Stream, mut logic: L) -> VecCollection + pub fn flat_map_batches(stream: Stream>, mut logic: L) -> VecCollection where I: IntoIterator, L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, @@ -339,7 +337,7 @@ where /// This operator arranges a stream of values into a shared trace, whose contents it maintains. /// It uses the supplied parallelization contract to distribute the data, which does not need to /// be consistently by key (though this is the most common). -pub fn arrange_core(stream: StreamCore, pact: P, name: &str) -> Arranged> +pub fn arrange_core(stream: Stream, pact: P, name: &str) -> Arranged> where G: Scope, P: ParallelizationContract, diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index 54f6e980d..e9dbe9cdb 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -101,7 +101,7 @@ use std::collections::{BinaryHeap, BTreeMap}; use timely::order::{PartialOrder, TotalOrder}; -use timely::dataflow::{Scope, StreamVec as Stream}; +use timely::dataflow::{Scope, Stream}; use timely::dataflow::operators::generic::Operator; use timely::dataflow::channels::pact::Exchange; use timely::progress::Timestamp; @@ -128,7 +128,7 @@ use super::TraceAgent; /// understand what a "sequence" of upserts would mean for partially ordered /// timestamps. pub fn arrange_from_upsert( - stream: Stream, G::Timestamp)>, + stream: Stream, G::Timestamp)>>, name: &str, ) -> Arranged> where diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index aa79a6d6e..dae4d31b3 100644 --- a/differential-dataflow/src/operators/join.rs +++ b/differential-dataflow/src/operators/join.rs @@ -9,8 +9,7 @@ use timely::{Accountable, ContainerBuilder}; use timely::container::PushInto; use timely::order::PartialOrder; use timely::progress::Timestamp; -use timely::dataflow::Stream as StreamCore; -use timely::dataflow::Scope; +use timely::dataflow::{Scope, Stream}; use timely::dataflow::operators::generic::{Operator, OutputBuilderSession, Session}; use timely::dataflow::channels::pact::Pipeline; use timely::dataflow::operators::Capability; @@ -67,7 +66,7 @@ impl, D> PushInto for EffortBuilder { /// The "correctness" of this method depends heavily on the behavior of the supplied `result` function. /// /// [`AsCollection`]: crate::collection::AsCollection -pub fn join_traces(arranged1: Arranged, arranged2: Arranged, mut result: L) -> StreamCore +pub fn join_traces(arranged1: Arranged, arranged2: Arranged, mut result: L) -> Stream where G: Scope, T1: TraceReader+Clone+'static,