diff --git a/Cargo.toml b/Cargo.toml index e0b811ad9..809919f54 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -4,7 +4,7 @@ members = [ # "advent_of_code_2017", "dogsdogsdogs", "experiments", - "interactive", + # "interactive", # TODO: Value enum is recursive, needs manual Columnar impl "server", "server/dataflows/degr_dist", "server/dataflows/neighborhood", diff --git a/differential-dataflow/examples/interpreted.rs b/differential-dataflow/examples/interpreted.rs index 4c6e3429c..c4bf521d7 100644 --- a/differential-dataflow/examples/interpreted.rs +++ b/differential-dataflow/examples/interpreted.rs @@ -39,7 +39,7 @@ fn main() { fn interpret(edges: VecCollection, relations: &[(usize, usize)]) -> VecCollection> where - G: Scope, + G: Scope, { // arrange the edge relation three ways. @@ -65,7 +65,7 @@ where // Both variables are bound, so this is a semijoin. results .map(move |vec| ((vec[src], vec[dst]), vec)) - .join_core(as_self.clone(), |_key, vec, &()| Some(vec.clone())) + .join_core(as_self.clone(), |_key, vec, _| Some(columnar::Columnar::into_owned(vec))) } (true, false) => { // Only `src` is bound, so we must use `forward` to propose `dst`. @@ -73,7 +73,7 @@ where results .map(move |vec| (vec[src], vec)) .join_core(forward.clone(), move |_src_val, vec, &dst_val| { - let mut temp = vec.clone(); + let mut temp: Vec = columnar::Columnar::into_owned(vec); while temp.len() <= dst { temp.push(0); } temp[dst] = dst_val; Some(temp) @@ -85,7 +85,7 @@ where results .map(move |vec| (vec[dst], vec)) .join_core(reverse.clone(), move |_dst_val, vec, &src_val| { - let mut temp = vec.clone(); + let mut temp: Vec = columnar::Columnar::into_owned(vec); while temp.len() <= src { temp.push(0); } temp[src] = src_val; Some(temp) diff --git a/differential-dataflow/src/algorithms/graphs/bfs.rs b/differential-dataflow/src/algorithms/graphs/bfs.rs index 1513622bd..ead58a232 100644 --- a/differential-dataflow/src/algorithms/graphs/bfs.rs +++ b/differential-dataflow/src/algorithms/graphs/bfs.rs @@ -11,7 +11,7 @@ use crate::lattice::Lattice; /// Returns pairs (node, dist) indicating distance of each node from a root. pub fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + G: Scope, N: ExchangeData+Hash, { let edges = edges.arrange_by_key(); @@ -26,7 +26,7 @@ pub fn bfs_arranged(edges: Arranged, roots: VecCollection where G: Scope, N: ExchangeData+Hash, - Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=isize>+Clone+'static, + Tr: TraceReader+Clone+'static, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); @@ -37,8 +37,8 @@ where let edges = edges.enter(&scope); let nodes = nodes.enter(&scope); - inner.join_core(edges, |_k,l,d| Some((d.clone(), l+1))) + inner.join_core(edges, |_k,l,d| Some((::into_owned(d), l+1))) .concat(nodes) - .reduce(|_, s, t| t.push((*s[0].0, 1))) + .reduce(|_, s, t| t.push((columnar::Columnar::into_owned(s[0].0), 1))) }) } diff --git a/differential-dataflow/src/algorithms/graphs/bijkstra.rs b/differential-dataflow/src/algorithms/graphs/bijkstra.rs index 9c61716ef..c57a2c38a 100644 --- a/differential-dataflow/src/algorithms/graphs/bijkstra.rs +++ b/differential-dataflow/src/algorithms/graphs/bijkstra.rs @@ -21,7 +21,7 @@ use crate::operators::iterate::Variable; /// could be good insurance here. pub fn bidijkstra(edges: VecCollection, goals: VecCollection) -> VecCollection where - G: Scope, + G: Scope, N: ExchangeData+Hash, { let forward = edges.clone().arrange_by_key(); @@ -41,7 +41,7 @@ pub fn bidijkstra_arranged( where G: Scope, N: ExchangeData+Hash, - Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Diff=isize>+Clone+'static, + Tr: TraceReader+Clone+'static, { forward .stream @@ -72,8 +72,8 @@ where let reached = forward .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))) + .join_map(reverse.clone(), |_, &(ref src, d1), &(ref dst, d2)| ((src.clone(), dst.clone()), d1 + d2)) + .reduce(|_key, s, t| t.push((::into_owned(s[0].0), 1))) .semijoin(goals.clone()); let active = @@ -92,10 +92,13 @@ where .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)))) + .join_core(forward_edges, |_med, srcdist, next| { + let (src, dist): (N, u32) = columnar::Columnar::into_owned(srcdist); + Some((::into_owned(next), (src, dist+1))) + }) .concat(forward) .map(|(next, (src, dist))| ((next, src), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s: &[(columnar::Ref<'_, u32>, isize)], t: &mut Vec<(u32, isize)>| t.push((::into_owned(s[0].0), 1))) .map(|((next, src), dist)| (next, (src, dist))); forward_next.clone().map(|_| ()).consolidate().inspect(|x| println!("forward_next: {:?}", x)); @@ -110,10 +113,13 @@ where .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)))) + .join_core(reverse_edges, |_med, revdist, next| { + let (rev, dist): (N, u32) = columnar::Columnar::into_owned(revdist); + Some((::into_owned(next), (rev, dist+1))) + }) .concat(reverse) .map(|(next, (rev, dist))| ((next, rev), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s: &[(columnar::Ref<'_, u32>, isize)], t: &mut Vec<(u32, isize)>| t.push((::into_owned(s[0].0), 1))) .map(|((next,rev), dist)| (next, (rev, dist))); reverse_next.clone().map(|_| ()).consolidate().inspect(|x| println!("reverse_next: {:?}", x)); diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 3d5d8c65c..9d8adb064 100644 --- a/differential-dataflow/src/algorithms/graphs/propagate.rs +++ b/differential-dataflow/src/algorithms/graphs/propagate.rs @@ -15,7 +15,7 @@ use crate::difference::{Abelian, Multiply}; /// method to limit the introduction of labels. pub fn propagate(edges: VecCollection, nodes: VecCollection) -> VecCollection where - G: Scope, + G: Scope, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, @@ -32,7 +32,7 @@ where /// method to limit the introduction of labels. pub fn propagate_at(edges: VecCollection, nodes: VecCollection, logic: F) -> VecCollection where - G: Scope, + G: Scope, N: ExchangeData+Hash, R: ExchangeData+Abelian, R: Multiply, @@ -59,7 +59,7 @@ where R: Multiply, R: From, L: ExchangeData, - Tr: for<'a> TraceReader=&'a N, Val<'a>=&'a N, Time:Hash, Diff=R>+Clone+'static, + Tr: TraceReader+Clone+'static, F: Fn(&L)->u64+Clone+'static, { // Morally the code performs the following iterative computation. However, in the interest of a simplified @@ -92,17 +92,17 @@ where let labels = proposals .concat(nodes) - .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine<_,_,_,_>>("Propagate", |_, s, t| t.push((s[0].0.clone(), R::from(1_i8)))); + .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine<_,_,_,_>>("Propagate", |_, s, t| t.push((::into_owned(s[0].0), R::from(1_i8)))); let propagate: VecCollection<_, (N, L), R> = labels .clone() - .join_core(edges, |_k, l: &L, d| Some((d.clone(), l.clone()))); + .join_core(edges, |_k, l, d| Some((::into_owned(d), ::into_owned(l)))); proposals_bind.set(propagate); labels - .as_collection(|k,v| (k.clone(), v.clone())) + .as_collection(|k,v| (::into_owned(k), ::into_owned(v))) .leave() }) } diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index f7bca525f..072bfaa1e 100644 --- a/differential-dataflow/src/algorithms/graphs/scc.rs +++ b/differential-dataflow/src/algorithms/graphs/scc.rs @@ -15,7 +15,7 @@ use super::propagate::propagate; /// Iteratively removes nodes with no in-edges. pub fn trim(graph: VecCollection) -> VecCollection where - G: Scope, + G: Scope, N: ExchangeData + Hash, R: ExchangeData + Abelian, R: Multiply, @@ -35,7 +35,7 @@ where /// Returns the subset of edges in the same strongly connected component. pub fn strongly_connected(graph: VecCollection) -> VecCollection where - G: Scope, + G: Scope, N: ExchangeData + Hash, R: ExchangeData + Abelian, R: Multiply, @@ -51,7 +51,7 @@ where fn trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection where - G: Scope, + G: Scope, N: ExchangeData + Hash, R: ExchangeData + Abelian, R: Multiply, @@ -66,7 +66,7 @@ where let labels = propagate(cycle, nodes); 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()))) + .join_map(labels, |e2,&(ref e1, ref 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 354ad32c2..0d88464d8 100644 --- a/differential-dataflow/src/algorithms/graphs/sequential.rs +++ b/differential-dataflow/src/algorithms/graphs/sequential.rs @@ -11,7 +11,7 @@ use crate::hashable::Hashable; fn _color(edges: VecCollection) -> VecCollection)> where - G: Scope, + G: Scope, N: ExchangeData+Hash, { // need some bogus initial values. @@ -45,7 +45,7 @@ pub fn sequence( edges: VecCollection, logic: F) -> VecCollection)> where - G: Scope, + G: Scope, N: ExchangeData+Hashable, V: ExchangeData, F: Fn(&N, &[(&V, isize)])->V+'static @@ -66,12 +66,12 @@ where 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.clone().filter(|x| x.1.is_none()).map(|x| x.0).distinct(); + let incomplete = new_messages.clone().filter(|x: &(N, Option)| x.1.is_none()).map(|x: (N, Option)| 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.clone()); @@ -82,7 +82,10 @@ where messages // .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))) + .reduce(move |k, vs, t| { + let vs_ref: Vec<(&V, isize)> = vs.iter().map(|(v, r)| (*v, *r)).collect(); + t.push((Some(logic(k, &vs_ref)), 1)) + }) .concat(incomplete.map(|x| (x, None))) }) } diff --git a/differential-dataflow/src/algorithms/identifiers.rs b/differential-dataflow/src/algorithms/identifiers.rs index 267e4c453..d23a1c43b 100644 --- a/differential-dataflow/src/algorithms/identifiers.rs +++ b/differential-dataflow/src/algorithms/identifiers.rs @@ -32,7 +32,7 @@ pub trait Identifiers { impl Identifiers for VecCollection where - G: Scope, + G: Scope, D: ExchangeData + ::std::hash::Hash, R: ExchangeData + Abelian, { @@ -64,19 +64,19 @@ where .map(|pair| (pair.hashed(), pair)) .reduce(|_hash, input, output| { // keep round-positive records as changes. - let ((round, record), count) = &input[0]; - if *round > 0 { + let (&(round, ref record), ref count) = input[0]; + if round > 0 { let mut neg_count = count.clone(); neg_count.negate(); - output.push(((0, record.clone()), neg_count)); - output.push(((*round, record.clone()), count.clone())); + output.push(((0i32, record.clone()), neg_count)); + output.push(((round, record.clone()), count.clone())); } // if any losers, increment their rounds. - for ((round, record), count) in input[1..].iter() { + for (&(round, ref record), ref count) in input[1..].iter() { let mut neg_count = count.clone(); neg_count.negate(); - output.push(((0, record.clone()), neg_count)); - output.push(((*round+1, record.clone()), count.clone())); + output.push(((0i32, record.clone()), neg_count)); + output.push(((round+1, record.clone()), count.clone())); } }) .map(|(_hash, pair)| pair) @@ -119,13 +119,13 @@ mod tests { // keep round-positive records as changes. let ((round, record), count) = &input[0]; if *round > 0 { - output.push(((0, record.clone()), -*count)); - output.push(((*round, record.clone()), *count)); + output.push(((0, columnar::Columnar::into_owned(record.clone())), -*count)); + output.push(((*round, columnar::Columnar::into_owned(record.clone())), *count)); } // if any losers, increment their rounds. for ((round, record), count) in input[1..].iter() { - output.push(((0, record.clone()), -*count)); - output.push(((*round+1, record.clone()), *count)); + output.push(((0, columnar::Columnar::into_owned(record.clone())), -*count)); + output.push(((*round+1, columnar::Columnar::into_owned(record.clone())), *count)); } }) .inspect(|x| println!("{:?}", x)) diff --git a/differential-dataflow/src/algorithms/prefix_sum.rs b/differential-dataflow/src/algorithms/prefix_sum.rs index f76f59d9c..a83a52e99 100644 --- a/differential-dataflow/src/algorithms/prefix_sum.rs +++ b/differential-dataflow/src/algorithms/prefix_sum.rs @@ -21,7 +21,7 @@ pub trait PrefixSum { impl PrefixSum for VecCollection where - G: Scope, + G: Scope, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, { @@ -42,7 +42,7 @@ where /// Accumulate data in `collection` into all powers-of-two intervals containing them. pub fn aggregate(collection: VecCollection, combine: F) -> VecCollection where - G: Scope, + G: Scope, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, F: Fn(&K,&D,&D)->D + 'static, @@ -64,8 +64,10 @@ where .filter(|&((_pos, log, _), _)| log < 64) .map(|((pos, log, key), data)| ((pos >> 1, log + 1, key), (pos, data))) .reduce(move |&(_pos, _log, ref key), input, output| { - let mut result = (input[0].0).1.clone(); - if input.len() > 1 { result = combine(key, &result, &(input[1].0).1); } + let mut result: D = (input[0].0).1.clone(); + if input.len() > 1 { + result = combine(key, &result, &(input[1].0).1); + } output.push((result, 1)); }) .concat(unit_ranges) @@ -79,7 +81,7 @@ pub fn broadcast( zero: D, combine: F) -> VecCollection where - G: Scope, + G: Scope, K: ExchangeData + ::std::hash::Hash, D: ExchangeData + ::std::hash::Hash, F: Fn(&K,&D,&D)->D + 'static, @@ -146,8 +148,9 @@ where used_ranges .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))) + .join_map(states, move |&(pos, ref key), &(log, ref data), state| { + ((pos + (1 << log), key.clone()), combine(key, state, data)) + }) .concat(init_states) .distinct() }) diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index c13227f2c..5a41d5248 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -660,7 +660,7 @@ pub mod vec { where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable + Semigroup, - G::Timestamp: Lattice+Ord, + G::Timestamp: Lattice+Ord+crate::Data, { self.consolidate() .inspect(|x| panic!("Assertion failed: non-empty collection: {:?}", x)); @@ -696,7 +696,7 @@ pub mod vec { where D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Hashable, - G::Timestamp: Lattice+Ord, + G::Timestamp: Lattice+Ord+crate::Data, { self.negate() .concat(other) @@ -709,7 +709,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, K: crate::ExchangeData+Hashable, V: crate::ExchangeData, R: crate::ExchangeData+Semigroup, @@ -741,19 +741,27 @@ 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, mut logic: L) -> Collection where L: FnMut(&K, &[(&V, R)], &mut Vec<(V2, R2)>)+'static { use crate::trace::implementations::{ValBuilder, ValSpine}; + // Wrap the user's &K/&V closure to work with the Ref-based reduce_abelian. + let mut owned_input: Vec<(V, R)> = Vec::new(); self.arrange_by_key_named(&format!("Arrange: {}", name)) - .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine>(name, logic) - .as_collection(|k,v| (k.clone(), v.clone())) + .reduce_abelian::<_,ValBuilder<_,_,_,_>,ValSpine>(name, move |key, input, output| { + let key = columnar::Columnar::into_owned(key); + owned_input.clear(); + owned_input.extend(input.iter().map(|(v, r)| (columnar::Columnar::into_owned(*v), r.clone()))); + let ref_input: Vec<(&V, R)> = owned_input.iter().map(|(v, r)| (v, r.clone())).collect(); + logic(&key, &ref_input, output); + }) + .as_collection(|k,v| (columnar::Columnar::into_owned(k), columnar::Columnar::into_owned(v))) } /// Applies `reduce` to arranged data, and returns an arrangement of output data. @@ -782,9 +790,9 @@ pub mod vec { /// ``` 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, + T2: Trace+'static, Bu: Builder, Output = T2::Batch>, - L: FnMut(&K, &[(&V, R)], &mut Vec<(V, T2::Diff)>)+'static, + L: FnMut(columnar::Ref<'_, K>, &[(columnar::Ref<'_, V>, R)], &mut Vec<(V, T2::Diff)>)+'static, { self.reduce_core::<_,Bu,T2>(name, move |key, input, output, change| { if !input.is_empty() { logic(key, input, change); } @@ -801,9 +809,9 @@ pub mod vec { pub fn reduce_core(self, name: &str, logic: L) -> Arranged> where V: Clone+'static, - T2: for<'a> Trace=&'a K, KeyOwn = K, ValOwn = V, Time=G::Timestamp>+'static, + T2: Trace+'static, Bu: Builder, Output = T2::Batch>, - L: FnMut(&K, &[(&V, R)], &mut Vec<(V,T2::Diff)>, &mut Vec<(V, T2::Diff)>)+'static, + L: FnMut(columnar::Ref<'_, K>, &[(columnar::Ref<'_, V>, R)], &mut Vec<(V,T2::Diff)>, &mut Vec<(V, T2::Diff)>)+'static, { self.arrange_by_key_named(&format!("Arrange: {}", name)) .reduce_core::<_,Bu,_>(name, logic) @@ -812,7 +820,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, K: crate::ExchangeData+Hashable, R1: crate::ExchangeData+Semigroup { @@ -840,7 +848,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)) } @@ -862,24 +870,24 @@ pub mod vec { /// .threshold(|_,c| c % 2); /// }); /// ``` - pub fn thresholdR2+'static>(self, thresh: F) -> Collection { + pub fn threshold, &R1)->R2+'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_named, &R1)->R2+'static>(self, name: &str, mut thresh: F) -> Collection { use crate::trace::implementations::{KeyBuilder, KeySpine}; self.arrange_by_self_named(&format!("Arrange: {}", name)) .reduce_abelian::<_,KeyBuilder,KeySpine>(name, move |k,s,t| t.push(((), thresh(k, &s[0].1)))) - .as_collection(|k,_| k.clone()) + .as_collection(|k,_| columnar::Columnar::into_owned(k)) } } impl Collection where - G: Scope, + G: Scope, K: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup { @@ -905,18 +913,18 @@ pub mod vec { /// 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>(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)))) - .as_collection(|k,c| (k.clone(), c.clone())) + .as_collection(|k,c| (columnar::Columnar::into_owned(k), columnar::Columnar::into_owned(c))) } } /// Methods which require data be arrangeable. impl Collection where - G: Scope, + G: Scope, D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup, { @@ -943,7 +951,7 @@ pub mod vec { /// ``` pub fn consolidate(self) -> Self { use crate::trace::implementations::{KeyBatcher, KeyBuilder, KeySpine}; - self.consolidate_named::,KeyBuilder<_,_,_>, KeySpine<_,_,_>,_>("Consolidate", |key,&()| key.clone()) + self.consolidate_named::,KeyBuilder<_,_,_>, KeySpine<_,_,_>,_>("Consolidate", |key,_| columnar::Columnar::into_owned(key)) } /// As `consolidate` but with the ability to name the operator, specify the trace type, @@ -953,7 +961,7 @@ pub mod vec { Ba: crate::trace::Batcher, Time=G::Timestamp> + 'static, Tr: for<'a> crate::trace::Trace+'static, Bu: crate::trace::Builder, - F: Fn(Tr::Key<'_>, Tr::Val<'_>) -> D + 'static, + F: Fn(columnar::Ref<'_, Tr::Key>, columnar::Ref<'_, Tr::Val>) -> D + 'static, { use crate::operators::arrange::arrangement::Arrange; self.map(|k| (k, ())) @@ -1029,7 +1037,7 @@ pub mod vec { impl Arrange> for Collection where - G: Scope, + G: Scope, { fn arrange_named(self, name: &str) -> Arranged> where @@ -1045,7 +1053,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, { /// Arranges a collection of `(Key, Val)` records by `Key`. /// @@ -1064,7 +1072,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, { /// Arranges a collection of `Key` records by `Key`. /// @@ -1084,7 +1092,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, K: crate::ExchangeData+Hashable, V: crate::ExchangeData, R: crate::ExchangeData+Semigroup, @@ -1139,7 +1147,12 @@ pub mod vec { 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| { + let k = columnar::Columnar::into_owned(k); + let v1 = columnar::Columnar::into_owned(v1); + let v2 = columnar::Columnar::into_owned(v2); + Some(logic(&k, &v1, &v2)) + }) } /// Matches pairs `(key, val)` and `key` based on `key`, producing the former with frequencies multiplied. @@ -1167,7 +1180,7 @@ pub mod vec { 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((columnar::Columnar::into_owned(k), columnar::Columnar::into_owned(v)))) } /// Subtracts the semijoin with `other` from `self`. @@ -1230,10 +1243,10 @@ pub mod vec { /// ``` 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, + Tr2: crate::trace::TraceReader+Clone+'static, R: Multiply, I: IntoIterator, - L: FnMut(&K,&V,Tr2::Val<'_>)->I+'static, + L: FnMut(columnar::Ref<'_, K>,columnar::Ref<'_, V>,columnar::Ref<'_, Tr2::Val>)->I+'static, { self.arrange_by_key() .join_core(stream2, result) diff --git a/differential-dataflow/src/lib.rs b/differential-dataflow/src/lib.rs index 615aa5ded..a778dd170 100644 --- a/differential-dataflow/src/lib.rs +++ b/differential-dataflow/src/lib.rs @@ -85,8 +85,8 @@ 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 : Ord + Debug + Clone + 'static { } -impl Data for T { } +pub trait Data : columnar::Columnar + Ord + Debug + Clone + 'static { } +impl + Ord + Debug + Clone + 'static> Data for T { } /// Data types exchangeable in differential dataflow. pub trait ExchangeData : timely::ExchangeData + Data { } diff --git a/differential-dataflow/src/operators/arrange/agent.rs b/differential-dataflow/src/operators/arrange/agent.rs index e1c2fde5f..df0d30552 100644 --- a/differential-dataflow/src/operators/arrange/agent.rs +++ b/differential-dataflow/src/operators/arrange/agent.rs @@ -36,13 +36,13 @@ pub struct TraceAgent { logging: Option, } -use crate::trace::implementations::WithLayout; -impl WithLayout for TraceAgent { - type Layout = Tr::Layout; -} - impl TraceReader for TraceAgent { + type Key = Tr::Key; + type Val = Tr::Val; + type Time = Tr::Time; + type Diff = Tr::Diff; + type Batch = Tr::Batch; type Storage = Tr::Storage; type Cursor = Tr::Cursor; diff --git a/differential-dataflow/src/operators/arrange/arrangement.rs b/differential-dataflow/src/operators/arrange/arrangement.rs index 5aaf8bb33..d6097a0e0 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -90,7 +90,7 @@ where pub fn enter<'a, TInner>(self, child: &Child<'a, G, TInner>) -> Arranged, TraceEnter> where - TInner: Refines+Lattice+Timestamp+Clone, + TInner: Refines+Lattice+Timestamp+Clone+crate::Data, { Arranged { stream: self.stream.enter(child).map(|bw| BatchEnter::make_from(bw)), @@ -118,8 +118,8 @@ where 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, - F: FnMut(Tr::Key<'_>, Tr::Val<'_>, Tr::TimeGat<'_>)->TInner+Clone+'static, + TInner: Refines+Lattice+Timestamp+Clone+'static+crate::Data, + F: FnMut(columnar::Ref<'_, Tr::Key>, columnar::Ref<'_, Tr::Val>, columnar::Ref<'_, Tr::Time>)->TInner+Clone+'static, P: FnMut(&TInner)->Tr::Time+Clone+'static, { let logic1 = logic.clone(); @@ -137,7 +137,7 @@ where /// supplied as arguments to an operator using the same key-value structure. pub fn as_collection(self, mut logic: L) -> VecCollection where - L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> D+'static, + L: FnMut(columnar::Ref<'_, Tr::Key>, columnar::Ref<'_, Tr::Val>) -> D+'static, { self.flat_map_ref(move |key, val| Some(logic(key,val))) } @@ -147,12 +147,12 @@ 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, + Tr::Key: crate::ExchangeData, + Tr::Val: crate::ExchangeData, { - self.flat_map_ref(move |key, val| [(Tr::owned_key(key), Tr::owned_val(val))]) + self.flat_map_ref(move |key, val| [(::into_owned(key), ::into_owned(val))]) } /// Extracts elements from an arrangement as a collection. @@ -162,7 +162,7 @@ where pub fn flat_map_ref(self, logic: L) -> VecCollection where I: IntoIterator, - L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, + L: FnMut(columnar::Ref<'_, Tr::Key>, columnar::Ref<'_, Tr::Val>) -> I+'static, { Self::flat_map_batches(self.stream, logic) } @@ -177,7 +177,7 @@ where pub fn flat_map_batches(stream: Stream>, mut logic: L) -> VecCollection where I: IntoIterator, - L: FnMut(Tr::Key<'_>, Tr::Val<'_>) -> I+'static, + L: FnMut(columnar::Ref<'_, Tr::Key>, columnar::Ref<'_, Tr::Val>) -> I+'static, { stream.unary(Pipeline, "AsCollection", move |_,_| move |input, output| { input.for_each(|time, data| { @@ -189,7 +189,7 @@ where while let Some(val) = cursor.get_val(batch) { for datum in logic(key, val) { cursor.map_times(batch, |time, diff| { - session.give((datum.clone(), Tr::owned_time(time), Tr::owned_diff(diff))); + session.give((datum.clone(), ::into_owned(time), ::into_owned(diff))); }); } cursor.step_val(batch); @@ -216,12 +216,12 @@ where /// 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> where - T2: for<'a> TraceReader=T1::Key<'a>,Time=T1::Time>+Clone+'static, + T2: TraceReader+Clone+'static, T1::Diff: Multiply, I: IntoIterator, - L: FnMut(T1::Key<'_>,T1::Val<'_>,T2::Val<'_>)->I+'static + L: FnMut(columnar::Ref<'_, T1::Key>,columnar::Ref<'_, T1::Val>,columnar::Ref<'_, T2::Val>)->I+'static { - let mut result = move |k: T1::Key<'_>, v1: T1::Val<'_>, v2: T2::Val<'_>, t: &G::Timestamp, r1: &T1::Diff, r2: &T2::Diff| { + let mut result = move |k: columnar::Ref<'_, T1::Key>, v1: columnar::Ref<'_, T1::Val>, v2: columnar::Ref<'_, T2::Val>, t: &G::Timestamp, r1: &T1::Diff, r2: &T2::Diff| { let t = t.clone(); let r = (r1.clone()).multiply(r2); result(k, v1, v2).into_iter().map(move |d| (d, t.clone(), r.clone())) @@ -251,16 +251,15 @@ where /// A direct implementation of `ReduceCore::reduce_abelian`. pub fn reduce_abelian(self, name: &str, mut logic: L) -> Arranged> where - T1: TraceReader, - T2: for<'a> Trace< - Key<'a>= T1::Key<'a>, - KeyOwn=T1::KeyOwn, - ValOwn: Data, + T1: TraceReader, + T2: Trace< + Key=T1::Key, + Val: Data, Time=T1::Time, Diff: Abelian, >+'static, - Bu: Builder>, - L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, + Bu: Builder>, + L: FnMut(columnar::Ref<'_, T1::Key>, &[(columnar::Ref<'_, T1::Val>, T1::Diff)], &mut Vec<(T2::Val, T2::Diff)>)+'static, { self.reduce_core::<_,Bu,T2>(name, move |key, input, output, change| { if !input.is_empty() { @@ -274,15 +273,14 @@ where /// A direct implementation of `ReduceCore::reduce_core`. pub fn reduce_core(self, name: &str, logic: L) -> Arranged> where - T1: TraceReader, - T2: for<'a> Trace< - Key<'a>=T1::Key<'a>, - KeyOwn=T1::KeyOwn, - ValOwn: Data, + T1: TraceReader, + T2: Trace< + Key=T1::Key, + Val: Data, Time=T1::Time, >+'static, - Bu: Builder>, - L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn, T2::Diff)>, &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, + Bu: Builder>, + L: FnMut(columnar::Ref<'_, T1::Key>, &[(columnar::Ref<'_, T1::Val>, T1::Diff)], &mut Vec<(T2::Val, T2::Diff)>, &mut Vec<(T2::Val, T2::Diff)>)+'static, { use crate::operators::reduce::reduce_trace; reduce_trace::<_,_,Bu,_,_>(self, name, logic) diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index e9dbe9cdb..51bf87605 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -113,6 +113,7 @@ use crate::trace::{Builder, Description}; use crate::trace::{self, Trace, TraceReader, Cursor}; use crate::{ExchangeData, Hashable}; +use crate::trace::implementations::Coltainer; use crate::trace::implementations::containers::BatchContainer; use super::TraceAgent; @@ -128,18 +129,18 @@ 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 G: Scope, - Tr: for<'a> Trace< - KeyOwn: ExchangeData+Hashable+std::hash::Hash, - ValOwn: ExchangeData, + Tr: Trace< + Key: ExchangeData+Hashable+std::hash::Hash, + Val: ExchangeData, Time: TotalOrder+ExchangeData, Diff=isize, >+'static, - Bu: Builder, Output = Tr::Batch>, + Bu: Builder, Output = Tr::Batch>, { let mut reader: Option> = None; @@ -148,7 +149,7 @@ where let reader = &mut reader; - let exchange = Exchange::new(move |update: &(Tr::KeyOwn,Option,G::Timestamp)| (update.0).hashed().into()); + let exchange = Exchange::new(move |update: &(Tr::Key,Option,G::Timestamp)| (update.0).hashed().into()); let scope = stream.scope(); stream.unary_frontier(exchange, name, move |_capability, info| { @@ -174,7 +175,7 @@ where let mut prev_frontier = Antichain::from_elem(::minimum()); // For stashing input upserts, ordered increasing by time (`BinaryHeap` is a max-heap). - let mut priority_queue = BinaryHeap::)>>::new(); + let mut priority_queue = BinaryHeap::)>>::new(); let mut updates = Vec::new(); move |(input, frontier), output| { @@ -234,13 +235,13 @@ where // new stuff that we add. let (mut trace_cursor, trace_storage) = reader_local.cursor(); let mut builder = Bu::new(); - let mut key_con = Tr::KeyContainer::with_capacity(1); + let mut key_con = Coltainer::::with_capacity(1); for (key, mut list) in to_process { key_con.clear(); key_con.push_own(&key); // The prior value associated with the key. - let mut prev_value: Option = None; + let mut prev_value: Option = None; // Attempt to find the key in the trace. trace_cursor.seek_key(&trace_storage, key_con.index(0)); @@ -248,11 +249,11 @@ where // Determine the prior value associated with the key. while let Some(val) = trace_cursor.get_val(&trace_storage) { let mut count = 0; - trace_cursor.map_times(&trace_storage, |_time, diff| count += Tr::owned_diff(diff)); + trace_cursor.map_times(&trace_storage, |_time, diff| count += ::into_owned(diff)); assert!(count == 0 || count == 1); if count == 1 { assert!(prev_value.is_none()); - prev_value = Some(Tr::owned_val(val)); + prev_value = Some(::into_owned(val)); } trace_cursor.step_val(&trace_storage); } diff --git a/differential-dataflow/src/operators/count.rs b/differential-dataflow/src/operators/count.rs index 713c73c86..60f263fbf 100644 --- a/differential-dataflow/src/operators/count.rs +++ b/differential-dataflow/src/operators/count.rs @@ -44,7 +44,8 @@ pub trait CountTotal, K: ExchangeDat impl CountTotal for VecCollection where - G: Scope, + G: Scope, + R: for<'a> Semigroup>, { fn count_total_core + 'static>(self) -> VecCollection { self.arrange_by_self_named("Arrange: CountTotal") @@ -55,11 +56,11 @@ where impl CountTotal for Arranged where G: Scope, - T1: for<'a> TraceReader< - Key<'a> = &'a K, - Val<'a>=&'a (), + T1: TraceReader< + Key = K, + Val = (), Time: TotalOrder, - Diff: ExchangeData+Semigroup> + Diff: ExchangeData, >+Clone+'static, K: ExchangeData, { @@ -108,23 +109,25 @@ where trace_cursor.seek_key(&trace_storage, key); if trace_cursor.get_key(&trace_storage) == Some(key) { trace_cursor.map_times(&trace_storage, |_, diff| { + let diff = ::into_owned(diff); count.as_mut().map(|c| c.plus_equals(&diff)); - if count.is_none() { count = Some(T1::owned_diff(diff)); } + if count.is_none() { count = Some(diff); } }); } batch_cursor.map_times(&batch_storage, |time, diff| { + let diff = ::into_owned(diff); if let Some(count) = count.as_ref() { if !count.is_zero() { - session.give(((key.clone(), count.clone()), T1::owned_time(time), R2::from(-1i8))); + session.give(((::into_owned(key), count.clone()), ::into_owned(time), R2::from(-1i8))); } } count.as_mut().map(|c| c.plus_equals(&diff)); - if count.is_none() { count = Some(T1::owned_diff(diff)); } + if count.is_none() { count = Some(diff); } if let Some(count) = count.as_ref() { if !count.is_zero() { - session.give(((key.clone(), count.clone()), T1::owned_time(time), R2::from(1i8))); + session.give(((::into_owned(key), count.clone()), ::into_owned(time), R2::from(1i8))); } } }); diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index dae4d31b3..6777a7f18 100644 --- a/differential-dataflow/src/operators/join.rs +++ b/differential-dataflow/src/operators/join.rs @@ -70,8 +70,8 @@ pub fn join_traces(arranged1: Arranged, arranged2: Arran where G: Scope, T1: TraceReader+Clone+'static, - T2: for<'a> TraceReader=T1::Key<'a>, Time=T1::Time>+Clone+'static, - L: FnMut(T1::Key<'_>,T1::Val<'_>,T2::Val<'_>,&G::Timestamp,&T1::Diff,&T2::Diff,&mut JoinSession>)+'static, + T2: TraceReader+Clone+'static, + L: FnMut(columnar::Ref<'_,T1::Key>,columnar::Ref<'_,T1::Val>,columnar::Ref<'_,T2::Val>,&G::Timestamp,&T1::Diff,&T2::Diff,&mut JoinSession>)+'static, CB: ContainerBuilder, { // Rename traces for symmetry from here on out. @@ -311,9 +311,9 @@ where /// dataflow system a chance to run operators that can consume and aggregate the data. struct Deferred where - T: Timestamp+Lattice+Ord, + T: Timestamp+Lattice+Ord+crate::Data, C1: Cursor, - C2: for<'a> Cursor=C1::Key<'a>, Time=T>, + C2: Cursor, { trace: C1, trace_storage: C1::Storage, @@ -326,8 +326,8 @@ where impl Deferred where C1: Cursor, - C2: for<'a> Cursor=C1::Key<'a>, Time=T>, - T: Timestamp+Lattice+Ord, + C2: Cursor, + T: Timestamp+Lattice+Ord+crate::Data, { fn new(trace: C1, trace_storage: C1::Storage, batch: C2, batch_storage: C2::Storage, capability: Capability) -> Self { Deferred { @@ -348,7 +348,7 @@ where #[inline(never)] fn work(&mut self, output: &mut OutputBuilderSession>, mut logic: L, fuel: &mut usize) where - L: for<'a> FnMut(C1::Key<'a>, C1::Val<'a>, C2::Val<'a>, &T, &C1::Diff, &C2::Diff, &mut JoinSession>), + L: for<'a> FnMut(columnar::Ref<'a, C1::Key>, columnar::Ref<'a, C1::Val>, columnar::Ref<'a, C2::Val>, &T, &C1::Diff, &C2::Diff, &mut JoinSession>), { let meet = self.capability.time(); @@ -372,11 +372,11 @@ where Ordering::Equal => { thinker.history1.edits.load(trace, trace_storage, |time| { - let mut time = C1::owned_time(time); + let mut time = ::into_owned(time); time.join_assign(meet); time }); - thinker.history2.edits.load(batch, batch_storage, |time| C2::owned_time(time)); + thinker.history2.edits.load(batch, batch_storage, |time| ::into_owned(time)); // populate `temp` with the results in the best way we know how. thinker.think(|v1,v2,t,r1,r2| { @@ -422,7 +422,7 @@ where } } - fn think,C2::Val<'a>,C1::Time,&C1::Diff,&C2::Diff)>(&mut self, mut results: F) { + fn think,columnar::Ref<'a, C2::Val>,C1::Time,&C1::Diff,&C2::Diff)>(&mut self, mut results: F) { // for reasonably sized edits, do the dead-simple thing. if self.history1.edits.len() < 10 || self.history2.edits.len() < 10 { diff --git a/differential-dataflow/src/operators/mod.rs b/differential-dataflow/src/operators/mod.rs index e511603e3..1144ac61c 100644 --- a/differential-dataflow/src/operators/mod.rs +++ b/differential-dataflow/src/operators/mod.rs @@ -20,7 +20,7 @@ use crate::trace::Cursor; /// An accumulation of (value, time, diff) updates. struct EditList<'a, C: Cursor> { - values: Vec<(C::Val<'a>, usize)>, + values: Vec<(columnar::Ref<'a, C::Val>, usize)>, edits: Vec<(C::Time, C::Diff)>, } @@ -36,11 +36,11 @@ impl<'a, C: Cursor> EditList<'a, C> { /// Loads the contents of a cursor. fn load(&mut self, cursor: &mut C, storage: &'a C::Storage, logic: L) where - L: Fn(C::TimeGat<'_>)->C::Time, + L: Fn(columnar::Ref<'_, C::Time>)->C::Time, { self.clear(); while let Some(val) = cursor.get_val(storage) { - cursor.map_times(storage, |time1, diff1| self.push(logic(time1), C::owned_diff(diff1))); + cursor.map_times(storage, |time1, diff1| self.push(logic(time1), ::into_owned(diff1))); self.seal(val); cursor.step_val(storage); } @@ -60,14 +60,14 @@ impl<'a, C: Cursor> EditList<'a, C> { } /// Associates all edits pushed since the previous `seal_value` call with `value`. #[inline] - fn seal(&mut self, value: C::Val<'a>) { + fn seal(&mut self, value: columnar::Ref<'a, C::Val>) { let prev = self.values.last().map(|x| x.1).unwrap_or(0); crate::consolidation::consolidate_from(&mut self.edits, prev); if self.edits.len() > prev { self.values.push((value, self.edits.len())); } } - fn map, &C::Time, &C::Diff)>(&self, mut logic: F) { + fn map, &C::Time, &C::Diff)>(&self, mut logic: F) { for index in 0 .. self.values.len() { let lower = if index == 0 { 0 } else { self.values[index-1].1 }; let upper = self.values[index].1; @@ -81,7 +81,7 @@ impl<'a, C: Cursor> EditList<'a, C> { struct ValueHistory<'storage, C: Cursor> { edits: EditList<'storage, C>, history: Vec<(C::Time, C::Time, usize, usize)>, // (time, meet, value_index, edit_offset) - buffer: Vec<((C::Val<'storage>, C::Time), C::Diff)>, // where we accumulate / collapse updates. + buffer: Vec<((columnar::Ref<'storage, C::Val>, C::Time), C::Diff)>, // where we accumulate / collapse updates. } impl<'storage, C: Cursor> ValueHistory<'storage, C> { @@ -99,7 +99,7 @@ impl<'storage, C: Cursor> ValueHistory<'storage, C> { } fn load(&mut self, cursor: &mut C, storage: &'storage C::Storage, logic: L) where - L: Fn(C::TimeGat<'_>)->C::Time, + L: Fn(columnar::Ref<'_, C::Time>)->C::Time, { self.edits.load(cursor, storage, logic); } @@ -111,11 +111,11 @@ impl<'storage, C: Cursor> ValueHistory<'storage, C> { &'history mut self, cursor: &mut C, storage: &'storage C::Storage, - key: C::Key<'storage>, + key: columnar::Ref<'storage, C::Key>, logic: L ) -> HistoryReplay<'storage, 'history, C> where - L: Fn(C::TimeGat<'_>)->C::Time, + L: Fn(columnar::Ref<'_, C::Time>)->C::Time, { self.clear(); cursor.seek_key(storage, key); @@ -157,11 +157,11 @@ struct HistoryReplay<'storage, 'history, C: Cursor> { impl<'storage, 'history, C: Cursor> HistoryReplay<'storage, 'history, C> { fn time(&self) -> Option<&C::Time> { self.replay.history.last().map(|x| &x.0) } fn meet(&self) -> Option<&C::Time> { self.replay.history.last().map(|x| &x.1) } - fn edit(&self) -> Option<(C::Val<'storage>, &C::Time, &C::Diff)> { + fn edit(&self) -> Option<(columnar::Ref<'storage, C::Val>, &C::Time, &C::Diff)> { self.replay.history.last().map(|&(ref t, _, v, e)| (self.replay.edits.values[v].0, t, &self.replay.edits.edits[e].1)) } - fn buffer(&self) -> &[((C::Val<'storage>, C::Time), C::Diff)] { + fn buffer(&self) -> &[((columnar::Ref<'storage, C::Val>, C::Time), C::Diff)] { &self.replay.buffer[..] } diff --git a/differential-dataflow/src/operators/reduce.rs b/differential-dataflow/src/operators/reduce.rs index f0302989f..f0e10211c 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -19,6 +19,7 @@ use timely::dataflow::operators::Capability; use crate::operators::arrange::{Arranged, TraceAgent}; use crate::trace::{BatchReader, Cursor, Trace, Builder, ExertionLogic, Description}; use crate::trace::cursor::CursorList; +use crate::trace::implementations::Coltainer; use crate::trace::implementations::containers::BatchContainer; use crate::trace::implementations::merge_batcher::container::MergerChunk; use crate::trace::TraceReader; @@ -29,10 +30,10 @@ use crate::trace::TraceReader; pub fn reduce_trace(trace: Arranged, name: &str, mut logic: L) -> Arranged> where G: Scope, - T1: TraceReader + Clone + 'static, - T2: for<'a> Trace=T1::Key<'a>, KeyOwn=T1::KeyOwn, ValOwn: Data, Time=T1::Time> + 'static, - Bu: Builder>, - L: FnMut(T1::Key<'_>, &[(T1::Val<'_>, T1::Diff)], &mut Vec<(T2::ValOwn,T2::Diff)>, &mut Vec<(T2::ValOwn, T2::Diff)>)+'static, + T1: TraceReader + Clone + 'static, + T2: Trace + 'static, + Bu: Builder>, + L: FnMut(columnar::Ref<'_, T1::Key>, &[(columnar::Ref<'_, T1::Val>, T1::Diff)], &mut Vec<(T2::Val,T2::Diff)>, &mut Vec<(T2::Val, T2::Diff)>)+'static, { let mut result_trace = None; @@ -67,7 +68,7 @@ where // Our implementation maintains a list of outstanding `(key, time)` synthetic interesting times, // as well as capabilities for these times (or their lower envelope, at least). - let mut interesting = Vec::<(T1::KeyOwn, G::Timestamp)>::new(); + let mut interesting = Vec::<(T1::Key, G::Timestamp)>::new(); let mut capabilities = Vec::>::new(); // buffers and logic for computing per-key interesting times "efficiently". @@ -146,8 +147,8 @@ where // We first extract those times from this list that lie in the interval we will process. sort_dedup(&mut interesting); // `exposed` contains interesting (key, time)s now below `upper_limit` - let mut exposed_keys = T1::KeyContainer::with_capacity(0); - let mut exposed_time = T1::TimeContainer::with_capacity(0); + let mut exposed_keys = Coltainer::::with_capacity(0); + let mut exposed_time = Coltainer::::with_capacity(0); // Keep pairs greater or equal to `upper_limit`, and "expose" other pairs. interesting.retain(|(key, time)| { if upper_limit.less_equal(time) { true } else { @@ -166,7 +167,7 @@ where // // TODO: It would be better if all updates went into one batch, but timely dataflow prevents // this as long as it requires that there is only one capability for each message. - let mut buffers = Vec::<(G::Timestamp, Vec<(T2::ValOwn, G::Timestamp, T2::Diff)>)>::new(); + let mut buffers = Vec::<(G::Timestamp, Vec<(T2::Val, G::Timestamp, T2::Diff)>)>::new(); let mut builders = Vec::new(); for cap in capabilities.iter() { buffers.push((cap.time().clone(), Vec::new())); @@ -211,7 +212,7 @@ where // Populate `interesting_times` with synthetic interesting times (below `upper_limit`) for this key. while exposed_keys.get(exposed_position) == Some(key) { - interesting_times.push(T1::owned_time(exposed_time.index(exposed_position))); + interesting_times.push(::into_owned(exposed_time.index(exposed_position))); exposed_position += 1; } @@ -238,7 +239,7 @@ where // Record future warnings about interesting times (and assert they should be "future"). for time in new_interesting_times.drain(..) { debug_assert!(upper_limit.less_equal(&time)); - interesting.push((T1::owned_key(key), time)); + interesting.push((::into_owned(key), time)); } // Sort each buffer by value and move into the corresponding builder. @@ -248,7 +249,7 @@ where for index in 0 .. buffers.len() { buffers[index].1.sort_by(|x,y| x.0.cmp(&y.0)); for (val, time, diff) in buffers[index].1.drain(..) { - buffer.push_into(((T1::owned_key(key), val), time, diff)); + buffer.push_into(((::into_owned(key), val), time, diff)); builders[index].push(&mut buffer); buffer.clear(); } @@ -348,14 +349,14 @@ fn sort_dedup(list: &mut Vec) { trait PerKeyCompute<'a, C1, C2, C3, V> where C1: Cursor, - C2: for<'b> Cursor = C1::Key<'a>, ValOwn = V, Time = C1::Time>, - C3: Cursor = C1::Key<'a>, Val<'a> = C1::Val<'a>, Time = C1::Time, Diff = C1::Diff>, - V: Clone + Ord, + C2: Cursor, + C3: Cursor, + V: crate::Data, { fn new() -> Self; fn compute( &mut self, - key: C1::Key<'a>, + key: columnar::Ref<'a, C1::Key>, source_cursor: (&mut C1, &'a C1::Storage), output_cursor: (&mut C2, &'a C2::Storage), batch_cursor: (&mut C3, &'a C3::Storage), @@ -366,8 +367,8 @@ where new_interesting: &mut Vec) -> (usize, usize) where L: FnMut( - C1::Key<'a>, - &[(C1::Val<'a>, C1::Diff)], + columnar::Ref<'a, C1::Key>, + &[(columnar::Ref<'a, C1::Val>, C1::Diff)], &mut Vec<(V, C2::Diff)>, &mut Vec<(V, C2::Diff)>, ); @@ -382,6 +383,7 @@ mod history_replay { use crate::lattice::Lattice; use crate::trace::Cursor; + use crate::operators::ValueHistory; use super::{PerKeyCompute, sort_dedup}; @@ -391,14 +393,14 @@ mod history_replay { pub struct HistoryReplayer<'a, C1, C2, C3, V> where C1: Cursor, - C2: Cursor = C1::Key<'a>, Time = C1::Time>, - C3: Cursor = C1::Key<'a>, Val<'a> = C1::Val<'a>, Time = C1::Time, Diff = C1::Diff>, + C2: Cursor, + C3: Cursor, V: Clone + Ord, { input_history: ValueHistory<'a, C1>, output_history: ValueHistory<'a, C2>, batch_history: ValueHistory<'a, C3>, - input_buffer: Vec<(C1::Val<'a>, C1::Diff)>, + input_buffer: Vec<(columnar::Ref<'a, C1::Val>, C1::Diff)>, output_buffer: Vec<(V, C2::Diff)>, update_buffer: Vec<(V, C2::Diff)>, output_produced: Vec<((V, C2::Time), C2::Diff)>, @@ -411,9 +413,9 @@ mod history_replay { impl<'a, C1, C2, C3, V> PerKeyCompute<'a, C1, C2, C3, V> for HistoryReplayer<'a, C1, C2, C3, V> where C1: Cursor, - C2: for<'b> Cursor = C1::Key<'a>, ValOwn = V, Time = C1::Time>, - C3: Cursor = C1::Key<'a>, Val<'a> = C1::Val<'a>, Time = C1::Time, Diff = C1::Diff>, - V: Clone + Ord, + C2: Cursor, + C3: Cursor, + V: crate::Data, { fn new() -> Self { HistoryReplayer { @@ -433,7 +435,7 @@ mod history_replay { #[inline(never)] fn compute( &mut self, - key: C1::Key<'a>, + key: columnar::Ref<'a, C1::Key>, (source_cursor, source_storage): (&mut C1, &'a C1::Storage), (output_cursor, output_storage): (&mut C2, &'a C2::Storage), (batch_cursor, batch_storage): (&mut C3, &'a C3::Storage), @@ -444,8 +446,8 @@ mod history_replay { new_interesting: &mut Vec) -> (usize, usize) where L: FnMut( - C1::Key<'a>, - &[(C1::Val<'a>, C1::Diff)], + columnar::Ref<'a, C1::Key>, + &[(columnar::Ref<'a, C1::Val>, C1::Diff)], &mut Vec<(V, C2::Diff)>, &mut Vec<(V, C2::Diff)>, ) @@ -460,7 +462,7 @@ mod history_replay { // loaded times by performing the lattice `join` with this value. // Load the batch contents. - let mut batch_replay = self.batch_history.replay_key(batch_cursor, batch_storage, key, |time| C3::owned_time(time)); + let mut batch_replay = self.batch_history.replay_key(batch_cursor, batch_storage, key, |time| ::into_owned(time)); // We determine the meet of times we must reconsider (those from `batch` and `times`). This meet // can be used to advance other historical times, which may consolidate their representation. As @@ -497,23 +499,23 @@ mod history_replay { // Load the input and output histories. let mut input_replay = if let Some(meet) = meet.as_ref() { self.input_history.replay_key(source_cursor, source_storage, key, |time| { - let mut time = C1::owned_time(time); + let mut time = ::into_owned(time); time.join_assign(meet); time }) } else { - self.input_history.replay_key(source_cursor, source_storage, key, |time| C1::owned_time(time)) + self.input_history.replay_key(source_cursor, source_storage, key, |time| ::into_owned(time)) }; let mut output_replay = if let Some(meet) = meet.as_ref() { self.output_history.replay_key(output_cursor, output_storage, key, |time| { - let mut time = C2::owned_time(time); + let mut time = ::into_owned(time); time.join_assign(meet); time }) } else { - self.output_history.replay_key(output_cursor, output_storage, key, |time| C2::owned_time(time)) + self.output_history.replay_key(output_cursor, output_storage, key, |time| ::into_owned(time)) }; self.synth_times.clear(); @@ -618,7 +620,7 @@ mod history_replay { meet.as_ref().map(|meet| output_replay.advance_buffer_by(meet)); for &((value, ref time), ref diff) in output_replay.buffer().iter() { if time.less_equal(&next_time) { - self.output_buffer.push((C2::owned_val(value), diff.clone())); + self.output_buffer.push((::into_owned(value), diff.clone())); } else { self.temporary.push(next_time.join(time)); diff --git a/differential-dataflow/src/operators/threshold.rs b/differential-dataflow/src/operators/threshold.rs index 91bbd16fe..2eee57f11 100644 --- a/differential-dataflow/src/operators/threshold.rs +++ b/differential-dataflow/src/operators/threshold.rs @@ -22,7 +22,7 @@ pub trait ThresholdTotal, K: Exchang fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, - F: FnMut(&K,&R,Option<&R>)->Option+'static, + F: for<'a> FnMut(columnar::Ref<'a, K>,&R,Option<&R>)->Option+'static, ; /// Reduces the collection to one occurrence of each distinct element. /// @@ -39,11 +39,14 @@ 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 + where K: columnar::Columnar, + { self.threshold_semigroup(move |key, new, old| { - let mut new = thresh(key, new); + let key_own = ::into_owned(key); + let mut new = thresh(&key_own, new); if let Some(old) = old { - let mut add = thresh(key, old); + let mut add = thresh(&key_own, old); add.negate(); new.plus_equals(&add); } @@ -86,12 +89,12 @@ pub trait ThresholdTotal, K: Exchang impl ThresholdTotal for VecCollection where - G: Scope, + G: Scope, { fn threshold_semigroup(self, thresh: F) -> VecCollection where R2: Semigroup+'static, - F: FnMut(&K,&R,Option<&R>)->Option+'static, + F: for<'a> FnMut(columnar::Ref<'a, K>,&R,Option<&R>)->Option+'static, { self.arrange_by_self_named("Arrange: ThresholdTotal") .threshold_semigroup(thresh) @@ -101,18 +104,18 @@ where impl ThresholdTotal for Arranged where G: Scope, - T1: for<'a> TraceReader< - Key<'a>=&'a K, - Val<'a>=&'a (), + T1: TraceReader< + Key = K, + Val = (), Time: TotalOrder, - Diff : ExchangeData + Semigroup>, + Diff: ExchangeData, >+Clone+'static, K: ExchangeData, { 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, + F: for<'a> FnMut(columnar::Ref<'a, T1::Key>,&T1::Diff,Option<&T1::Diff>)->Option+'static, { let mut trace = self.trace.clone(); @@ -159,8 +162,9 @@ where trace_cursor.seek_key(&trace_storage, key); if trace_cursor.get_key(&trace_storage) == Some(key) { trace_cursor.map_times(&trace_storage, |_, diff| { + let diff = ::into_owned(diff); count.as_mut().map(|c| c.plus_equals(&diff)); - if count.is_none() { count = Some(T1::owned_diff(diff)); } + if count.is_none() { count = Some(diff); } }); } @@ -168,6 +172,7 @@ where // If the result is non-zero, send it along. batch_cursor.map_times(&batch_storage, |time, diff| { + let diff = ::into_owned(diff); let difference = match &count { Some(old) => { @@ -175,7 +180,7 @@ where temp.plus_equals(&diff); thresh(key, &temp, Some(old)) }, - None => { thresh(key, &T1::owned_diff(diff), None) }, + None => { thresh(key, &diff, None) }, }; // Either add or assign `diff` to `count`. @@ -183,12 +188,12 @@ where count.plus_equals(&diff); } else { - count = Some(T1::owned_diff(diff)); + count = Some(diff); } if let Some(difference) = difference { if !difference.is_zero() { - session.give((key.clone(), T1::owned_time(time), difference)); + session.give((::into_owned(key), ::into_owned(time), difference)); } } }); diff --git a/differential-dataflow/src/trace/cursor/cursor_list.rs b/differential-dataflow/src/trace/cursor/cursor_list.rs index 6d2d5b57b..7d67aa888 100644 --- a/differential-dataflow/src/trace/cursor/cursor_list.rs +++ b/differential-dataflow/src/trace/cursor/cursor_list.rs @@ -26,20 +26,9 @@ impl CursorList { result } - /// Initialize min_key with the indices of cursors with the minimum key. - /// - /// This method scans the current keys of each cursor, and tracks the indices - /// of cursors whose key equals the minimum valid key seen so far. As it goes, - /// if it observes an improved key it clears the current list, updates the - /// minimum key, and continues. - /// - /// Once finished, it invokes `minimize_vals()` to ensure the value cursor is - /// in a consistent state as well. fn minimize_keys(&mut self, storage: &[C::Storage]) { - self.min_key.clear(); - // We'll visit each non-`None` key, maintaining the indexes of the least keys in `self.min_key`. let mut iter = self.cursors.iter().enumerate().flat_map(|(idx, cur)| cur.get_key(&storage[idx]).map(|key| (idx, key))); if let Some((idx, key)) = iter.next() { let mut min_key = key; @@ -62,17 +51,9 @@ impl CursorList { self.minimize_vals(storage); } - /// Initialize min_val with the indices of minimum key cursors with the minimum value. - /// - /// This method scans the current values of cursor with minimum keys, and tracks the - /// indices of cursors whose value equals the minimum valid value seen so far. As it - /// goes, if it observes an improved value it clears the current list, updates the minimum - /// value, and continues. fn minimize_vals(&mut self, storage: &[C::Storage]) { - self.min_val.clear(); - // We'll visit each non-`None` value, maintaining the indexes of the least values in `self.min_val`. let mut iter = self.min_key.iter().cloned().flat_map(|idx| self.cursors[idx].get_val(&storage[idx]).map(|val| (idx, val))); if let Some((idx, val)) = iter.next() { let mut min_val = val; @@ -94,52 +75,48 @@ impl CursorList { } } -use crate::trace::implementations::WithLayout; -impl WithLayout for CursorList { - type Layout = C::Layout; -} - impl Cursor for CursorList { + type Key = C::Key; + type Val = C::Val; + type Time = C::Time; + type Diff = C::Diff; type Storage = Vec; - // validation methods #[inline] fn key_valid(&self, _storage: &Vec) -> bool { !self.min_key.is_empty() } #[inline] fn val_valid(&self, _storage: &Vec) -> bool { !self.min_val.is_empty() } - // accessors #[inline] - fn key<'a>(&self, storage: &'a Vec) -> Self::Key<'a> { + fn key<'a>(&self, storage: &'a Vec) -> columnar::Ref<'a, Self::Key> { debug_assert!(self.key_valid(storage)); debug_assert!(self.cursors[self.min_key[0]].key_valid(&storage[self.min_key[0]])); self.cursors[self.min_key[0]].key(&storage[self.min_key[0]]) } #[inline] - fn val<'a>(&self, storage: &'a Vec) -> Self::Val<'a> { + fn val<'a>(&self, storage: &'a Vec) -> columnar::Ref<'a, Self::Val> { debug_assert!(self.key_valid(storage)); debug_assert!(self.val_valid(storage)); debug_assert!(self.cursors[self.min_val[0]].val_valid(&storage[self.min_val[0]])); self.cursors[self.min_val[0]].val(&storage[self.min_val[0]]) } #[inline] - fn get_key<'a>(&self, storage: &'a Vec) -> Option> { + fn get_key<'a>(&self, storage: &'a Vec) -> Option> { self.min_key.get(0).map(|idx| self.cursors[*idx].key(&storage[*idx])) } #[inline] - fn get_val<'a>(&self, storage: &'a Vec) -> Option> { + fn get_val<'a>(&self, storage: &'a Vec) -> Option> { self.min_val.get(0).map(|idx| self.cursors[*idx].val(&storage[*idx])) } #[inline] - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Vec, mut logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Vec, mut logic: L) { for &index in self.min_val.iter() { self.cursors[index].map_times(&storage[index], |t,d| logic(t,d)); } } - // key methods #[inline] fn step_key(&mut self, storage: &Vec) { for &index in self.min_key.iter() { @@ -148,14 +125,13 @@ impl Cursor for CursorList { self.minimize_keys(storage); } #[inline] - fn seek_key(&mut self, storage: &Vec, key: Self::Key<'_>) { + fn seek_key(&mut self, storage: &Vec, key: columnar::Ref<'_, Self::Key>) { for (cursor, storage) in self.cursors.iter_mut().zip(storage) { cursor.seek_key(storage, key); } self.minimize_keys(storage); } - // value methods #[inline] fn step_val(&mut self, storage: &Vec) { for &index in self.min_val.iter() { @@ -164,14 +140,13 @@ impl Cursor for CursorList { self.minimize_vals(storage); } #[inline] - fn seek_val(&mut self, storage: &Vec, val: Self::Val<'_>) { + fn seek_val(&mut self, storage: &Vec, val: columnar::Ref<'_, Self::Val>) { for (cursor, storage) in self.cursors.iter_mut().zip(storage) { cursor.seek_val(storage, val); } self.minimize_vals(storage); } - // rewinding methods #[inline] fn rewind_keys(&mut self, storage: &Vec) { for (cursor, storage) in self.cursors.iter_mut().zip(storage) { diff --git a/differential-dataflow/src/trace/cursor/mod.rs b/differential-dataflow/src/trace/cursor/mod.rs index d0d973249..73a0c575b 100644 --- a/differential-dataflow/src/trace/cursor/mod.rs +++ b/differential-dataflow/src/trace/cursor/mod.rs @@ -9,10 +9,22 @@ pub mod cursor_list; pub use self::cursor_list::CursorList; -use crate::trace::implementations::LayoutExt; +use crate::trace::implementations::Data; /// A cursor for navigating ordered `(key, val, time, diff)` updates. -pub trait Cursor : LayoutExt { +/// +/// Types are columnar: owned types implement `Data` (which includes `Columnar`), +/// and all reference types are the corresponding `columnar::Ref<'a, T>`. +pub trait Cursor { + + /// The owned key type. + type Key: Data; + /// The owned val type. + type Val: Data; + /// The owned time type. + type Time: Data + crate::lattice::Lattice + timely::progress::Timestamp; + /// The owned diff type. + type Diff: Data + crate::difference::Semigroup; /// Storage required by the cursor. type Storage; @@ -27,28 +39,28 @@ pub trait Cursor : LayoutExt { fn val_valid(&self, storage: &Self::Storage) -> bool; /// A reference to the current key. Asserts if invalid. - fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a>; + fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key>; /// A reference to the current value. Asserts if invalid. - fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a>; + fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val>; /// Returns a reference to the current key, if valid. - fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option>; + fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option>; /// Returns a reference to the current value, if valid. - fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option>; + fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option>; /// Applies `logic` to each pair of time and difference. Intended for mutation of the /// closure's scope. - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, logic: L); + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, logic: L); /// Advances the cursor to the next key. fn step_key(&mut self, storage: &Self::Storage); /// Advances the cursor to the specified key. - fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>); + fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>); /// Advances the cursor to the next value. fn step_val(&mut self, storage: &Self::Storage); /// Advances the cursor to the specified value. - fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>); + fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>); /// Rewinds the cursor to the first key. fn rewind_keys(&mut self, storage: &Self::Storage); @@ -58,8 +70,8 @@ pub trait Cursor : LayoutExt { /// Rewinds the cursor and outputs its contents to a Vec fn to_vec(&mut self, storage: &Self::Storage, into_key: IK, into_val: IV) -> Vec<((K, V), Vec<(Self::Time, Self::Diff)>)> where - IK: for<'a> Fn(Self::Key<'a>) -> K, - IV: for<'a> Fn(Self::Val<'a>) -> V, + IK: for<'a> Fn(columnar::Ref<'a, Self::Key>) -> K, + IV: for<'a> Fn(columnar::Ref<'a, Self::Val>) -> V, { let mut out = Vec::new(); self.rewind_keys(storage); @@ -68,7 +80,10 @@ pub trait Cursor : LayoutExt { while let Some(val) = self.get_val(storage) { let mut kv_out = Vec::new(); self.map_times(storage, |ts, r| { - kv_out.push((Self::owned_time(ts), Self::owned_diff(r))); + kv_out.push(( + ::into_owned(ts), + ::into_owned(r), + )); }); out.push(((into_key(key), into_val(val)), kv_out)); self.step_val(storage); diff --git a/differential-dataflow/src/trace/implementations/mod.rs b/differential-dataflow/src/trace/implementations/mod.rs index d73eb71f4..09ebed20a 100644 --- a/differential-dataflow/src/trace/implementations/mod.rs +++ b/differential-dataflow/src/trace/implementations/mod.rs @@ -4,39 +4,6 @@ //! there is substantial flexibility in implementations of this trait. Depending on characteristics of //! the data, we may wish to represent the data in different ways. This module contains several of these //! implementations, and combiners for merging the results of different traces. -//! -//! As examples of implementations, -//! -//! * The `trie` module is meant to represent general update tuples, with no particular assumptions made -//! about their contents. It organizes the data first by key, then by val, and then leaves the rest -//! in an unordered pile. -//! -//! * The `keys` module is meant for collections whose value type is `()`, which is to say there is no -//! (key, val) structure on the records; all of them are just viewed as "keys". -//! -//! * The `time` module is meant for collections with a single time value. This can remove repetition -//! from the representation, at the cost of requiring more instances and run-time merging. -//! -//! * The `base` module is meant for collections with a single time value equivalent to the least time. -//! These collections must always accumulate to non-negative collections, and as such we can indicate -//! the frequency of an element by its multiplicity. This removes both the time and weight from the -//! representation, but is only appropriate for a subset (often substantial) of the data. -//! -//! Each of these representations is best suited for different data, but they can be combined to get the -//! benefits of each, as appropriate. There are several `Cursor` combiners, `CursorList` and `CursorPair`, -//! for homogeneous and inhomogeneous cursors, respectively. -//! -//! #Musings -//! -//! What is less clear is how to transfer updates between the representations at merge time in a tasteful -//! way. Perhaps we could put an ordering on the representations, each pair with a dominant representation, -//! and part of merging the latter filters updates into the former. Although back and forth might be -//! appealing, more thinking is required to negotiate all of these policies. -//! -//! One option would be to require the layer builder to handle these smarts. Merging is currently done by -//! the layer as part of custom code, but we could make it simply be "iterate through cursor, push results -//! into 'ordered builder'". Then the builder would be bright enough to emit a "batch" for the composite -//! trace, rather than just a batch of the type merged. pub mod spine_fueled; @@ -57,33 +24,47 @@ pub use self::ord_neu::RcOrdKeyBuilder as KeyBuilder; use std::convert::TryInto; -use columnation::Columnation; use serde::{Deserialize, Serialize}; -use timely::container::{DrainContainer, PushInto}; +use timely::container::PushInto; use timely::progress::Timestamp; -use crate::containers::TimelyStack; use crate::lattice::Lattice; use crate::difference::Semigroup; +// --------------------------------------------------------------------------- +// Core columnar data abstraction +// --------------------------------------------------------------------------- + +/// A columnar container whose references can be ordered. +pub trait OrdContainer: for<'a> columnar::Container: Ord> {} +impl columnar::Container: Ord>> OrdContainer for C {} + +/// A type suitable for use as data in differential dataflow traces. +/// +/// All data types must implement `Columnar` (for columnar storage), `Ord + Clone` +/// (for sorting and ownership), and their columnar reference types must be `Ord` +/// (for seeking and comparison in cursors). +pub trait Data: columnar::Columnar + Ord + Clone + 'static {} +impl + Ord + Clone + 'static> Data for T {} + /// A type that names constituent update types. pub trait Update { /// Key by which data are grouped. - type Key: Ord + Clone + 'static; + type Key: Data; /// Values associated with the key. - type Val: Ord + Clone + 'static; + type Val: Data; /// Time at which updates occur. - type Time: Ord + Clone + Lattice + timely::progress::Timestamp; + type Time: Data + Lattice + Timestamp; /// Way in which updates occur. - type Diff: Ord + Semigroup + 'static; + type Diff: Data + Semigroup; } impl Update for ((K, V), T, R) where - K: Ord+Clone+'static, - V: Ord+Clone+'static, - T: Ord+Clone+Lattice+timely::progress::Timestamp, - R: Ord+Semigroup+'static, + K: Data, + V: Data, + T: Data + Lattice + Timestamp, + R: Data + Semigroup, { type Key = K; type Val = V; @@ -91,164 +72,108 @@ where type Diff = R; } -/// A type with opinions on how updates should be laid out. -pub trait Layout { - /// Container for update keys. - type KeyContainer: BatchContainer; - /// Container for update vals. - type ValContainer: BatchContainer; - /// Container for times. - type TimeContainer: BatchContainer; - /// Container for diffs. - type DiffContainer: BatchContainer; - /// Container for offsets. - type OffsetContainer: for<'a> BatchContainer = usize>; -} +// --------------------------------------------------------------------------- +// Coltainer: the columnar BatchContainer +// --------------------------------------------------------------------------- -/// A type bearing a layout. -pub trait WithLayout { - /// The layout. - type Layout: Layout; +/// A container backed by a columnar store. +/// +/// This wraps a `Columnar::Container` and implements `BatchContainer`, +/// providing the bridge between the columnar storage system and +/// differential dataflow's batch infrastructure. +pub struct Coltainer { + /// The underlying columnar container. + pub container: C::Container, } -/// Automatically implemented trait for types with layouts. -pub trait LayoutExt : WithLayout> { - /// Alias for an owned key of a layout. - type KeyOwn; - /// Alias for an borrowed key of a layout. - type Key<'a>: Copy + Ord; - /// Alias for an owned val of a layout. - type ValOwn: Clone + Ord; - /// Alias for an borrowed val of a layout. - type Val<'a>: Copy + Ord; - /// Alias for an owned time of a layout. - type Time: Lattice + timely::progress::Timestamp; - /// Alias for an borrowed time of a layout. - type TimeGat<'a>: Copy + Ord; - /// Alias for an owned diff of a layout. - type Diff: Semigroup + 'static; - /// Alias for an borrowed diff of a layout. - type DiffGat<'a>: Copy + Ord; - - /// Container for update keys. - type KeyContainer: for<'a> BatchContainer = Self::Key<'a>, Owned = Self::KeyOwn>; - /// Container for update vals. - type ValContainer: for<'a> BatchContainer = Self::Val<'a>, Owned = Self::ValOwn>; - /// Container for times. - type TimeContainer: for<'a> BatchContainer = Self::TimeGat<'a>, Owned = Self::Time>; - /// Container for diffs. - type DiffContainer: for<'a> BatchContainer = Self::DiffGat<'a>, Owned = Self::Diff>; - - /// Construct an owned key from a reference. - fn owned_key(key: Self::Key<'_>) -> Self::KeyOwn; - /// Construct an owned val from a reference. - fn owned_val(val: Self::Val<'_>) -> Self::ValOwn; - /// Construct an owned time from a reference. - fn owned_time(time: Self::TimeGat<'_>) -> Self::Time; - /// Construct an owned diff from a reference. - fn owned_diff(diff: Self::DiffGat<'_>) -> Self::Diff; - - /// Clones a reference time onto an owned time. - fn clone_time_onto(time: Self::TimeGat<'_>, onto: &mut Self::Time); +impl Default for Coltainer { + fn default() -> Self { Self { container: Default::default() } } } -impl LayoutExt for L { - type KeyOwn = <::KeyContainer as BatchContainer>::Owned; - type Key<'a> = <::KeyContainer as BatchContainer>::ReadItem<'a>; - type ValOwn = <::ValContainer as BatchContainer>::Owned; - type Val<'a> = <::ValContainer as BatchContainer>::ReadItem<'a>; - type Time = <::TimeContainer as BatchContainer>::Owned; - type TimeGat<'a> = <::TimeContainer as BatchContainer>::ReadItem<'a>; - type Diff = <::DiffContainer as BatchContainer>::Owned; - type DiffGat<'a> = <::DiffContainer as BatchContainer>::ReadItem<'a>; - - type KeyContainer = ::KeyContainer; - type ValContainer = ::ValContainer; - type TimeContainer = ::TimeContainer; - type DiffContainer = ::DiffContainer; - - #[inline(always)] fn owned_key(key: Self::Key<'_>) -> Self::KeyOwn { ::KeyContainer::into_owned(key) } - #[inline(always)] fn owned_val(val: Self::Val<'_>) -> Self::ValOwn { ::ValContainer::into_owned(val) } - #[inline(always)] fn owned_time(time: Self::TimeGat<'_>) -> Self::Time { ::TimeContainer::into_owned(time) } - #[inline(always)] fn owned_diff(diff: Self::DiffGat<'_>) -> Self::Diff { ::DiffContainer::into_owned(diff) } - #[inline(always)] fn clone_time_onto(time: Self::TimeGat<'_>, onto: &mut Self::Time) { ::TimeContainer::clone_onto(time, onto) } - +impl Clone for Coltainer { + fn clone(&self) -> Self { Self { container: self.container.clone() } } } -// An easy way to provide an explicit layout: as a 5-tuple. -// Valuable when one wants to perform layout surgery. -impl Layout for (KC, VC, TC, DC, OC) -where - KC: BatchContainer, - VC: BatchContainer, - TC: BatchContainer, - DC: BatchContainer, - OC: for<'a> BatchContainer = usize>, +impl std::fmt::Debug for Coltainer +where C::Container: std::fmt::Debug { - type KeyContainer = KC; - type ValContainer = VC; - type TimeContainer = TC; - type DiffContainer = DC; - type OffsetContainer = OC; + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.container.fmt(f) + } } -/// Aliases for types provided by the containers within a `Layout`. -/// -/// For clarity, prefer `use layout;` and then `layout::Key` to retain the layout context. -pub mod layout { - use crate::trace::implementations::{BatchContainer, Layout}; - - /// Alias for an owned key of a layout. - pub type Key = <::KeyContainer as BatchContainer>::Owned; - /// Alias for an borrowed key of a layout. - pub type KeyRef<'a, L> = <::KeyContainer as BatchContainer>::ReadItem<'a>; - /// Alias for an owned val of a layout. - pub type Val = <::ValContainer as BatchContainer>::Owned; - /// Alias for an borrowed val of a layout. - pub type ValRef<'a, L> = <::ValContainer as BatchContainer>::ReadItem<'a>; - /// Alias for an owned time of a layout. - pub type Time = <::TimeContainer as BatchContainer>::Owned; - /// Alias for an borrowed time of a layout. - pub type TimeRef<'a, L> = <::TimeContainer as BatchContainer>::ReadItem<'a>; - /// Alias for an owned diff of a layout. - pub type Diff = <::DiffContainer as BatchContainer>::Owned; - /// Alias for an borrowed diff of a layout. - pub type DiffRef<'a, L> = <::DiffContainer as BatchContainer>::ReadItem<'a>; -} +impl BatchContainer for Coltainer { + type ReadItem<'a> = columnar::Ref<'a, C>; + type Owned = C; -/// A layout that uses vectors -pub struct Vector { - phantom: std::marker::PhantomData, -} + #[inline(always)] + fn into_owned<'a>(item: Self::ReadItem<'a>) -> Self::Owned { C::into_owned(item) } + #[inline(always)] + fn clone_onto<'a>(item: Self::ReadItem<'a>, other: &mut Self::Owned) { other.copy_from(item) } + + #[inline(always)] + fn push_ref(&mut self, item: Self::ReadItem<'_>) { + use columnar::Push; + self.container.push(item) + } + #[inline(always)] + fn push_own(&mut self, item: &Self::Owned) { + >::push(&mut self.container, item) + } + + fn clear(&mut self) { columnar::Clear::clear(&mut self.container) } -impl> Layout for Vector { - type KeyContainer = Vec; - type ValContainer = Vec; - type TimeContainer = Vec; - type DiffContainer = Vec; - type OffsetContainer = OffsetList; + fn with_capacity(_size: usize) -> Self { Self::default() } + fn merge_capacity(cont1: &Self, cont2: &Self) -> Self { + use columnar::Borrow; + Self { + container: columnar::Container::with_capacity_for( + [cont1.container.borrow(), cont2.container.borrow()].into_iter() + ), + } + } + + #[inline(always)] + fn reborrow<'b, 'a: 'b>(item: Self::ReadItem<'a>) -> Self::ReadItem<'b> { + ::reborrow_ref(item) + } + + #[inline(always)] + fn index(&self, index: usize) -> Self::ReadItem<'_> { + use columnar::{Borrow, Index}; + self.container.borrow().get(index) + } + + fn len(&self) -> usize { + columnar::Len::len(&self.container) + } } -/// A layout based on timely stacks -pub struct TStack { - phantom: std::marker::PhantomData, +impl PushInto<&C> for Coltainer { + fn push_into(&mut self, item: &C) { + >::push(&mut self.container, item) + } } -impl Layout for TStack -where - U: Update< - Key: Columnation, - Val: Columnation, - Time: Columnation, - Diff: Columnation + Ord, - >, +// Serde support for Coltainer: delegate to the inner container. +impl Serialize for Coltainer +where C::Container: Serialize { - type KeyContainer = TimelyStack; - type ValContainer = TimelyStack; - type TimeContainer = TimelyStack; - type DiffContainer = TimelyStack; - type OffsetContainer = OffsetList; + fn serialize(&self, serializer: S) -> Result { + self.container.serialize(serializer) + } } +impl<'de, C: columnar::Columnar> Deserialize<'de> for Coltainer +where C::Container: Deserialize<'de> +{ + fn deserialize>(deserializer: D) -> Result { + Ok(Self { container: C::Container::deserialize(deserializer)? }) + } +} + +// --------------------------------------------------------------------------- +// OffsetList: compact offset storage +// --------------------------------------------------------------------------- /// A list of unsigned integers that uses `u32` elements as long as they are small enough, and switches to `u64` once they are not. #[derive(Eq, PartialEq, Ord, PartialOrd, Clone, Serialize, Deserialize)] @@ -377,148 +302,19 @@ impl BatchContainer for OffsetList { } } -/// Behavior to split an update into principal components. -pub trait BuilderInput: DrainContainer + Sized { - /// Key portion - type Key<'a>: Ord; - /// Value portion - type Val<'a>: Ord; - /// Time - type Time; - /// Diff - type Diff; - - /// Split an item into separate parts. - fn into_parts<'a>(item: Self::Item<'a>) -> (Self::Key<'a>, Self::Val<'a>, Self::Time, Self::Diff); - - /// Test that the key equals a key in the layout's key container. - fn key_eq(this: &Self::Key<'_>, other: K::ReadItem<'_>) -> bool; - - /// Test that the value equals a key in the layout's value container. - fn val_eq(this: &Self::Val<'_>, other: V::ReadItem<'_>) -> bool; - - /// Count the number of distinct keys, (key, val) pairs, and total updates. - fn key_val_upd_counts(chain: &[Self]) -> (usize, usize, usize); -} - -impl BuilderInput for Vec<((K, V), T, R)> -where - K: Ord + Clone + 'static, - KBC: for<'a> BatchContainer: PartialEq<&'a K>>, - V: Ord + Clone + 'static, - VBC: for<'a> BatchContainer: PartialEq<&'a V>>, - T: Timestamp + Lattice + Clone + 'static, - R: Ord + Semigroup + 'static, -{ - type Key<'a> = K; - type Val<'a> = V; - type Time = T; - type Diff = R; - - fn into_parts<'a>(((key, val), time, diff): Self::Item<'a>) -> (Self::Key<'a>, Self::Val<'a>, Self::Time, Self::Diff) { - (key, val, time, diff) - } - - fn key_eq(this: &K, other: KBC::ReadItem<'_>) -> bool { - KBC::reborrow(other) == this - } - - fn val_eq(this: &V, other: VBC::ReadItem<'_>) -> bool { - VBC::reborrow(other) == this - } - - fn key_val_upd_counts(chain: &[Self]) -> (usize, usize, usize) { - let mut keys = 0; - let mut vals = 0; - let mut upds = 0; - let mut prev_keyval = None; - for link in chain.iter() { - for ((key, val), _, _) in link.iter() { - if let Some((p_key, p_val)) = prev_keyval { - if p_key != key { - keys += 1; - vals += 1; - } else if p_val != val { - vals += 1; - } - } else { - keys += 1; - vals += 1; - } - upds += 1; - prev_keyval = Some((key, val)); - } - } - (keys, vals, upds) - } -} - -impl BuilderInput for TimelyStack<((K::Owned, V::Owned), T, R)> -where - K: for<'a> BatchContainer< - ReadItem<'a>: PartialEq<&'a K::Owned>, - Owned: Ord + Columnation + Clone + 'static, - >, - V: for<'a> BatchContainer< - ReadItem<'a>: PartialEq<&'a V::Owned>, - Owned: Ord + Columnation + Clone + 'static, - >, - T: Timestamp + Lattice + Columnation + Clone + 'static, - R: Ord + Clone + Semigroup + Columnation + 'static, -{ - type Key<'a> = &'a K::Owned; - type Val<'a> = &'a V::Owned; - type Time = T; - type Diff = R; - - fn into_parts<'a>(((key, val), time, diff): Self::Item<'a>) -> (Self::Key<'a>, Self::Val<'a>, Self::Time, Self::Diff) { - (key, val, time.clone(), diff.clone()) - } +// BuilderInput removed: Columnar already provides the owned↔ref relationship. - fn key_eq(this: &&K::Owned, other: K::ReadItem<'_>) -> bool { - K::reborrow(other) == *this - } - - fn val_eq(this: &&V::Owned, other: V::ReadItem<'_>) -> bool { - V::reborrow(other) == *this - } - - fn key_val_upd_counts(chain: &[Self]) -> (usize, usize, usize) { - let mut keys = 0; - let mut vals = 0; - let mut upds = 0; - let mut prev_keyval = None; - for link in chain.iter() { - for ((key, val), _, _) in link.iter() { - if let Some((p_key, p_val)) = prev_keyval { - if p_key != key { - keys += 1; - vals += 1; - } else if p_val != val { - vals += 1; - } - } else { - keys += 1; - vals += 1; - } - upds += 1; - prev_keyval = Some((key, val)); - } - } - (keys, vals, upds) - } -} +// --------------------------------------------------------------------------- +// BatchContainer trait +// --------------------------------------------------------------------------- pub use self::containers::{BatchContainer, SliceContainer}; /// Containers for data that resemble `Vec`, with leaner implementations. pub mod containers { - use columnation::Columnation; use timely::container::PushInto; - use crate::containers::TimelyStack; - /// A general-purpose container resembling `Vec`. pub trait BatchContainer: 'static { /// An owned instance of `Self::ReadItem<'_>`. @@ -527,7 +323,6 @@ pub mod containers { /// The type that can be read back out of the container. type ReadItem<'a>: Copy + Ord; - /// Conversion from an instance of this type to the owned type. #[must_use] fn into_owned<'a>(item: Self::ReadItem<'a>) -> Self::Owned; @@ -623,8 +418,7 @@ pub mod containers { } } - // All `T: Clone` also implement `ToOwned`, but without the constraint Rust - // struggles to understand why the owned type must be `T` (i.e. the one blanket impl). + // Vec impl retained for builder compatibility (accepts Vec<((K,V),T,R)> input). impl BatchContainer for Vec { type Owned = T; type ReadItem<'a> = &'a T; @@ -656,38 +450,6 @@ pub mod containers { } } - // The `ToOwned` requirement exists to satisfy `self.reserve_items`, who must for now - // be presented with the actual contained type, rather than a type that borrows into it. - impl BatchContainer for TimelyStack { - type Owned = T; - type ReadItem<'a> = &'a T; - - #[inline(always)] fn into_owned<'a>(item: Self::ReadItem<'a>) -> Self::Owned { item.clone() } - #[inline(always)] fn clone_onto<'a>(item: Self::ReadItem<'a>, other: &mut Self::Owned) { other.clone_from(item); } - - fn reborrow<'b, 'a: 'b>(item: Self::ReadItem<'a>) -> Self::ReadItem<'b> { item } - - fn push_ref(&mut self, item: Self::ReadItem<'_>) { self.push_into(item) } - fn push_own(&mut self, item: &Self::Owned) { self.push_into(item) } - - fn clear(&mut self) { self.clear() } - - fn with_capacity(size: usize) -> Self { - Self::with_capacity(size) - } - fn merge_capacity(cont1: &Self, cont2: &Self) -> Self { - let mut new = Self::default(); - new.reserve_regions(std::iter::once(cont1).chain(std::iter::once(cont2))); - new - } - fn index(&self, index: usize) -> Self::ReadItem<'_> { - &self[index] - } - fn len(&self) -> usize { - self[..].len() - } - } - /// A container that accepts slices `[B::Item]`. pub struct SliceContainer { /// Offsets that bound each contained slice. diff --git a/differential-dataflow/src/trace/implementations/ord_neu.rs b/differential-dataflow/src/trace/implementations/ord_neu.rs index b6f9a198e..7ad4e067b 100644 --- a/differential-dataflow/src/trace/implementations/ord_neu.rs +++ b/differential-dataflow/src/trace/implementations/ord_neu.rs @@ -10,53 +10,29 @@ use std::rc::Rc; -use crate::containers::TimelyStack; -use crate::trace::implementations::chunker::{ColumnationChunker, VecChunker}; +use crate::trace::implementations::chunker::VecChunker; use crate::trace::implementations::spine_fueled::Spine; -use crate::trace::implementations::merge_batcher::{MergeBatcher, VecMerger, ColMerger}; +use crate::trace::implementations::merge_batcher::{MergeBatcher, VecMerger}; use crate::trace::rc_blanket_impls::RcBuilder; -use super::{Layout, Vector, TStack}; +use super::Update; pub use self::val_batch::{OrdValBatch, OrdValBuilder}; pub use self::key_batch::{OrdKeyBatch, OrdKeyBuilder}; /// A trace implementation using a spine of ordered lists. -pub type OrdValSpine = Spine>>>; +pub type OrdValSpine = Spine>>; /// A batcher using ordered lists. pub type OrdValBatcher = MergeBatcher, VecChunker<((K,V),T,R)>, VecMerger<(K, V), T, R>>; /// A builder using ordered lists. -pub type RcOrdValBuilder = RcBuilder, Vec<((K,V),T,R)>>>; - -// /// A trace implementation for empty values using a spine of ordered lists. -// pub type OrdKeySpine = Spine>>>; - -/// A trace implementation backed by columnar storage. -pub type ColValSpine = Spine>>>; -/// A batcher for columnar storage. -pub type ColValBatcher = MergeBatcher, ColumnationChunker<((K,V),T,R)>, ColMerger<(K,V),T,R>>; -/// A builder for columnar storage. -pub type ColValBuilder = RcBuilder, TimelyStack<((K,V),T,R)>>>; +pub type RcOrdValBuilder = RcBuilder>; /// A trace implementation using a spine of ordered lists. -pub type OrdKeySpine = Spine>>>; +pub type OrdKeySpine = Spine>>; /// A batcher for ordered lists. pub type OrdKeyBatcher = MergeBatcher, VecChunker<((K,()),T,R)>, VecMerger<(K, ()), T, R>>; /// A builder for ordered lists. -pub type RcOrdKeyBuilder = RcBuilder, Vec<((K,()),T,R)>>>; - -// /// A trace implementation for empty values using a spine of ordered lists. -// pub type OrdKeySpine = Spine>>>; - -/// A trace implementation backed by columnar storage. -pub type ColKeySpine = Spine>>>; -/// A batcher for columnar storage -pub type ColKeyBatcher = MergeBatcher, ColumnationChunker<((K,()),T,R)>, ColMerger<(K,()),T,R>>; -/// A builder for columnar storage -pub type ColKeyBuilder = RcBuilder, TimelyStack<((K,()),T,R)>>>; - -// /// A trace implementation backed by columnar storage. -// pub type ColKeySpine = Spine>>>; +pub type RcOrdKeyBuilder = RcBuilder>; pub use layers::{Vals, Upds}; /// Layers are containers of lists of some type. @@ -93,11 +69,6 @@ pub mod layers { (self.offs.index(index), self.offs.index(index+1)) } /// Retrieves a value using relative indexes. - /// - /// The first index identifies a list, and the second an item within the list. - /// The method adds the list's lower bound to the item index, and then calls - /// `get_abs`. Using absolute indexes within the list's bounds can be more - /// efficient than using relative indexing. pub fn get_rel(&self, list_idx: usize, item_idx: usize) -> V::ReadItem<'_> { self.get_abs(self.bounds(list_idx).0 + item_idx) } @@ -151,7 +122,6 @@ pub mod layers { let mut lower = self.offs.index(index); let upper = self.offs.index(index+1); // We use equal lower and upper to encode "singleton update; just before here". - // It should only apply when there is a prior element, so `lower` should be greater than zero. if lower == upper { assert!(lower > 0); lower -= 1; @@ -159,11 +129,6 @@ pub mod layers { (lower, upper) } /// Retrieves a value using relative indexes. - /// - /// The first index identifies a list, and the second an item within the list. - /// The method adds the list's lower bound to the item index, and then calls - /// `get_abs`. Using absolute indexes within the list's bounds can be more - /// efficient than using relative indexing. pub fn get_rel(&self, list_idx: usize, item_idx: usize) -> (T::ReadItem<'_>, D::ReadItem<'_>) { self.get_abs(self.bounds(list_idx).0 + item_idx) } @@ -198,19 +163,9 @@ pub mod layers { /// Helper type for constructing `Upds` containers. pub struct UpdsBuilder { - /// Local stash of updates, to use for consolidation. - /// - /// We could emulate a `ChangeBatch` here, with related compaction smarts. - /// A `ChangeBatch` itself needs an `i64` diff type, which we have not. stash: Vec<(T::Owned, D::Owned)>, - /// Total number of consolidated updates. - /// - /// Tracked independently to account for duplicate compression. total: usize, - - /// Time container to stage singleton times for evaluation. time_con: T, - /// Diff container to stage singleton times for evaluation. diff_con: D, } @@ -218,7 +173,6 @@ pub mod layers { fn default() -> Self { Self { stash: Vec::default(), total: 0, time_con: BatchContainer::with_capacity(1), diff_con: BatchContainer::with_capacity(1) } } } - impl UpdsBuilder where T: BatchContainer, @@ -230,14 +184,10 @@ pub mod layers { } /// Consolidate and insert (if non-empty) the stashed updates. - /// - /// The return indicates whether the results were indeed non-empty. pub fn seal BatchContainer = usize>>(&mut self, upds: &mut Upds) -> bool { use crate::consolidation; consolidation::consolidate(&mut self.stash); - // If everything consolidates away, return false. if self.stash.is_empty() { return false; } - // If there is a singleton, we may be able to optimize. if self.stash.len() == 1 { let (time, diff) = self.stash.last().unwrap(); self.time_con.clear(); self.time_con.push_own(time); @@ -249,7 +199,6 @@ pub mod layers { return true; } } - // Conventional; move `stash` into `updates`. self.total += self.stash.len(); for (time, diff) in self.stash.drain(..) { upds.times.push_own(&time); @@ -268,66 +217,62 @@ pub mod layers { pub mod val_batch { use std::marker::PhantomData; - use serde::{Deserialize, Serialize}; - use timely::container::PushInto; + use timely::progress::{Antichain, frontier::AntichainRef}; use crate::trace::{Batch, BatchReader, Builder, Cursor, Description, Merger}; - use crate::trace::implementations::{BatchContainer, BuilderInput}; - use crate::trace::implementations::layout; - - use super::{Layout, Vals, Upds, layers::UpdsBuilder}; + use crate::trace::implementations::{BatchContainer, Coltainer, OffsetList}; + use super::{Update, Vals, Upds, layers::UpdsBuilder}; - /// An immutable collection of update tuples, from a contiguous interval of logical times. - #[derive(Debug, Serialize, Deserialize)] - #[serde(bound = " - L::KeyContainer: Serialize + for<'a> Deserialize<'a>, - L::ValContainer: Serialize + for<'a> Deserialize<'a>, - L::OffsetContainer: Serialize + for<'a> Deserialize<'a>, - L::TimeContainer: Serialize + for<'a> Deserialize<'a>, - L::DiffContainer: Serialize + for<'a> Deserialize<'a>, - ")] - pub struct OrdValStorage { + /// Storage for an ordered collection of `(key, val, time, diff)` updates. + pub struct OrdValStorage { /// An ordered list of keys. - pub keys: L::KeyContainer, + pub keys: Coltainer, /// For each key in `keys`, a list of values. - pub vals: Vals, + pub vals: Vals>, /// For each val in `vals`, a list of (time, diff) updates. - pub upds: Upds, + pub upds: Upds, Coltainer>, + } + + // Manual impls because derive can't handle the Update bound well. + impl Clone for OrdValStorage { + fn clone(&self) -> Self { + Self { keys: self.keys.clone(), vals: Vals { offs: self.vals.offs.clone(), vals: self.vals.vals.clone() }, upds: Upds { offs: self.upds.offs.clone(), times: self.upds.times.clone(), diffs: self.upds.diffs.clone() } } + } + } + impl Default for OrdValStorage { + fn default() -> Self { + Self { + keys: Default::default(), + vals: Default::default(), + upds: Default::default(), + } + } } /// An immutable collection of update tuples, from a contiguous interval of logical times. - /// - /// The `L` parameter captures how the updates should be laid out, and `C` determines which - /// merge batcher to select. - #[derive(Serialize, Deserialize)] - #[serde(bound = " - L::KeyContainer: Serialize + for<'a> Deserialize<'a>, - L::ValContainer: Serialize + for<'a> Deserialize<'a>, - L::OffsetContainer: Serialize + for<'a> Deserialize<'a>, - L::TimeContainer: Serialize + for<'a> Deserialize<'a>, - L::DiffContainer: Serialize + for<'a> Deserialize<'a>, - ")] - pub struct OrdValBatch { + pub struct OrdValBatch { /// The updates themselves. - pub storage: OrdValStorage, + pub storage: OrdValStorage, /// Description of the update times this layer represents. - pub description: Description>, + pub description: Description, /// The number of updates reflected in the batch. - /// - /// We track this separately from `storage` because due to the singleton optimization, - /// we may have many more updates than `storage.updates.len()`. It should equal that - /// length, plus the number of singleton optimizations employed. pub updates: usize, } - impl WithLayout for OrdValBatch { - type Layout = L; + impl Clone for OrdValBatch { + fn clone(&self) -> Self { + Self { storage: self.storage.clone(), description: self.description.clone(), updates: self.updates } + } } - impl BatchReader for OrdValBatch { + impl BatchReader for OrdValBatch { + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; - type Cursor = OrdValCursor; + type Cursor = OrdValCursor; fn cursor(&self) -> Self::Cursor { OrdValCursor { key_cursor: 0, @@ -335,62 +280,48 @@ pub mod val_batch { phantom: PhantomData, } } - fn len(&self) -> usize { - // Normally this would be `self.updates.len()`, but we have a clever compact encoding. - // Perhaps we should count such exceptions to the side, to provide a correct accounting. - self.updates - } - fn description(&self) -> &Description> { &self.description } + fn len(&self) -> usize { self.updates } + fn description(&self) -> &Description { &self.description } } - impl Batch for OrdValBatch { - type Merger = OrdValMerger; + impl Batch for OrdValBatch { + type Merger = OrdValMerger; - fn begin_merge(&self, other: &Self, compaction_frontier: AntichainRef>) -> Self::Merger { + fn begin_merge(&self, other: &Self, compaction_frontier: AntichainRef) -> Self::Merger { OrdValMerger::new(self, other, compaction_frontier) } - fn empty(lower: Antichain, upper: Antichain) -> Self { + fn empty(lower: Antichain, upper: Antichain) -> Self { use timely::progress::Timestamp; Self { storage: OrdValStorage { - keys: L::KeyContainer::with_capacity(0), + keys: Coltainer::default(), vals: Default::default(), upds: Default::default(), }, - description: Description::new(lower, upper, Antichain::from_elem(Self::Time::minimum())), + description: Description::new(lower, upper, Antichain::from_elem(U::Time::minimum())), updates: 0, } } } /// State for an in-progress merge. - pub struct OrdValMerger { - /// Key position to merge next in the first batch. + pub struct OrdValMerger { key_cursor1: usize, - /// Key position to merge next in the second batch. key_cursor2: usize, - /// result that we are currently assembling. - result: OrdValStorage, - /// description - description: Description>, - /// Staging area to consolidate owned times and diffs, before sealing. - staging: UpdsBuilder, + result: OrdValStorage, + description: Description, + staging: UpdsBuilder, Coltainer>, } - impl Merger> for OrdValMerger - where - OrdValBatch: Batch>, - { - fn new(batch1: &OrdValBatch, batch2: &OrdValBatch, compaction_frontier: AntichainRef>) -> Self { - + impl Merger> for OrdValMerger { + fn new(batch1: &OrdValBatch, batch2: &OrdValBatch, compaction_frontier: AntichainRef) -> Self { assert!(batch1.upper() == batch2.lower()); use crate::lattice::Lattice; let mut since = batch1.description().since().join(batch2.description().since()); since = since.join(&compaction_frontier.to_owned()); let description = Description::new(batch1.lower().clone(), batch2.upper().clone(), since); - let batch1 = &batch1.storage; let batch2 = &batch2.storage; @@ -398,7 +329,7 @@ pub mod val_batch { key_cursor1: 0, key_cursor2: 0, result: OrdValStorage { - keys: L::KeyContainer::merge_capacity(&batch1.keys, &batch2.keys), + keys: Coltainer::merge_capacity(&batch1.keys, &batch2.keys), vals: Vals::merge_capacity(&batch1.vals, &batch2.vals), upds: Upds::merge_capacity(&batch1.upds, &batch2.upds), }, @@ -406,28 +337,22 @@ pub mod val_batch { staging: UpdsBuilder::default(), } } - fn done(self) -> OrdValBatch { + fn done(self) -> OrdValBatch { OrdValBatch { updates: self.staging.total(), storage: self.result, description: self.description, } } - fn work(&mut self, source1: &OrdValBatch, source2: &OrdValBatch, fuel: &mut isize) { - - // An (incomplete) indication of the amount of work we've done so far. + fn work(&mut self, source1: &OrdValBatch, source2: &OrdValBatch, fuel: &mut isize) { let starting_updates = self.staging.total(); let mut effort = 0isize; - // While both mergees are still active, perform single-key merges. while self.key_cursor1 < source1.storage.keys.len() && self.key_cursor2 < source2.storage.keys.len() && effort < *fuel { self.merge_key(&source1.storage, &source2.storage); - // An (incomplete) accounting of the work we've done. effort = (self.staging.total() - starting_updates) as isize; } - // Merging is complete, and only copying remains. - // Key-by-key copying allows effort interruption, and compaction. while self.key_cursor1 < source1.storage.keys.len() && effort < *fuel { self.copy_key(&source1.storage, self.key_cursor1); self.key_cursor1 += 1; @@ -443,17 +368,8 @@ pub mod val_batch { } } - // Helper methods in support of merging batches. - impl OrdValMerger { - /// Copy the next key in `source`. - /// - /// The method extracts the key in `source` at `cursor`, and merges it in to `self`. - /// If the result does not wholly cancel, they key will be present in `self` with the - /// compacted values and updates. - /// - /// The caller should be certain to update the cursor, as this method does not do this. - fn copy_key(&mut self, source: &OrdValStorage, cursor: usize) { - // Capture the initial number of values to determine if the merge was ultimately non-empty. + impl OrdValMerger { + fn copy_key(&mut self, source: &OrdValStorage, cursor: usize) { let init_vals = self.result.vals.vals.len(); let (mut lower, upper) = source.vals.bounds(cursor); while lower < upper { @@ -463,18 +379,13 @@ pub mod val_batch { } lower += 1; } - - // If we have pushed any values, copy the key as well. if self.result.vals.vals.len() > init_vals { self.result.keys.push_ref(source.keys.index(cursor)); self.result.vals.offs.push_ref(self.result.vals.vals.len()); } } - /// Merge the next key in each of `source1` and `source2` into `self`, updating the appropriate cursors. - /// - /// This method only merges a single key. It applies all compaction necessary, and may result in no output - /// if the updates cancel either directly or after compaction. - fn merge_key(&mut self, source1: &OrdValStorage, source2: &OrdValStorage) { + + fn merge_key(&mut self, source1: &OrdValStorage, source2: &OrdValStorage) { use ::std::cmp::Ordering; match source1.keys.index(self.key_cursor1).cmp(&source2.keys.index(self.key_cursor2)) { Ordering::Less => { @@ -482,14 +393,12 @@ pub mod val_batch { self.key_cursor1 += 1; }, Ordering::Equal => { - // Keys are equal; must merge all values from both sources for this one key. let (lower1, upper1) = source1.vals.bounds(self.key_cursor1); let (lower2, upper2) = source2.vals.bounds(self.key_cursor2); if let Some(off) = self.merge_vals((source1, lower1, upper1), (source2, lower2, upper2)) { self.result.keys.push_ref(source1.keys.index(self.key_cursor1)); self.result.vals.offs.push_ref(off); } - // Increment cursors in either case; the keys are merged. self.key_cursor1 += 1; self.key_cursor2 += 1; }, @@ -499,25 +408,17 @@ pub mod val_batch { }, } } - /// Merge two ranges of values into `self`. - /// - /// If the compacted result contains values with non-empty updates, the function returns - /// an offset that should be recorded to indicate the upper extent of the result values. + fn merge_vals( &mut self, - (source1, mut lower1, upper1): (&OrdValStorage, usize, usize), - (source2, mut lower2, upper2): (&OrdValStorage, usize, usize), + (source1, mut lower1, upper1): (&OrdValStorage, usize, usize), + (source2, mut lower2, upper2): (&OrdValStorage, usize, usize), ) -> Option { - // Capture the initial number of values to determine if the merge was ultimately non-empty. let init_vals = self.result.vals.vals.len(); while lower1 < upper1 && lower2 < upper2 { - // We compare values, and fold in updates for the lowest values; - // if they are non-empty post-consolidation, we write the value. - // We could multi-way merge and it wouldn't be very complicated. use ::std::cmp::Ordering; match source1.vals.get_abs(lower1).cmp(&source2.vals.get_abs(lower2)) { Ordering::Less => { - // Extend stash by updates, with logical compaction applied. self.stash_updates_for_val(source1, lower1); if self.staging.seal(&mut self.result.upds) { self.result.vals.vals.push_ref(source1.vals.get_abs(lower1)); @@ -534,7 +435,6 @@ pub mod val_batch { lower2 += 1; }, Ordering::Greater => { - // Extend stash by updates, with logical compaction applied. self.stash_updates_for_val(source2, lower2); if self.staging.seal(&mut self.result.upds) { self.result.vals.vals.push_ref(source2.vals.get_abs(lower2)); @@ -543,7 +443,6 @@ pub mod val_batch { }, } } - // Merging is complete, but we may have remaining elements to push. while lower1 < upper1 { self.stash_updates_for_val(source1, lower1); if self.staging.seal(&mut self.result.upds) { @@ -559,7 +458,6 @@ pub mod val_batch { lower2 += 1; } - // Values being pushed indicate non-emptiness. if self.result.vals.vals.len() > init_vals { Some(self.result.vals.vals.len()) } else { @@ -567,54 +465,47 @@ pub mod val_batch { } } - /// Transfer updates for an indexed value in `source` into `self`, with compaction applied. - fn stash_updates_for_val(&mut self, source: &OrdValStorage, index: usize) { + fn stash_updates_for_val(&mut self, source: &OrdValStorage, index: usize) { let (lower, upper) = source.upds.bounds(index); for i in lower .. upper { - // NB: Here is where we would need to look back if `lower == upper`. let (time, diff) = source.upds.get_abs(i); use crate::lattice::Lattice; - let mut new_time: layout::Time = L::TimeContainer::into_owned(time); + let mut new_time: U::Time = ::into_owned(time); new_time.advance_by(self.description.since().borrow()); - self.staging.push(new_time, L::DiffContainer::into_owned(diff)); + self.staging.push(new_time, ::into_owned(diff)); } } } /// A cursor for navigating a single layer. - pub struct OrdValCursor { - /// Absolute position of the current key. + pub struct OrdValCursor { key_cursor: usize, - /// Absolute position of the current value. val_cursor: usize, - /// Phantom marker for Rust happiness. - phantom: PhantomData, - } - - use crate::trace::implementations::WithLayout; - impl WithLayout for OrdValCursor { - type Layout = L; + phantom: PhantomData, } - impl Cursor for OrdValCursor { + impl Cursor for OrdValCursor { + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; + type Storage = OrdValBatch; - type Storage = OrdValBatch; + fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { storage.storage.keys.get(self.key_cursor) } + fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { if self.val_valid(storage) { Some(self.val(storage)) } else { None } } - fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { storage.storage.keys.get(self.key_cursor) } - fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { if self.val_valid(storage) { Some(self.val(storage)) } else { None } } - - fn key<'a>(&self, storage: &'a OrdValBatch) -> Self::Key<'a> { storage.storage.keys.index(self.key_cursor) } - fn val<'a>(&self, storage: &'a OrdValBatch) -> Self::Val<'a> { storage.storage.vals.get_abs(self.val_cursor) } - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &OrdValBatch, mut logic: L2) { + fn key<'a>(&self, storage: &'a OrdValBatch) -> columnar::Ref<'a, Self::Key> { storage.storage.keys.index(self.key_cursor) } + fn val<'a>(&self, storage: &'a OrdValBatch) -> columnar::Ref<'a, Self::Val> { storage.storage.vals.get_abs(self.val_cursor) } + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &OrdValBatch, mut logic: L2) { let (lower, upper) = storage.storage.upds.bounds(self.val_cursor); for index in lower .. upper { let (time, diff) = storage.storage.upds.get_abs(index); logic(time, diff); } } - fn key_valid(&self, storage: &OrdValBatch) -> bool { self.key_cursor < storage.storage.keys.len() } - fn val_valid(&self, storage: &OrdValBatch) -> bool { self.val_cursor < storage.storage.vals.bounds(self.key_cursor).1 } - fn step_key(&mut self, storage: &OrdValBatch){ + fn key_valid(&self, storage: &OrdValBatch) -> bool { self.key_cursor < storage.storage.keys.len() } + fn val_valid(&self, storage: &OrdValBatch) -> bool { self.val_cursor < storage.storage.vals.bounds(self.key_cursor).1 } + fn step_key(&mut self, storage: &OrdValBatch){ self.key_cursor += 1; if self.key_valid(storage) { self.rewind_vals(storage); @@ -623,102 +514,83 @@ pub mod val_batch { self.key_cursor = storage.storage.keys.len(); } } - fn seek_key(&mut self, storage: &OrdValBatch, key: Self::Key<'_>) { - self.key_cursor += storage.storage.keys.advance(self.key_cursor, storage.storage.keys.len(), |x| ::reborrow(x).lt(&::reborrow(key))); + fn seek_key(&mut self, storage: &OrdValBatch, key: columnar::Ref<'_, Self::Key>) { + self.key_cursor += storage.storage.keys.advance(self.key_cursor, storage.storage.keys.len(), |x| Coltainer::::reborrow(x).lt(&Coltainer::::reborrow(key))); if self.key_valid(storage) { self.rewind_vals(storage); } } - fn step_val(&mut self, storage: &OrdValBatch) { + fn step_val(&mut self, storage: &OrdValBatch) { self.val_cursor += 1; if !self.val_valid(storage) { self.val_cursor = storage.storage.vals.bounds(self.key_cursor).1; } } - fn seek_val(&mut self, storage: &OrdValBatch, val: Self::Val<'_>) { - self.val_cursor += storage.storage.vals.vals.advance(self.val_cursor, storage.storage.vals.bounds(self.key_cursor).1, |x| ::reborrow(x).lt(&::reborrow(val))); + fn seek_val(&mut self, storage: &OrdValBatch, val: columnar::Ref<'_, Self::Val>) { + self.val_cursor += storage.storage.vals.vals.advance(self.val_cursor, storage.storage.vals.bounds(self.key_cursor).1, |x| Coltainer::::reborrow(x).lt(&Coltainer::::reborrow(val))); } - fn rewind_keys(&mut self, storage: &OrdValBatch) { + fn rewind_keys(&mut self, storage: &OrdValBatch) { self.key_cursor = 0; if self.key_valid(storage) { self.rewind_vals(storage) } } - fn rewind_vals(&mut self, storage: &OrdValBatch) { + fn rewind_vals(&mut self, storage: &OrdValBatch) { self.val_cursor = storage.storage.vals.bounds(self.key_cursor).0; } } - /// A builder for creating layers from unsorted update tuples. - pub struct OrdValBuilder { + /// A builder for creating layers from sorted update tuples. + pub struct OrdValBuilder { /// The in-progress result. - /// - /// This is public to allow container implementors to set and inspect their container. - pub result: OrdValStorage, - staging: UpdsBuilder, - _marker: PhantomData, + pub result: OrdValStorage, + staging: UpdsBuilder, Coltainer>, } - impl Builder for OrdValBuilder - where - L: for<'a> Layout< - KeyContainer: PushInto>, - ValContainer: PushInto>, - >, - CI: for<'a> BuilderInput, Diff=layout::Diff>, - { - - type Input = CI; - type Time = layout::Time; - type Output = OrdValBatch; + impl Builder for OrdValBuilder { + type Input = Vec<((U::Key, U::Val), U::Time, U::Diff)>; + type Time = U::Time; + type Output = OrdValBatch; fn with_capacity(keys: usize, vals: usize, upds: usize) -> Self { Self { result: OrdValStorage { - keys: L::KeyContainer::with_capacity(keys), + keys: Coltainer::with_capacity(keys), vals: Vals::with_capacity(keys + 1, vals), upds: Upds::with_capacity(vals + 1, upds), }, staging: UpdsBuilder::default(), - _marker: PhantomData, } } #[inline] fn push(&mut self, chunk: &mut Self::Input) { - for item in chunk.drain() { - let (key, val, time, diff) = CI::into_parts(item); - - // Pre-load the first update. + for ((key, val), time, diff) in chunk.drain(..) { if self.result.keys.is_empty() { - self.result.vals.vals.push_into(val); - self.result.keys.push_into(key); + self.result.keys.push_own(&key); + self.result.vals.vals.push_own(&val); self.staging.push(time, diff); } - // Perhaps this is a continuation of an already received key. - else if self.result.keys.last().map(|k| CI::key_eq(&key, k)).unwrap_or(false) { - // Perhaps this is a continuation of an already received value. - if self.result.vals.vals.last().map(|v| CI::val_eq(&val, v)).unwrap_or(false) { + else if self.result.keys.last().map(|k| Coltainer::::into_owned(k) == key).unwrap_or(false) { + if self.result.vals.vals.last().map(|v| Coltainer::::into_owned(v) == val).unwrap_or(false) { self.staging.push(time, diff); } else { - // New value; complete representation of prior value. self.staging.seal(&mut self.result.upds); self.staging.push(time, diff); - self.result.vals.vals.push_into(val); + self.result.vals.vals.push_own(&val); } } else { - // New key; complete representation of prior key. self.staging.seal(&mut self.result.upds); self.staging.push(time, diff); self.result.vals.offs.push_ref(self.result.vals.vals.len()); - self.result.vals.vals.push_into(val); - self.result.keys.push_into(key); + self.result.vals.vals.push_own(&val); + self.result.keys.push_own(&key); } } } #[inline(never)] - fn done(mut self, description: Description) -> OrdValBatch { + fn done(mut self, description: Description) -> OrdValBatch { self.staging.seal(&mut self.result.upds); self.result.vals.offs.push_ref(self.result.vals.vals.len()); OrdValBatch { @@ -729,12 +601,24 @@ pub mod val_batch { } fn seal(chain: &mut Vec, description: Description) -> Self::Output { - let (keys, vals, upds) = Self::Input::key_val_upd_counts(&chain[..]); + let mut keys = 0; + let mut vals = 0; + let mut upds = 0; + for link in chain.iter() { + let mut prev_keyval = None; + for ((key, val), _, _) in link.iter() { + if let Some((p_key, p_val)) = prev_keyval { + if p_key != key { keys += 1; vals += 1; } + else if p_val != val { vals += 1; } + } else { keys += 1; vals += 1; } + upds += 1; + prev_keyval = Some((key, val)); + } + } let mut builder = Self::with_capacity(keys, vals, upds); for mut chunk in chain.drain(..) { builder.push(&mut chunk); } - builder.done(description) } } @@ -744,105 +628,87 @@ pub mod val_batch { pub mod key_batch { use std::marker::PhantomData; - use serde::{Deserialize, Serialize}; - use timely::container::PushInto; + use timely::progress::{Antichain, frontier::AntichainRef}; use crate::trace::{Batch, BatchReader, Builder, Cursor, Description, Merger}; - use crate::trace::implementations::{BatchContainer, BuilderInput}; - use crate::trace::implementations::layout; - - use super::{Layout, Upds, layers::UpdsBuilder}; + use crate::trace::implementations::{BatchContainer, Coltainer, OffsetList}; + use super::{Update, Upds, layers::UpdsBuilder}; - /// An immutable collection of update tuples, from a contiguous interval of logical times. - #[derive(Debug, Serialize, Deserialize)] - #[serde(bound = " - L::KeyContainer: Serialize + for<'a> Deserialize<'a>, - L::OffsetContainer: Serialize + for<'a> Deserialize<'a>, - L::TimeContainer: Serialize + for<'a> Deserialize<'a>, - L::DiffContainer: Serialize + for<'a> Deserialize<'a>, - ")] - pub struct OrdKeyStorage { - /// An ordered list of keys, corresponding to entries in `keys_offs`. - pub keys: L::KeyContainer, + /// Storage for an ordered collection of `(key, time, diff)` updates. + pub struct OrdKeyStorage { + /// An ordered list of keys. + pub keys: Coltainer, /// For each key in `keys`, a list of (time, diff) updates. - pub upds: Upds, + pub upds: Upds, Coltainer>, + } + + impl Clone for OrdKeyStorage { + fn clone(&self) -> Self { + Self { keys: self.keys.clone(), upds: Upds { offs: self.upds.offs.clone(), times: self.upds.times.clone(), diffs: self.upds.diffs.clone() } } + } + } + impl Default for OrdKeyStorage { + fn default() -> Self { Self { keys: Default::default(), upds: Default::default() } } } /// An immutable collection of update tuples, from a contiguous interval of logical times. - /// - /// The `L` parameter captures how the updates should be laid out, and `C` determines which - /// merge batcher to select. - #[derive(Serialize, Deserialize)] - #[serde(bound = " - L::KeyContainer: Serialize + for<'a> Deserialize<'a>, - L::ValContainer: Serialize + for<'a> Deserialize<'a>, - L::OffsetContainer: Serialize + for<'a> Deserialize<'a>, - L::TimeContainer: Serialize + for<'a> Deserialize<'a>, - L::DiffContainer: Serialize + for<'a> Deserialize<'a>, - ")] - pub struct OrdKeyBatch { + pub struct OrdKeyBatch { /// The updates themselves. - pub storage: OrdKeyStorage, + pub storage: OrdKeyStorage, /// Description of the update times this layer represents. - pub description: Description>, + pub description: Description, /// The number of updates reflected in the batch. - /// - /// We track this separately from `storage` because due to the singleton optimization, - /// we may have many more updates than `storage.updates.len()`. It should equal that - /// length, plus the number of singleton optimizations employed. pub updates: usize, + /// Single value to return if asked (for key-only batches, val is always `()`). + pub value: Coltainer, + } - /// Single value to return if asked. - pub value: L::ValContainer, + impl Clone for OrdKeyBatch { + fn clone(&self) -> Self { + Self { storage: self.storage.clone(), description: self.description.clone(), updates: self.updates, value: self.value.clone() } + } } - impl>> OrdKeyBatch { + impl> OrdKeyBatch { /// Creates a container with one value, to slot in to `self.value`. - pub fn create_value() -> L::ValContainer { - let mut value = L::ValContainer::with_capacity(1); + pub fn create_value() -> Coltainer { + let mut value = Coltainer::::with_capacity(1); value.push_own(&Default::default()); value } } - impl>> WithLayout for OrdKeyBatch { - type Layout = L; - } + impl> BatchReader for OrdKeyBatch { + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; - impl>> BatchReader for OrdKeyBatch { - - type Cursor = OrdKeyCursor; + type Cursor = OrdKeyCursor; fn cursor(&self) -> Self::Cursor { OrdKeyCursor { key_cursor: 0, val_stepped: false, - phantom: std::marker::PhantomData, + phantom: PhantomData, } } - fn len(&self) -> usize { - // Normally this would be `self.updates.len()`, but we have a clever compact encoding. - // Perhaps we should count such exceptions to the side, to provide a correct accounting. - self.updates - } - fn description(&self) -> &Description> { &self.description } + fn len(&self) -> usize { self.updates } + fn description(&self) -> &Description { &self.description } } - impl>> Batch for OrdKeyBatch { - type Merger = OrdKeyMerger; + impl> Batch for OrdKeyBatch { + type Merger = OrdKeyMerger; - fn begin_merge(&self, other: &Self, compaction_frontier: AntichainRef>) -> Self::Merger { + fn begin_merge(&self, other: &Self, compaction_frontier: AntichainRef) -> Self::Merger { OrdKeyMerger::new(self, other, compaction_frontier) } - fn empty(lower: Antichain, upper: Antichain) -> Self { + fn empty(lower: Antichain, upper: Antichain) -> Self { use timely::progress::Timestamp; Self { - storage: OrdKeyStorage { - keys: L::KeyContainer::with_capacity(0), - upds: Upds::default(), - }, - description: Description::new(lower, upper, Antichain::from_elem(Self::Time::minimum())), + storage: OrdKeyStorage::default(), + description: Description::new(lower, upper, Antichain::from_elem(U::Time::minimum())), updates: 0, value: Self::create_value(), } @@ -850,33 +716,21 @@ pub mod key_batch { } /// State for an in-progress merge. - pub struct OrdKeyMerger { - /// Key position to merge next in the first batch. + pub struct OrdKeyMerger { key_cursor1: usize, - /// Key position to merge next in the second batch. key_cursor2: usize, - /// result that we are currently assembling. - result: OrdKeyStorage, - /// description - description: Description>, - - /// Local stash of updates, to use for consolidation. - staging: UpdsBuilder, + result: OrdKeyStorage, + description: Description, + staging: UpdsBuilder, Coltainer>, } - impl>> Merger> for OrdKeyMerger - where - OrdKeyBatch: Batch>, - { - fn new(batch1: &OrdKeyBatch, batch2: &OrdKeyBatch, compaction_frontier: AntichainRef>) -> Self { - + impl> Merger> for OrdKeyMerger { + fn new(batch1: &OrdKeyBatch, batch2: &OrdKeyBatch, compaction_frontier: AntichainRef) -> Self { assert!(batch1.upper() == batch2.lower()); use crate::lattice::Lattice; let mut since = batch1.description().since().join(batch2.description().since()); since = since.join(&compaction_frontier.to_owned()); - let description = Description::new(batch1.lower().clone(), batch2.upper().clone(), since); - let batch1 = &batch1.storage; let batch2 = &batch2.storage; @@ -884,36 +738,30 @@ pub mod key_batch { key_cursor1: 0, key_cursor2: 0, result: OrdKeyStorage { - keys: L::KeyContainer::merge_capacity(&batch1.keys, &batch2.keys), + keys: Coltainer::merge_capacity(&batch1.keys, &batch2.keys), upds: Upds::merge_capacity(&batch1.upds, &batch2.upds), }, description, staging: UpdsBuilder::default(), } } - fn done(self) -> OrdKeyBatch { + fn done(self) -> OrdKeyBatch { OrdKeyBatch { updates: self.staging.total(), storage: self.result, description: self.description, - value: OrdKeyBatch::::create_value(), + value: OrdKeyBatch::::create_value(), } } - fn work(&mut self, source1: &OrdKeyBatch, source2: &OrdKeyBatch, fuel: &mut isize) { - - // An (incomplete) indication of the amount of work we've done so far. + fn work(&mut self, source1: &OrdKeyBatch, source2: &OrdKeyBatch, fuel: &mut isize) { let starting_updates = self.staging.total(); let mut effort = 0isize; - // While both mergees are still active, perform single-key merges. while self.key_cursor1 < source1.storage.keys.len() && self.key_cursor2 < source2.storage.keys.len() && effort < *fuel { self.merge_key(&source1.storage, &source2.storage); - // An (incomplete) accounting of the work we've done. effort = (self.staging.total() - starting_updates) as isize; } - // Merging is complete, and only copying remains. - // Key-by-key copying allows effort interruption, and compaction. while self.key_cursor1 < source1.storage.keys.len() && effort < *fuel { self.copy_key(&source1.storage, self.key_cursor1); self.key_cursor1 += 1; @@ -929,26 +777,15 @@ pub mod key_batch { } } - // Helper methods in support of merging batches. - impl OrdKeyMerger { - /// Copy the next key in `source`. - /// - /// The method extracts the key in `source` at `cursor`, and merges it in to `self`. - /// If the result does not wholly cancel, they key will be present in `self` with the - /// compacted values and updates. - /// - /// The caller should be certain to update the cursor, as this method does not do this. - fn copy_key(&mut self, source: &OrdKeyStorage, cursor: usize) { + impl OrdKeyMerger { + fn copy_key(&mut self, source: &OrdKeyStorage, cursor: usize) { self.stash_updates_for_key(source, cursor); if self.staging.seal(&mut self.result.upds) { self.result.keys.push_ref(source.keys.index(cursor)); } } - /// Merge the next key in each of `source1` and `source2` into `self`, updating the appropriate cursors. - /// - /// This method only merges a single key. It applies all compaction necessary, and may result in no output - /// if the updates cancel either directly or after compaction. - fn merge_key(&mut self, source1: &OrdKeyStorage, source2: &OrdKeyStorage) { + + fn merge_key(&mut self, source1: &OrdKeyStorage, source2: &OrdKeyStorage) { use ::std::cmp::Ordering; match source1.keys.index(self.key_cursor1).cmp(&source2.keys.index(self.key_cursor2)) { Ordering::Less => { @@ -956,13 +793,11 @@ pub mod key_batch { self.key_cursor1 += 1; }, Ordering::Equal => { - // Keys are equal; must merge all updates from both sources for this one key. self.stash_updates_for_key(source1, self.key_cursor1); self.stash_updates_for_key(source2, self.key_cursor2); if self.staging.seal(&mut self.result.upds) { self.result.keys.push_ref(source1.keys.index(self.key_cursor1)); } - // Increment cursors in either case; the keys are merged. self.key_cursor1 += 1; self.key_cursor2 += 1; }, @@ -973,54 +808,47 @@ pub mod key_batch { } } - /// Transfer updates for an indexed value in `source` into `self`, with compaction applied. - fn stash_updates_for_key(&mut self, source: &OrdKeyStorage, index: usize) { + fn stash_updates_for_key(&mut self, source: &OrdKeyStorage, index: usize) { let (lower, upper) = source.upds.bounds(index); for i in lower .. upper { - // NB: Here is where we would need to look back if `lower == upper`. let (time, diff) = source.upds.get_abs(i); use crate::lattice::Lattice; - let mut new_time = L::TimeContainer::into_owned(time); + let mut new_time: U::Time = ::into_owned(time); new_time.advance_by(self.description.since().borrow()); - self.staging.push(new_time, L::DiffContainer::into_owned(diff)); + self.staging.push(new_time, ::into_owned(diff)); } } } /// A cursor for navigating a single layer. - pub struct OrdKeyCursor { - /// Absolute position of the current key. + pub struct OrdKeyCursor { key_cursor: usize, - /// If the value has been stepped for the key, there are no more values. val_stepped: bool, - /// Phantom marker for Rust happiness. - phantom: PhantomData, - } - - use crate::trace::implementations::WithLayout; - impl> WithLayout for OrdKeyCursor { - type Layout = L; + phantom: PhantomData, } - impl Layout>> Cursor for OrdKeyCursor { - - type Storage = OrdKeyBatch; + impl> Cursor for OrdKeyCursor { + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; + type Storage = OrdKeyBatch; - fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { storage.storage.keys.get(self.key_cursor) } - fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { if self.val_valid(storage) { Some(self.val(storage)) } else { None } } + fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { storage.storage.keys.get(self.key_cursor) } + fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { if self.val_valid(storage) { Some(self.val(storage)) } else { None } } - fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { storage.storage.keys.index(self.key_cursor) } - fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { storage.value.index(0) } - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, mut logic: L2) { + fn key<'a>(&self, storage: &'a OrdKeyBatch) -> columnar::Ref<'a, Self::Key> { storage.storage.keys.index(self.key_cursor) } + fn val<'a>(&self, storage: &'a OrdKeyBatch) -> columnar::Ref<'a, Self::Val> { storage.value.index(0) } + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &OrdKeyBatch, mut logic: L2) { let (lower, upper) = storage.storage.upds.bounds(self.key_cursor); for index in lower .. upper { let (time, diff) = storage.storage.upds.get_abs(index); logic(time, diff); } } - fn key_valid(&self, storage: &Self::Storage) -> bool { self.key_cursor < storage.storage.keys.len() } - fn val_valid(&self, _storage: &Self::Storage) -> bool { !self.val_stepped } - fn step_key(&mut self, storage: &Self::Storage){ + fn key_valid(&self, storage: &OrdKeyBatch) -> bool { self.key_cursor < storage.storage.keys.len() } + fn val_valid(&self, _storage: &OrdKeyBatch) -> bool { !self.val_stepped } + fn step_key(&mut self, storage: &OrdKeyBatch){ self.key_cursor += 1; if self.key_valid(storage) { self.rewind_vals(storage); @@ -1029,98 +857,93 @@ pub mod key_batch { self.key_cursor = storage.storage.keys.len(); } } - fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { - self.key_cursor += storage.storage.keys.advance(self.key_cursor, storage.storage.keys.len(), |x| ::reborrow(x).lt(&::reborrow(key))); + fn seek_key(&mut self, storage: &OrdKeyBatch, key: columnar::Ref<'_, Self::Key>) { + self.key_cursor += storage.storage.keys.advance(self.key_cursor, storage.storage.keys.len(), |x| Coltainer::::reborrow(x).lt(&Coltainer::::reborrow(key))); if self.key_valid(storage) { self.rewind_vals(storage); } } - fn step_val(&mut self, _storage: &Self::Storage) { + fn step_val(&mut self, _storage: &OrdKeyBatch) { self.val_stepped = true; } - fn seek_val(&mut self, _storage: &Self::Storage, _val: Self::Val<'_>) { } - fn rewind_keys(&mut self, storage: &Self::Storage) { + fn seek_val(&mut self, _storage: &OrdKeyBatch, _val: columnar::Ref<'_, Self::Val>) { } + fn rewind_keys(&mut self, storage: &OrdKeyBatch) { self.key_cursor = 0; if self.key_valid(storage) { self.rewind_vals(storage) } } - fn rewind_vals(&mut self, _storage: &Self::Storage) { + fn rewind_vals(&mut self, _storage: &OrdKeyBatch) { self.val_stepped = false; } } - /// A builder for creating layers from unsorted update tuples. - pub struct OrdKeyBuilder { + /// A builder for creating layers from sorted update tuples. + pub struct OrdKeyBuilder { /// The in-progress result. - /// - /// This is public to allow container implementors to set and inspect their container. - pub result: OrdKeyStorage, - staging: UpdsBuilder, - _marker: PhantomData, + pub result: OrdKeyStorage, + staging: UpdsBuilder, Coltainer>, } - impl Builder for OrdKeyBuilder - where - L: for<'a> Layout>>, - L: Layout>, - CI: BuilderInput, Diff=layout::Diff>, - { - - type Input = CI; - type Time = layout::Time; - type Output = OrdKeyBatch; + impl> Builder for OrdKeyBuilder { + type Input = Vec<((U::Key, U::Val), U::Time, U::Diff)>; + type Time = U::Time; + type Output = OrdKeyBatch; fn with_capacity(keys: usize, _vals: usize, upds: usize) -> Self { Self { result: OrdKeyStorage { - keys: L::KeyContainer::with_capacity(keys), + keys: Coltainer::with_capacity(keys), upds: Upds::with_capacity(keys+1, upds), }, staging: UpdsBuilder::default(), - _marker: PhantomData, } } #[inline] fn push(&mut self, chunk: &mut Self::Input) { - for item in chunk.drain() { - let (key, _val, time, diff) = CI::into_parts(item); + for ((key, _val), time, diff) in chunk.drain(..) { if self.result.keys.is_empty() { - self.result.keys.push_into(key); + self.result.keys.push_own(&key); self.staging.push(time, diff); } - // Perhaps this is a continuation of an already received key. - else if self.result.keys.last().map(|k| CI::key_eq(&key, k)).unwrap_or(false) { + else if self.result.keys.last().map(|k| Coltainer::::into_owned(k) == key).unwrap_or(false) { self.staging.push(time, diff); } else { self.staging.seal(&mut self.result.upds); self.staging.push(time, diff); - self.result.keys.push_into(key); + self.result.keys.push_own(&key); } } } #[inline(never)] - fn done(mut self, description: Description) -> OrdKeyBatch { + fn done(mut self, description: Description) -> OrdKeyBatch { self.staging.seal(&mut self.result.upds); OrdKeyBatch { updates: self.staging.total(), storage: self.result, description, - value: OrdKeyBatch::::create_value(), + value: OrdKeyBatch::::create_value(), } } fn seal(chain: &mut Vec, description: Description) -> Self::Output { - let (keys, vals, upds) = Self::Input::key_val_upd_counts(&chain[..]); - let mut builder = Self::with_capacity(keys, vals, upds); + let mut keys = 0; + let mut upds = 0; + for link in chain.iter() { + let mut prev_key = None; + for ((key, _), _, _) in link.iter() { + if prev_key.map(|pk| pk != key).unwrap_or(true) { keys += 1; } + upds += 1; + prev_key = Some(key); + } + } + let mut builder = Self::with_capacity(keys, 0, upds); for mut chunk in chain.drain(..) { builder.push(&mut chunk); } - builder.done(description) } } - } diff --git a/differential-dataflow/src/trace/implementations/rhh.rs b/differential-dataflow/src/trace/implementations/rhh.rs index 71d20b69d..ec8432631 100644 --- a/differential-dataflow/src/trace/implementations/rhh.rs +++ b/differential-dataflow/src/trace/implementations/rhh.rs @@ -11,35 +11,21 @@ use std::cmp::Ordering; use serde::{Deserialize, Serialize}; use crate::Hashable; -use crate::containers::TimelyStack; -use crate::trace::implementations::chunker::{ColumnationChunker, VecChunker}; -use crate::trace::implementations::merge_batcher::{MergeBatcher, VecMerger, ColMerger}; +use crate::trace::implementations::chunker::VecChunker; +use crate::trace::implementations::merge_batcher::{MergeBatcher, VecMerger}; use crate::trace::implementations::spine_fueled::Spine; use crate::trace::rc_blanket_impls::RcBuilder; -use super::{Layout, Vector, TStack}; +use super::Update; use self::val_batch::{RhhValBatch, RhhValBuilder}; /// A trace implementation using a spine of ordered lists. -pub type VecSpine = Spine>>>; +pub type VecSpine = Spine>>; /// A batcher for ordered lists. pub type VecBatcher = MergeBatcher, VecChunker<((K,V),T,R)>, VecMerger<(K, V), T, R>>; /// A builder for ordered lists. -pub type VecBuilder = RcBuilder, Vec<((K,V),T,R)>>>; - -// /// A trace implementation for empty values using a spine of ordered lists. -// pub type OrdKeySpine = Spine>>>; - -/// A trace implementation backed by columnar storage. -pub type ColSpine = Spine>>>; -/// A batcher for columnar storage. -pub type ColBatcher = MergeBatcher, ColumnationChunker<((K,V),T,R)>, ColMerger<(K,V),T,R>>; -/// A builder for columnar storage. -pub type ColBuilder = RcBuilder, TimelyStack<((K,V),T,R)>>>; - -// /// A trace implementation backed by columnar storage. -// pub type ColKeySpine = Spine>>>; +pub type VecBuilder = RcBuilder>; /// A carrier trait indicating that the type's `Ord` and `PartialOrd` implementations are by `Hashable::hashed()`. pub trait HashOrdered: Hashable { } @@ -85,16 +71,14 @@ mod val_batch { use std::convert::TryInto; use std::marker::PhantomData; - use serde::{Deserialize, Serialize}; - use timely::container::PushInto; + use timely::progress::{Antichain, frontier::AntichainRef}; use crate::hashable::Hashable; use crate::trace::{Batch, BatchReader, Builder, Cursor, Description, Merger}; - use crate::trace::implementations::{BatchContainer, BuilderInput}; - use crate::trace::implementations::layout; + use crate::trace::implementations::{BatchContainer, Coltainer, OffsetList}; - use super::{Layout, HashOrdered}; + use super::{Update, HashOrdered}; /// Update tuples organized as a Robin Hood Hash map, ordered by `(hash(Key), Key, Val, Time)`. /// @@ -110,10 +94,9 @@ mod val_batch { /// We will use the `Hashable` trait here, but any consistent hash function should work out ok. /// We specifically want to use the highest bits of the result (we will) because the low bits have /// likely been spent shuffling the data between workers (by key), and are likely low entropy. - #[derive(Debug, Serialize, Deserialize)] - pub struct RhhValStorage + pub struct RhhValStorage where - layout::Key: Default + HashOrdered, + U::Key: Default + HashOrdered, { /// The requested capacity for `keys`. We use this when determining where a key with a certain hash @@ -127,13 +110,13 @@ mod val_batch { pub key_count: usize, /// An ordered list of keys, corresponding to entries in `keys_offs`. - pub keys: L::KeyContainer, + pub keys: Coltainer, /// Offsets used to provide indexes from keys to values. /// /// The length of this list is one longer than `keys`, so that we can avoid bounds logic. - pub keys_offs: L::OffsetContainer, + pub keys_offs: OffsetList, /// Concatenated ordered lists of values, bracketed by offsets in `keys_offs`. - pub vals: L::ValContainer, + pub vals: Coltainer, /// Offsets used to provide indexes from values to updates. /// /// This list has a special representation that any empty range indicates the singleton @@ -142,17 +125,17 @@ mod val_batch { /// single common update values (e.g. in a snapshot, the minimal time and a diff of one). /// /// The length of this list is one longer than `vals`, so that we can avoid bounds logic. - pub vals_offs: L::OffsetContainer, + pub vals_offs: OffsetList, /// Concatenated ordered lists of update times, bracketed by offsets in `vals_offs`. - pub times: L::TimeContainer, + pub times: Coltainer, /// Concatenated ordered lists of update diffs, bracketed by offsets in `vals_offs`. - pub diffs: L::DiffContainer, + pub diffs: Coltainer, } - impl RhhValStorage + impl RhhValStorage where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> columnar::Ref<'a, U::Key>: HashOrdered, { /// Lower and upper bounds in `self.vals` corresponding to the key at `index`. fn values_for_key(&self, index: usize) -> (usize, usize) { @@ -184,14 +167,14 @@ mod val_batch { /// If `offset` is specified, we will insert it at the appropriate location. If it is not specified, /// we leave `keys_offs` ready to receive it as the next `push`. This is so that builders that may /// not know the final offset at the moment of key insertion can prepare for receiving the offset. - fn insert_key(&mut self, key: layout::KeyRef<'_, L>, offset: Option) { + fn insert_key(&mut self, key: columnar::Ref<'_, U::Key>, offset: Option) { let desired = self.desired_location(&key); // Were we to push the key now, it would be at `self.keys.len()`, so while that is wrong, // push additional blank entries in. while self.keys.len() < desired { // We insert a default (dummy) key and repeat the offset to indicate this. let current_offset = self.keys_offs.index(self.keys.len()); - self.keys.push_own(& as Default>::default()); + self.keys.push_own(&::default()); self.keys_offs.push_ref(current_offset); } @@ -205,8 +188,8 @@ mod val_batch { } /// Inserts a reference to an owned key, inefficiently. Should be removed. - fn insert_key_own(&mut self, key: &layout::Key, offset: Option) { - let mut key_con = L::KeyContainer::with_capacity(1); + fn insert_key_own(&mut self, key: &U::Key, offset: Option) { + let mut key_con = Coltainer::::with_capacity(1); key_con.push_own(&key); self.insert_key(key_con.index(0), offset) } @@ -220,9 +203,9 @@ mod val_batch { } /// Returns true if one should advance one's index in the search for `key`. - fn advance_key(&self, index: usize, key: layout::KeyRef<'_, L>) -> bool { + fn advance_key(&self, index: usize, key: columnar::Ref<'_, U::Key>) -> bool { // Ideally this short-circuits, as `self.keys[index]` is bogus data. - !self.live_key(index) || self.keys.index(index).lt(&::reborrow(key)) + !self.live_key(index) || self.keys.index(index).lt(&Coltainer::::reborrow(key)) } /// Indicates that a key is valid, rather than dead space, by looking for a valid offset range. @@ -254,24 +237,15 @@ mod val_batch { /// An immutable collection of update tuples, from a contiguous interval of logical times. /// - /// The `L` parameter captures how the updates should be laid out, and `C` determines which - /// merge batcher to select. - #[derive(Serialize, Deserialize)] - #[serde(bound = " - L::KeyContainer: Serialize + for<'a> Deserialize<'a>, - L::ValContainer: Serialize + for<'a> Deserialize<'a>, - L::OffsetContainer: Serialize + for<'a> Deserialize<'a>, - L::TimeContainer: Serialize + for<'a> Deserialize<'a>, - L::DiffContainer: Serialize + for<'a> Deserialize<'a>, - ")] - pub struct RhhValBatch + /// The `U` parameter captures the update type (key, val, time, diff). + pub struct RhhValBatch where - layout::Key: Default + HashOrdered, + U::Key: Default + HashOrdered, { /// The updates themselves. - pub storage: RhhValStorage, + pub storage: RhhValStorage, /// Description of the update times this layer represents. - pub description: Description>, + pub description: Description, /// The number of updates reflected in the batch. /// /// We track this separately from `storage` because due to the singleton optimization, @@ -280,20 +254,17 @@ mod val_batch { pub updates: usize, } - impl WithLayout for RhhValBatch + impl BatchReader for RhhValBatch where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> columnar::Ref<'a, U::Key>: HashOrdered, { - type Layout = L; - } + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; - impl BatchReader for RhhValBatch - where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, - { - type Cursor = RhhValCursor; + type Cursor = RhhValCursor; fn cursor(&self) -> Self::Cursor { let mut cursor = RhhValCursor { key_cursor: 0, @@ -308,17 +279,17 @@ mod val_batch { // Perhaps we should count such exceptions to the side, to provide a correct accounting. self.updates } - fn description(&self) -> &Description> { &self.description } + fn description(&self) -> &Description { &self.description } } - impl Batch for RhhValBatch + impl Batch for RhhValBatch where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> columnar::Ref<'a, U::Key>: HashOrdered, { - type Merger = RhhValMerger; + type Merger = RhhValMerger; - fn begin_merge(&self, other: &Self, compaction_frontier: AntichainRef>) -> Self::Merger { + fn begin_merge(&self, other: &Self, compaction_frontier: AntichainRef) -> Self::Merger { RhhValMerger::new(self, other, compaction_frontier) } @@ -326,12 +297,12 @@ mod val_batch { use timely::progress::Timestamp; Self { storage: RhhValStorage { - keys: L::KeyContainer::with_capacity(0), - keys_offs: L::OffsetContainer::with_capacity(0), - vals: L::ValContainer::with_capacity(0), - vals_offs: L::OffsetContainer::with_capacity(0), - times: L::TimeContainer::with_capacity(0), - diffs: L::DiffContainer::with_capacity(0), + keys: Coltainer::::with_capacity(0), + keys_offs: OffsetList::with_capacity(0), + vals: Coltainer::::with_capacity(0), + vals_offs: OffsetList::with_capacity(0), + times: Coltainer::::with_capacity(0), + diffs: Coltainer::::with_capacity(0), key_count: 0, key_capacity: 0, divisor: 0, @@ -343,35 +314,34 @@ mod val_batch { } /// State for an in-progress merge. - pub struct RhhValMerger + pub struct RhhValMerger where - layout::Key: Default + HashOrdered, + U::Key: Default + HashOrdered, { /// Key position to merge next in the first batch. key_cursor1: usize, /// Key position to merge next in the second batch. key_cursor2: usize, /// result that we are currently assembling. - result: RhhValStorage, + result: RhhValStorage, /// description - description: Description>, + description: Description, /// Local stash of updates, to use for consolidation. /// /// We could emulate a `ChangeBatch` here, with related compaction smarts. /// A `ChangeBatch` itself needs an `i64` diff type, which we have not. - update_stash: Vec<(layout::Time, layout::Diff)>, + update_stash: Vec<(U::Time, U::Diff)>, /// Counts the number of singleton-optimized entries, that we may correctly count the updates. singletons: usize, } - impl Merger> for RhhValMerger + impl Merger> for RhhValMerger where - layout::Key: Default + HashOrdered, - RhhValBatch: Batch>, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> columnar::Ref<'a, U::Key>: HashOrdered, { - fn new(batch1: &RhhValBatch, batch2: &RhhValBatch, compaction_frontier: AntichainRef>) -> Self { + fn new(batch1: &RhhValBatch, batch2: &RhhValBatch, compaction_frontier: AntichainRef) -> Self { assert!(batch1.upper() == batch2.lower()); use crate::lattice::Lattice; @@ -389,21 +359,21 @@ mod val_batch { let batch2 = &batch2.storage; let mut storage = RhhValStorage { - keys: L::KeyContainer::merge_capacity(&batch1.keys, &batch2.keys), - keys_offs: L::OffsetContainer::with_capacity(batch1.keys_offs.len() + batch2.keys_offs.len()), - vals: L::ValContainer::merge_capacity(&batch1.vals, &batch2.vals), - vals_offs: L::OffsetContainer::with_capacity(batch1.vals_offs.len() + batch2.vals_offs.len()), - times: L::TimeContainer::merge_capacity(&batch1.times, &batch2.times), - diffs: L::DiffContainer::merge_capacity(&batch1.diffs, &batch2.diffs), + keys: Coltainer::::merge_capacity(&batch1.keys, &batch2.keys), + keys_offs: OffsetList::with_capacity(batch1.keys_offs.len() + batch2.keys_offs.len()), + vals: Coltainer::::merge_capacity(&batch1.vals, &batch2.vals), + vals_offs: OffsetList::with_capacity(batch1.vals_offs.len() + batch2.vals_offs.len()), + times: Coltainer::::merge_capacity(&batch1.times, &batch2.times), + diffs: Coltainer::::merge_capacity(&batch1.diffs, &batch2.diffs), key_count: 0, key_capacity: rhh_cap, - divisor: RhhValStorage::::divisor_for_capacity(rhh_cap), + divisor: RhhValStorage::::divisor_for_capacity(rhh_cap), }; // Mark explicit types because type inference fails to resolve it. - let keys_offs: &mut L::OffsetContainer = &mut storage.keys_offs; + let keys_offs: &mut OffsetList = &mut storage.keys_offs; keys_offs.push_ref(0); - let vals_offs: &mut L::OffsetContainer = &mut storage.vals_offs; + let vals_offs: &mut OffsetList = &mut storage.vals_offs; vals_offs.push_ref(0); RhhValMerger { @@ -415,14 +385,14 @@ mod val_batch { singletons: 0, } } - fn done(self) -> RhhValBatch { + fn done(self) -> RhhValBatch { RhhValBatch { updates: self.result.times.len() + self.singletons, storage: self.result, description: self.description, } } - fn work(&mut self, source1: &RhhValBatch, source2: &RhhValBatch, fuel: &mut isize) { + fn work(&mut self, source1: &RhhValBatch, source2: &RhhValBatch, fuel: &mut isize) { // An (incomplete) indication of the amount of work we've done so far. let starting_updates = self.result.times.len(); @@ -460,10 +430,10 @@ mod val_batch { } // Helper methods in support of merging batches. - impl RhhValMerger + impl RhhValMerger where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> columnar::Ref<'a, U::Key>: HashOrdered, { /// Copy the next key in `source`. /// @@ -472,7 +442,7 @@ mod val_batch { /// compacted values and updates. /// /// The caller should be certain to update the cursor, as this method does not do this. - fn copy_key(&mut self, source: &RhhValStorage, cursor: usize) { + fn copy_key(&mut self, source: &RhhValStorage, cursor: usize) { // Capture the initial number of values to determine if the merge was ultimately non-empty. let init_vals = self.result.vals.len(); let (mut lower, upper) = source.values_for_key(cursor); @@ -494,7 +464,7 @@ mod val_batch { /// /// This method only merges a single key. It applies all compaction necessary, and may result in no output /// if the updates cancel either directly or after compaction. - fn merge_key(&mut self, source1: &RhhValStorage, source2: &RhhValStorage) { + fn merge_key(&mut self, source1: &RhhValStorage, source2: &RhhValStorage) { use ::std::cmp::Ordering; match source1.keys.index(self.key_cursor1).cmp(&source2.keys.index(self.key_cursor2)) { @@ -525,8 +495,8 @@ mod val_batch { /// an offset that should be recorded to indicate the upper extent of the result values. fn merge_vals( &mut self, - (source1, mut lower1, upper1): (&RhhValStorage, usize, usize), - (source2, mut lower2, upper2): (&RhhValStorage, usize, usize), + (source1, mut lower1, upper1): (&RhhValStorage, usize, usize), + (source2, mut lower2, upper2): (&RhhValStorage, usize, usize), ) -> Option { // Capture the initial number of values to determine if the merge was ultimately non-empty. let init_vals = self.result.vals.len(); @@ -593,16 +563,16 @@ mod val_batch { } /// Transfer updates for an indexed value in `source` into `self`, with compaction applied. - fn stash_updates_for_val(&mut self, source: &RhhValStorage, index: usize) { + fn stash_updates_for_val(&mut self, source: &RhhValStorage, index: usize) { let (lower, upper) = source.updates_for_value(index); for i in lower .. upper { // NB: Here is where we would need to look back if `lower == upper`. let time = source.times.index(i); let diff = source.diffs.index(i); - let mut new_time = L::TimeContainer::into_owned(time); + let mut new_time = ::into_owned(time); use crate::lattice::Lattice; new_time.advance_by(self.description.since().borrow()); - self.update_stash.push((new_time, L::DiffContainer::into_owned(diff))); + self.update_stash.push((new_time, ::into_owned(diff))); } } @@ -616,7 +586,7 @@ mod val_batch { let time_diff = self.result.times.last().zip(self.result.diffs.last()); let last_eq = self.update_stash.last().zip(time_diff).map(|((t1, d1), (t2, d2))| { // TODO: The use of `into_owned` is a work-around for not having reference types. - *t1 == L::TimeContainer::into_owned(t2) && *d1 == L::DiffContainer::into_owned(d2) + *t1 == ::into_owned(t2) && *d1 == ::into_owned(d2) }); if self.update_stash.len() == 1 && last_eq.unwrap_or(false) { // Just clear out update_stash, as we won't drain it here. @@ -645,39 +615,34 @@ mod val_batch { /// Importantly, we should skip over invalid keys, rather than report them as /// invalid through `key_valid`: that method is meant to indicate the end of /// the cursor, rather than internal state. - pub struct RhhValCursor + pub struct RhhValCursor where - layout::Key: Default + HashOrdered, + U::Key: Default + HashOrdered, { /// Absolute position of the current key. key_cursor: usize, /// Absolute position of the current value. val_cursor: usize, /// Phantom marker for Rust happiness. - phantom: PhantomData, + phantom: PhantomData, } - use crate::trace::implementations::WithLayout; - impl WithLayout for RhhValCursor + impl Cursor for RhhValCursor where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> columnar::Ref<'a, U::Key>: HashOrdered, { - type Layout = L; - } - - impl Cursor for RhhValCursor - where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, - { - type Storage = RhhValBatch; - - fn get_key<'a>(&self, storage: &'a RhhValBatch) -> Option> { storage.storage.keys.get(self.key_cursor) } - fn get_val<'a>(&self, storage: &'a RhhValBatch) -> Option> { if self.val_valid(storage) { storage.storage.vals.get(self.val_cursor) } else { None } } - fn key<'a>(&self, storage: &'a RhhValBatch) -> Self::Key<'a> { storage.storage.keys.index(self.key_cursor) } - fn val<'a>(&self, storage: &'a RhhValBatch) -> Self::Val<'a> { storage.storage.vals.index(self.val_cursor) } - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &RhhValBatch, mut logic: L2) { + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; + type Storage = RhhValBatch; + + fn get_key<'a>(&self, storage: &'a RhhValBatch) -> Option> { storage.storage.keys.get(self.key_cursor) } + fn get_val<'a>(&self, storage: &'a RhhValBatch) -> Option> { if self.val_valid(storage) { Some(self.val(storage)) } else { None } } + fn key<'a>(&self, storage: &'a RhhValBatch) -> columnar::Ref<'a, Self::Key> { storage.storage.keys.index(self.key_cursor) } + fn val<'a>(&self, storage: &'a RhhValBatch) -> columnar::Ref<'a, Self::Val> { storage.storage.vals.index(self.val_cursor) } + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &RhhValBatch, mut logic: L2) { let (lower, upper) = storage.storage.updates_for_value(self.val_cursor); for index in lower .. upper { let time = storage.storage.times.index(index); @@ -685,9 +650,9 @@ mod val_batch { logic(time, diff); } } - fn key_valid(&self, storage: &RhhValBatch) -> bool { self.key_cursor < storage.storage.keys.len() } - fn val_valid(&self, storage: &RhhValBatch) -> bool { self.val_cursor < storage.storage.values_for_key(self.key_cursor).1 } - fn step_key(&mut self, storage: &RhhValBatch){ + fn key_valid(&self, storage: &RhhValBatch) -> bool { self.key_cursor < storage.storage.keys.len() } + fn val_valid(&self, storage: &RhhValBatch) -> bool { self.val_cursor < storage.storage.values_for_key(self.key_cursor).1 } + fn step_key(&mut self, storage: &RhhValBatch){ // We advance the cursor by one for certain, and then as long as we need to find a valid key. self.key_cursor += 1; storage.storage.advance_to_live_key(&mut self.key_cursor); @@ -699,7 +664,7 @@ mod val_batch { self.key_cursor = storage.storage.keys.len(); } } - fn seek_key(&mut self, storage: &RhhValBatch, key: Self::Key<'_>) { + fn seek_key(&mut self, storage: &RhhValBatch, key: columnar::Ref<'_, Self::Key>) { // self.key_cursor += storage.storage.keys.advance(self.key_cursor, storage.storage.keys.len(), |x| x.lt(key)); let desired = storage.storage.desired_location(&key); // Advance the cursor, if `desired` is ahead of it. @@ -719,16 +684,16 @@ mod val_batch { self.rewind_vals(storage); } } - fn step_val(&mut self, storage: &RhhValBatch) { + fn step_val(&mut self, storage: &RhhValBatch) { self.val_cursor += 1; if !self.val_valid(storage) { self.val_cursor = storage.storage.values_for_key(self.key_cursor).1; } } - fn seek_val(&mut self, storage: &RhhValBatch, val: Self::Val<'_>) { - self.val_cursor += storage.storage.vals.advance(self.val_cursor, storage.storage.values_for_key(self.key_cursor).1, |x| ::reborrow(x).lt(&::reborrow(val))); + fn seek_val(&mut self, storage: &RhhValBatch, val: columnar::Ref<'_, Self::Val>) { + self.val_cursor += storage.storage.vals.advance(self.val_cursor, storage.storage.values_for_key(self.key_cursor).1, |x| Coltainer::::reborrow(x).lt(&Coltainer::::reborrow(val))); } - fn rewind_keys(&mut self, storage: &RhhValBatch) { + fn rewind_keys(&mut self, storage: &RhhValBatch) { self.key_cursor = 0; storage.storage.advance_to_live_key(&mut self.key_cursor); @@ -736,29 +701,28 @@ mod val_batch { self.rewind_vals(storage) } } - fn rewind_vals(&mut self, storage: &RhhValBatch) { + fn rewind_vals(&mut self, storage: &RhhValBatch) { self.val_cursor = storage.storage.values_for_key(self.key_cursor).0; } } /// A builder for creating layers from unsorted update tuples. - pub struct RhhValBuilder + pub struct RhhValBuilder where - layout::Key: Default + HashOrdered, + U::Key: Default + HashOrdered, { - result: RhhValStorage, - singleton: Option<(layout::Time, layout::Diff)>, + result: RhhValStorage, + singleton: Option<(U::Time, U::Diff)>, /// Counts the number of singleton optimizations we performed. /// /// This number allows us to correctly gauge the total number of updates reflected in a batch, /// even though `updates.len()` may be much shorter than this amount. singletons: usize, - _marker: PhantomData, } - impl RhhValBuilder + impl RhhValBuilder where - layout::Key: Default + HashOrdered, + U::Key: Default + HashOrdered, { /// Pushes a single update, which may set `self.singleton` rather than push. /// @@ -771,10 +735,10 @@ mod val_batch { /// previously pushed update exactly. In that case, we do not push the update into `updates`. /// The update tuple is retained in `self.singleton` in case we see another update and need /// to recover the singleton to push it into `updates` to join the second update. - fn push_update(&mut self, time: layout::Time, diff: layout::Diff) { + fn push_update(&mut self, time: U::Time, diff: U::Diff) { // If a just-pushed update exactly equals `(time, diff)` we can avoid pushing it. // TODO: The use of `into_owned` is a bandage for not having references we can compare. - if self.result.times.last().map(|t| L::TimeContainer::into_owned(t) == time).unwrap_or(false) && self.result.diffs.last().map(|d| L::DiffContainer::into_owned(d) == diff).unwrap_or(false) { + if self.result.times.last().map(|t| ::into_owned(t) == time).unwrap_or(false) && self.result.diffs.last().map(|d| ::into_owned(d) == diff).unwrap_or(false) { assert!(self.singleton.is_none()); self.singleton = Some((time, diff)); } @@ -790,22 +754,20 @@ mod val_batch { } } - impl Builder for RhhValBuilder + impl Builder for RhhValBuilder where - layout::Key: Default + HashOrdered, - CI: for<'a> BuilderInput = layout::Key, Time=layout::Time, Diff=layout::Diff>, - for<'a> L::ValContainer: PushInto>, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> columnar::Ref<'a, U::Key>: HashOrdered, { - type Input = CI; - type Time = layout::Time; - type Output = RhhValBatch; + type Input = Vec<((U::Key, U::Val), U::Time, U::Diff)>; + type Time = U::Time; + type Output = RhhValBatch; fn with_capacity(keys: usize, vals: usize, upds: usize) -> Self { // Double the capacity for RHH; probably excessive. let rhh_capacity = 2 * keys; - let divisor = RhhValStorage::::divisor_for_capacity(rhh_capacity); + let divisor = RhhValStorage::::divisor_for_capacity(rhh_capacity); // We want some additive slop, in case we spill over. // This number magically chosen based on nothing in particular. // Worst case, we will re-alloc and copy if we spill beyond this. @@ -814,37 +776,35 @@ mod val_batch { // We don't introduce zero offsets as they will be introduced by the first `push` call. Self { result: RhhValStorage { - keys: L::KeyContainer::with_capacity(keys), - keys_offs: L::OffsetContainer::with_capacity(keys + 1), - vals: L::ValContainer::with_capacity(vals), - vals_offs: L::OffsetContainer::with_capacity(vals + 1), - times: L::TimeContainer::with_capacity(upds), - diffs: L::DiffContainer::with_capacity(upds), + keys: Coltainer::::with_capacity(keys), + keys_offs: OffsetList::with_capacity(keys + 1), + vals: Coltainer::::with_capacity(vals), + vals_offs: OffsetList::with_capacity(vals + 1), + times: Coltainer::::with_capacity(upds), + diffs: Coltainer::::with_capacity(upds), key_count: 0, key_capacity: rhh_capacity, divisor, }, singleton: None, singletons: 0, - _marker: PhantomData, } } #[inline] fn push(&mut self, chunk: &mut Self::Input) { - for item in chunk.drain() { - let (key, val, time, diff) = CI::into_parts(item); + for ((key, val), time, diff) in chunk.drain(..) { // Perhaps this is a continuation of an already received key. - if self.result.keys.last().map(|k| CI::key_eq(&key, k)).unwrap_or(false) { + if self.result.keys.last().map(|k| Coltainer::::into_owned(k) == key).unwrap_or(false) { // Perhaps this is a continuation of an already received value. - if self.result.vals.last().map(|v| CI::val_eq(&val, v)).unwrap_or(false) { + if self.result.vals.last().map(|v| Coltainer::::into_owned(v) == val).unwrap_or(false) { self.push_update(time, diff); } else { // New value; complete representation of prior value. self.result.vals_offs.push_ref(self.result.times.len()); if self.singleton.take().is_some() { self.singletons += 1; } self.push_update(time, diff); - self.result.vals.push_into(val); + self.result.vals.push_own(&val); } } else { // New key; complete representation of prior key. @@ -852,7 +812,7 @@ mod val_batch { if self.singleton.take().is_some() { self.singletons += 1; } self.result.keys_offs.push_ref(self.result.vals.len()); self.push_update(time, diff); - self.result.vals.push_into(val); + self.result.vals.push_own(&val); // Insert the key, but with no specified offset. self.result.insert_key_own(&key, None); } @@ -860,7 +820,7 @@ mod val_batch { } #[inline(never)] - fn done(mut self, description: Description) -> RhhValBatch { + fn done(mut self, description: Description) -> RhhValBatch { // Record the final offsets self.result.vals_offs.push_ref(self.result.times.len()); // Remove any pending singleton, and if it was set increment our count. @@ -874,7 +834,18 @@ mod val_batch { } fn seal(chain: &mut Vec, description: Description) -> Self::Output { - let (keys, vals, upds) = Self::Input::key_val_upd_counts(&chain[..]); + let mut keys = 0; let mut vals = 0; let mut upds = 0; + for link in chain.iter() { + let mut prev_keyval = None; + for ((key, val), _, _) in link.iter() { + if let Some((pk, pv)) = prev_keyval { + if pk != key { keys += 1; vals += 1; } + else if pv != val { vals += 1; } + } else { keys += 1; vals += 1; } + upds += 1; + prev_keyval = Some((key, val)); + } + } let mut builder = Self::with_capacity(keys, vals, upds); for mut chunk in chain.drain(..) { builder.push(&mut chunk); diff --git a/differential-dataflow/src/trace/implementations/spine_fueled.rs b/differential-dataflow/src/trace/implementations/spine_fueled.rs index b7be06b9c..186b3d8fe 100644 --- a/differential-dataflow/src/trace/implementations/spine_fueled.rs +++ b/differential-dataflow/src/trace/implementations/spine_fueled.rs @@ -99,13 +99,13 @@ pub struct Spine { exert_logic: Option, } -use crate::trace::WithLayout; -impl WithLayout for Spine { - type Layout = B::Layout; -} - impl TraceReader for Spine { + type Key = B::Key; + type Val = B::Val; + type Time = B::Time; + type Diff = B::Diff; + type Batch = B; type Storage = Vec; type Cursor = CursorList<::Cursor>; diff --git a/differential-dataflow/src/trace/mod.rs b/differential-dataflow/src/trace/mod.rs index 0197e09c0..76a3c44a6 100644 --- a/differential-dataflow/src/trace/mod.rs +++ b/differential-dataflow/src/trace/mod.rs @@ -19,78 +19,51 @@ use crate::logging::Logger; pub use self::cursor::Cursor; pub use self::description::Description; -use crate::trace::implementations::LayoutExt; +use crate::trace::implementations::Data; /// A type used to express how much effort a trace should exert even in the absence of updates. pub type ExertionLogic = std::sync::Arc Fn(&'a [(usize, usize, usize)])->Option+Send+Sync>; -// The traces and batch and cursors want the flexibility to appear as if they manage certain types of keys and -// values and such, while perhaps using other representations, I'm thinking mostly of wrappers around the keys -// and vals that change the `Ord` implementation, or stash hash codes, or the like. -// -// This complicates what requirements we make so that the trace is still usable by someone who knows only about -// the base key and value types. For example, the complex types should likely dereference to the simpler types, -// so that the user can make sense of the result as if they were given references to the simpler types. At the -// same time, the collection should be formable from base types (perhaps we need an `Into` or `From` constraint) -// and we should, somehow, be able to take a reference to the simple types to compare against the more complex -// types. This second one is also like an `Into` or `From` constraint, except that we start with a reference and -// really don't need anything more complex than a reference, but we can't form an owned copy of the complex type -// without cloning it. -// -// We could just start by cloning things. Worry about wrapping references later on. - /// A trace whose contents may be read. /// /// This is a restricted interface to the more general `Trace` trait, which extends this trait with further methods /// to update the contents of the trace. These methods are used to examine the contents, and to update the reader's /// capabilities (which may release restrictions on the mutations to the underlying trace and cause work to happen). -pub trait TraceReader : LayoutExt { +pub trait TraceReader { + + /// The owned key type. + type Key: Data; + /// The owned val type. + type Val: Data; + /// The owned time type. + type Time: Data + crate::lattice::Lattice + Timestamp; + /// The owned diff type. + type Diff: Data + crate::difference::Semigroup; /// The type of an immutable collection of updates. type Batch: 'static + Clone + - BatchReader + - WithLayout + - for<'a> LayoutExt< - Key<'a> = Self::Key<'a>, - KeyOwn = Self::KeyOwn, - Val<'a> = Self::Val<'a>, - ValOwn = Self::ValOwn, + BatchReader< + Key = Self::Key, + Val = Self::Val, Time = Self::Time, - TimeGat<'a> = Self::TimeGat<'a>, Diff = Self::Diff, - DiffGat<'a> = Self::DiffGat<'a>, - KeyContainer = Self::KeyContainer, - ValContainer = Self::ValContainer, - TimeContainer = Self::TimeContainer, - DiffContainer = Self::DiffContainer, >; - /// Storage type for `Self::Cursor`. Likely related to `Self::Batch`. type Storage; /// The type used to enumerate the collections contents. type Cursor: - Cursor + - WithLayout + - for<'a> LayoutExt< - Key<'a> = Self::Key<'a>, - KeyOwn = Self::KeyOwn, - Val<'a> = Self::Val<'a>, - ValOwn = Self::ValOwn, + Cursor< + Key = Self::Key, + Val = Self::Val, Time = Self::Time, - TimeGat<'a> = Self::TimeGat<'a>, Diff = Self::Diff, - DiffGat<'a> = Self::DiffGat<'a>, - KeyContainer = Self::KeyContainer, - ValContainer = Self::ValContainer, - TimeContainer = Self::TimeContainer, - DiffContainer = Self::DiffContainer, + Storage = Self::Storage, >; - /// Provides a cursor over updates contained in the trace. fn cursor(&mut self) -> (Self::Cursor, Self::Storage) { if let Some(cursor) = self.cursor_through(Antichain::new().borrow()) { @@ -103,87 +76,34 @@ pub trait TraceReader : LayoutExt { /// Acquires a cursor to the restriction of the collection's contents to updates at times not greater or /// equal to an element of `upper`. - /// - /// This method is expected to work if called with an `upper` that (i) was an observed bound in batches from - /// the trace, and (ii) the trace has not been advanced beyond `upper`. Practically, the implementation should - /// be expected to look for a "clean cut" using `upper`, and if it finds such a cut can return a cursor. This - /// should allow `upper` such as `&[]` as used by `self.cursor()`, though it is difficult to imagine other uses. fn cursor_through(&mut self, upper: AntichainRef) -> Option<(Self::Cursor, Self::Storage)>; /// Advances the frontier that constrains logical compaction. - /// - /// Logical compaction is the ability of the trace to change the times of the updates it contains. - /// Update times may be changed as long as their comparison to all query times beyond the logical compaction - /// frontier remains unchanged. Practically, this means that groups of timestamps not beyond the frontier can - /// be coalesced into fewer representative times. - /// - /// Logical compaction is important, as it allows the trace to forget historical distinctions between update - /// times, and maintain a compact memory footprint over an unbounded update history. - /// - /// By advancing the logical compaction frontier, the caller unblocks merging of otherwise equivalent updates, - /// but loses the ability to observe historical detail that is not beyond `frontier`. - /// - /// It is an error to call this method with a frontier not equal to or beyond the most recent arguments to - /// this method, or the initial value of `get_logical_compaction()` if this method has not yet been called. fn set_logical_compaction(&mut self, frontier: AntichainRef); /// Reports the logical compaction frontier. - /// - /// All update times beyond this frontier will be presented with their original times, and all update times - /// not beyond this frontier will present as a time that compares identically with all query times beyond - /// this frontier. Practically, update times not beyond this frontier should not be taken to be accurate as - /// presented, and should be used carefully, only in accumulation to times that are beyond the frontier. fn get_logical_compaction(&mut self) -> AntichainRef<'_, Self::Time>; /// Advances the frontier that constrains physical compaction. - /// - /// Physical compaction is the ability of the trace to merge the batches of updates it maintains. Physical - /// compaction does not change the updates or their timestamps, although it is also the moment at which - /// logical compaction is most likely to happen. - /// - /// Physical compaction allows the trace to maintain a logarithmic number of batches of updates, which is - /// what allows the trace to provide efficient random access by keys and values. - /// - /// By advancing the physical compaction frontier, the caller unblocks the merging of batches of updates, - /// but loses the ability to create a cursor through any frontier not beyond `frontier`. - /// - /// It is an error to call this method with a frontier not equal to or beyond the most recent arguments to - /// this method, or the initial value of `get_physical_compaction()` if this method has not yet been called. fn set_physical_compaction(&mut self, frontier: AntichainRef<'_, Self::Time>); /// Reports the physical compaction frontier. - /// - /// All batches containing updates beyond this frontier will not be merged with other batches. This allows - /// the caller to create a cursor through any frontier beyond the physical compaction frontier, with the - /// `cursor_through()` method. This functionality is primarily of interest to the `join` operator, and any - /// other operators who need to take notice of the physical structure of update batches. fn get_physical_compaction(&mut self) -> AntichainRef<'_, Self::Time>; /// Maps logic across the non-empty sequence of batches in the trace. - /// - /// This is currently used only to extract historical data to prime late-starting operators who want to reproduce - /// the stream of batches moving past the trace. It could also be a fine basis for a default implementation of the - /// cursor methods, as they (by default) just move through batches accumulating cursors into a cursor list. fn map_batches(&self, f: F); /// Reads the upper frontier of committed times. - /// - /// #[inline] fn read_upper(&mut self, target: &mut Antichain) { target.clear(); - target.insert(::minimum()); + target.insert(::minimum()); self.map_batches(|batch| { target.clone_from(batch.upper()); }); } /// Advances `upper` by any empty batches. - /// - /// An empty batch whose `batch.lower` bound equals the current - /// contents of `upper` will advance `upper` to `batch.upper`. - /// Taken across all batches, this should advance `upper` across - /// empty batch regions. fn advance_upper(&mut self, upper: &mut Antichain) { self.map_batches(|batch| { if batch.is_empty() && batch.lower() == upper { @@ -191,16 +111,9 @@ pub trait TraceReader : LayoutExt { } }); } - } /// An append-only collection of `(key, val, time, diff)` tuples. -/// -/// The trace must pretend to look like a collection of `(Key, Val, Time, isize)` tuples, but is permitted -/// to introduce new types `KeyRef`, `ValRef`, and `TimeRef` which can be dereference to the types above. -/// -/// The trace must be constructable from, and navigable by the `Key`, `Val`, `Time` types, but does not need -/// to return them. pub trait Trace : TraceReader { /// Allocates a new empty trace. @@ -214,59 +127,38 @@ pub trait Trace : TraceReader { fn exert(&mut self); /// Sets the logic for exertion in the absence of updates. - /// - /// The function receives an iterator over batch levels, from large to small, as triples `(level, count, length)`, - /// indicating the level, the number of batches, and their total length in updates. It should return a number of - /// updates to perform, or `None` if no work is required. fn set_exert_logic(&mut self, logic: ExertionLogic); /// Introduces a batch of updates to the trace. - /// - /// Batches describe the time intervals they contain, and they should be added to the trace in contiguous - /// intervals. If a batch arrives with a lower bound that does not equal the upper bound of the most recent - /// addition, the trace will add an empty batch. It is an error to then try to populate that region of time. - /// - /// This restriction could be relaxed, especially if we discover ways in which batch interval order could - /// commute. For now, the trace should complain, to the extent that it cares about contiguous intervals. fn insert(&mut self, batch: Self::Batch); /// Introduces an empty batch concluding the trace. - /// - /// This method should be logically equivalent to introducing an empty batch whose lower frontier equals - /// the upper frontier of the most recently introduced batch, and whose upper frontier is empty. fn close(&mut self); } -use crate::trace::implementations::WithLayout; - /// A batch of updates whose contents may be read. -/// -/// This is a restricted interface to batches of updates, which support the reading of the batch's contents, -/// but do not expose ways to construct the batches. This trait is appropriate for views of the batch, and is -/// especially useful for views derived from other sources in ways that prevent the construction of batches -/// from the type of data in the view (for example, filtered views, or views with extended time coordinates). -pub trait BatchReader : LayoutExt + Sized { +pub trait BatchReader : Sized { + + /// The owned key type. + type Key: Data; + /// The owned val type. + type Val: Data; + /// The owned time type. + type Time: Data + crate::lattice::Lattice + Timestamp; + /// The owned diff type. + type Diff: Data + crate::difference::Semigroup; /// The type used to enumerate the batch's contents. type Cursor: - Cursor + - WithLayout + - for<'a> LayoutExt< - Key<'a> = Self::Key<'a>, - KeyOwn = Self::KeyOwn, - Val<'a> = Self::Val<'a>, - ValOwn = Self::ValOwn, + Cursor< + Key = Self::Key, + Val = Self::Val, Time = Self::Time, - TimeGat<'a> = Self::TimeGat<'a>, Diff = Self::Diff, - DiffGat<'a> = Self::DiffGat<'a>, - KeyContainer = Self::KeyContainer, - ValContainer = Self::ValContainer, - TimeContainer = Self::TimeContainer, - DiffContainer = Self::DiffContainer, + Storage = Self, >; - /// Acquires a cursor to the batch's contents. + /// Acquires a cursor to the batch's contents. fn cursor(&self) -> Self::Cursor; /// The number of updates in the batch. fn len(&self) -> usize; @@ -287,10 +179,6 @@ pub trait Batch : BatchReader + Sized { type Merger: Merger; /// Initiates the merging of consecutive batches. - /// - /// The result of this method can be exercised to eventually produce the same result - /// that a call to `self.merge(other)` would produce, but it can be done in a measured - /// fashion. This can help to avoid latency spikes where a large merge needs to happen. fn begin_merge(&self, other: &Self, compaction_frontier: AntichainRef) -> Self::Merger { Self::Merger::new(self, other, compaction_frontier) } @@ -327,26 +215,15 @@ pub trait Builder: Sized { type Output; /// Allocates an empty builder. - /// - /// Ideally we deprecate this and insist all non-trivial building happens via `with_capacity()`. - // #[deprecated] fn new() -> Self { Self::with_capacity(0, 0, 0) } /// Allocates an empty builder with capacity for the specified keys, values, and updates. - /// - /// They represent respectively the number of distinct `key`, `(key, val)`, and total updates. fn with_capacity(keys: usize, vals: usize, upds: usize) -> Self; /// Adds a chunk of elements to the batch. - /// - /// Adds all elements from `chunk` to the builder and leaves `chunk` in an undefined state. fn push(&mut self, chunk: &mut Self::Input); /// Completes building and returns the batch. fn done(self, description: Description) -> Self::Output; /// Builds a batch from a chain of updates corresponding to the indicated lower and upper bounds. - /// - /// This method relies on the chain only containing updates greater or equal to the lower frontier, - /// and not greater or equal to the upper frontier, as encoded in the description. Chains must also - /// be sorted and consolidated. fn seal(chain: &mut Vec, description: Description) -> Self::Output; } @@ -356,15 +233,8 @@ pub trait Merger { /// up to the supplied frontier. fn new(source1: &Output, source2: &Output, compaction_frontier: AntichainRef) -> Self; /// Perform some amount of work, decrementing `fuel`. - /// - /// If `fuel` is non-zero after the call, the merging is complete and - /// one should call `done` to extract the merged results. fn work(&mut self, source1: &Output, source2: &Output, fuel: &mut isize); /// Extracts merged results. - /// - /// This method should only be called after `work` has been called and - /// has not brought `fuel` to zero. Otherwise, the merge is still in - /// progress. fn done(self) -> Output; } @@ -377,22 +247,19 @@ pub mod rc_blanket_impls { use timely::progress::{Antichain, frontier::AntichainRef}; use super::{Batch, BatchReader, Builder, Merger, Cursor, Description}; - impl WithLayout for Rc { - type Layout = B::Layout; - } - impl BatchReader for Rc { - /// The type used to enumerate the batch's contents. + type Key = B::Key; + type Val = B::Val; + type Time = B::Time; + type Diff = B::Diff; + type Cursor = RcBatchCursor; - /// Acquires a cursor to the batch's contents. fn cursor(&self) -> Self::Cursor { RcBatchCursor::new((**self).cursor()) } - /// The number of updates in the batch. fn len(&self) -> usize { (**self).len() } - /// Describes the times of the updates in the batch. fn description(&self) -> &Description { (**self).description() } } @@ -401,42 +268,39 @@ pub mod rc_blanket_impls { cursor: C, } - use crate::trace::implementations::WithLayout; - impl WithLayout for RcBatchCursor { - type Layout = C::Layout; - } - impl RcBatchCursor { fn new(cursor: C) -> Self { - RcBatchCursor { - cursor, - } + RcBatchCursor { cursor } } } impl Cursor for RcBatchCursor { + type Key = C::Key; + type Val = C::Val; + type Time = C::Time; + type Diff = C::Diff; type Storage = Rc; #[inline] fn key_valid(&self, storage: &Self::Storage) -> bool { self.cursor.key_valid(storage) } #[inline] fn val_valid(&self, storage: &Self::Storage) -> bool { self.cursor.val_valid(storage) } - #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { self.cursor.key(storage) } - #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { self.cursor.val(storage) } + #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key> { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val> { self.cursor.val(storage) } - #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } - #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } + #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } #[inline] - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, logic: L) { self.cursor.map_times(storage, logic) } #[inline] fn step_key(&mut self, storage: &Self::Storage) { self.cursor.step_key(storage) } - #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { self.cursor.seek_key(storage, key) } + #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>) { self.cursor.seek_key(storage, key) } #[inline] fn step_val(&mut self, storage: &Self::Storage) { self.cursor.step_val(storage) } - #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>) { self.cursor.seek_val(storage, val) } + #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>) { self.cursor.seek_val(storage, val) } #[inline] fn rewind_keys(&mut self, storage: &Self::Storage) { self.cursor.rewind_keys(storage) } #[inline] fn rewind_vals(&mut self, storage: &Self::Storage) { self.cursor.rewind_vals(storage) } diff --git a/differential-dataflow/src/trace/wrappers/enter.rs b/differential-dataflow/src/trace/wrappers/enter.rs index cea7c8b27..14fb7acea 100644 --- a/differential-dataflow/src/trace/wrappers/enter.rs +++ b/differential-dataflow/src/trace/wrappers/enter.rs @@ -1,12 +1,10 @@ //! Wrappers to provide trace access to nested scopes. -// use timely::progress::nested::product::Product; use timely::progress::timestamp::Refines; use timely::progress::{Antichain, frontier::AntichainRef}; use crate::lattice::Lattice; -use crate::trace::{TraceReader, BatchReader, Description}; -use crate::trace::cursor::Cursor; +use crate::trace::{TraceReader, BatchReader, Cursor, Description}; /// Wrapper to provide trace to nested scope. pub struct TraceEnter { @@ -15,7 +13,7 @@ pub struct TraceEnter { stash2: Antichain, } -impl Clone for TraceEnter { +impl Clone for TraceEnter { fn clone(&self) -> Self { TraceEnter { trace: self.trace.clone(), @@ -25,25 +23,16 @@ impl Clone for TraceEnter { } } -impl WithLayout for TraceEnter -where - Tr: TraceReader, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl TraceReader for TraceEnter where Tr: TraceReader, - TInner: Refines+Lattice, + TInner: crate::trace::implementations::Data + Refines+Lattice, { + type Key = Tr::Key; + type Val = Tr::Val; + type Time = TInner; + type Diff = Tr::Diff; + type Batch = BatchEnter; type Storage = Tr::Storage; type Cursor = CursorEnter; @@ -108,33 +97,31 @@ where } } - /// Wrapper to provide batch to nested scope. -#[derive(Clone)] pub struct BatchEnter { batch: B, description: Description, } -impl WithLayout for BatchEnter -where - B: BatchReader, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); +impl Clone for BatchEnter { + fn clone(&self) -> Self { + BatchEnter { + batch: self.batch.clone(), + description: self.description.clone(), + } + } } impl BatchReader for BatchEnter where B: BatchReader, - TInner: Refines+Lattice, + TInner: crate::trace::implementations::Data + Refines+Lattice, { + type Key = B::Key; + type Val = B::Val; + type Time = TInner; + type Diff = B::Diff; + type Cursor = BatchCursorEnter; fn cursor(&self) -> Self::Cursor { @@ -163,31 +150,17 @@ where } /// Wrapper to provide cursor to nested scope. -pub struct CursorEnter { - phantom: ::std::marker::PhantomData, +pub struct CursorEnter { cursor: C, + /// Container for synthesized times, used by `map_times`. + times: TInner::Container, } -use crate::trace::implementations::{Layout, WithLayout}; -impl WithLayout for CursorEnter -where - C: Cursor, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - -impl CursorEnter { +impl CursorEnter { fn new(cursor: C) -> Self { CursorEnter { - phantom: ::std::marker::PhantomData, cursor, + times: Default::default(), } } } @@ -195,94 +168,114 @@ impl CursorEnter { impl Cursor for CursorEnter where C: Cursor, - TInner: Refines+Lattice, + TInner: crate::trace::implementations::Data + Refines+Lattice, { + type Key = C::Key; + type Val = C::Val; + type Time = TInner; + type Diff = C::Diff; type Storage = C::Storage; #[inline] fn key_valid(&self, storage: &Self::Storage) -> bool { self.cursor.key_valid(storage) } #[inline] fn val_valid(&self, storage: &Self::Storage) -> bool { self.cursor.val_valid(storage) } - #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { self.cursor.key(storage) } - #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { self.cursor.val(storage) } + #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key> { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val> { self.cursor.val(storage) } - #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } - #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } + #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, mut logic: L) { + use columnar::{Clear, Borrow, Index}; + // Collect synthesized times into a container, then replay with refs. + self.times.clear(); + let mut diffs = Vec::new(); self.cursor.map_times(storage, |time, diff| { - logic(&TInner::to_inner(C::owned_time(time)), diff) - }) + let inner_time = TInner::to_inner(::into_owned(time)); + >::push(&mut self.times, &inner_time); + diffs.push(::into_owned(diff)); + }); + let times_borrowed = self.times.borrow(); + for (i, diff_owned) in diffs.iter().enumerate() { + // Push diff into a temporary container to get a Ref. + + + let mut diff_container = ::Container::default(); + columnar::Push::push(&mut diff_container, diff_owned); + logic(times_borrowed.get(i), diff_container.borrow().get(0)); + } } #[inline] fn step_key(&mut self, storage: &Self::Storage) { self.cursor.step_key(storage) } - #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { self.cursor.seek_key(storage, key) } + #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>) { self.cursor.seek_key(storage, key) } #[inline] fn step_val(&mut self, storage: &Self::Storage) { self.cursor.step_val(storage) } - #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>) { self.cursor.seek_val(storage, val) } + #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>) { self.cursor.seek_val(storage, val) } #[inline] fn rewind_keys(&mut self, storage: &Self::Storage) { self.cursor.rewind_keys(storage) } #[inline] fn rewind_vals(&mut self, storage: &Self::Storage) { self.cursor.rewind_vals(storage) } } - - /// Wrapper to provide cursor to nested scope. -pub struct BatchCursorEnter { - phantom: ::std::marker::PhantomData, +pub struct BatchCursorEnter { cursor: C, + /// Container for synthesized times, used by `map_times`. + times: TInner::Container, } -impl BatchCursorEnter { +impl BatchCursorEnter { fn new(cursor: C) -> Self { BatchCursorEnter { - phantom: ::std::marker::PhantomData, cursor, + times: Default::default(), } } } -impl WithLayout for BatchCursorEnter -where - C: Cursor, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl Cursor for BatchCursorEnter where - TInner: Refines+Lattice, + TInner: crate::trace::implementations::Data + Refines+Lattice, { + type Key = C::Key; + type Val = C::Val; + type Time = TInner; + type Diff = C::Diff; type Storage = BatchEnter; #[inline] fn key_valid(&self, storage: &Self::Storage) -> bool { self.cursor.key_valid(&storage.batch) } #[inline] fn val_valid(&self, storage: &Self::Storage) -> bool { self.cursor.val_valid(&storage.batch) } - #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { self.cursor.key(&storage.batch) } - #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { self.cursor.val(&storage.batch) } + #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key> { self.cursor.key(&storage.batch) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val> { self.cursor.val(&storage.batch) } - #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(&storage.batch) } - #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(&storage.batch) } + #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(&storage.batch) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(&storage.batch) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, mut logic: L) { + use columnar::{Clear, Borrow, Index}; + self.times.clear(); + let mut diffs = Vec::new(); self.cursor.map_times(&storage.batch, |time, diff| { - logic(&TInner::to_inner(C::owned_time(time)), diff) - }) + let inner_time = TInner::to_inner(::into_owned(time)); + >::push(&mut self.times, &inner_time); + diffs.push(::into_owned(diff)); + }); + let times_borrowed = self.times.borrow(); + for (i, diff_owned) in diffs.iter().enumerate() { + + let mut diff_container = ::Container::default(); + columnar::Push::push(&mut diff_container, diff_owned); + logic(times_borrowed.get(i), diff_container.borrow().get(0)); + } } #[inline] fn step_key(&mut self, storage: &Self::Storage) { self.cursor.step_key(&storage.batch) } - #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { self.cursor.seek_key(&storage.batch, key) } + #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>) { self.cursor.seek_key(&storage.batch, key) } #[inline] fn step_val(&mut self, storage: &Self::Storage) { self.cursor.step_val(&storage.batch) } - #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>) { self.cursor.seek_val(&storage.batch, val) } + #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>) { self.cursor.seek_val(&storage.batch, val) } #[inline] fn rewind_keys(&mut self, storage: &Self::Storage) { self.cursor.rewind_keys(&storage.batch) } #[inline] fn rewind_vals(&mut self, storage: &Self::Storage) { self.cursor.rewind_vals(&storage.batch) } diff --git a/differential-dataflow/src/trace/wrappers/enter_at.rs b/differential-dataflow/src/trace/wrappers/enter_at.rs index 7a4739f08..8ab821bf6 100644 --- a/differential-dataflow/src/trace/wrappers/enter_at.rs +++ b/differential-dataflow/src/trace/wrappers/enter_at.rs @@ -40,30 +40,19 @@ where } } -impl WithLayout for TraceEnter -where - Tr: TraceReader, - TInner: Refines+Lattice, - F: Clone, - G: Clone, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl TraceReader for TraceEnter where Tr: TraceReader, - TInner: Refines+Lattice, + TInner: crate::trace::implementations::Data + Refines+Lattice, F: 'static, - F: FnMut(Tr::Key<'_>, Tr::Val<'_>, Tr::TimeGat<'_>)->TInner+Clone, + F: FnMut(columnar::Ref<'_, Tr::Key>, columnar::Ref<'_, Tr::Val>, columnar::Ref<'_, Tr::Time>)->TInner+Clone, G: FnMut(&TInner)->Tr::Time+Clone+'static, { + type Key = Tr::Key; + type Val = Tr::Val; + type Time = TInner; + type Diff = Tr::Diff; + type Batch = BatchEnter; type Storage = Tr::Storage; type Cursor = CursorEnter; @@ -140,27 +129,17 @@ pub struct BatchEnter { logic: F, } -impl WithLayout for BatchEnter -where - B: BatchReader, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - -use crate::trace::implementations::LayoutExt; impl BatchReader for BatchEnter where B: BatchReader, - TInner: Refines+Lattice, - F: FnMut(B::Key<'_>, ::Val<'_>, B::TimeGat<'_>)->TInner+Clone, + TInner: crate::trace::implementations::Data + Refines+Lattice, + F: FnMut(columnar::Ref<'_, B::Key>, columnar::Ref<'_, B::Val>, columnar::Ref<'_, B::Time>)->TInner+Clone, { + type Key = B::Key; + type Val = B::Val; + type Time = TInner; + type Diff = B::Diff; + type Cursor = BatchCursorEnter; fn cursor(&self) -> Self::Cursor { @@ -190,33 +169,19 @@ where } /// Wrapper to provide cursor to nested scope. -pub struct CursorEnter { - phantom: ::std::marker::PhantomData, +pub struct CursorEnter { cursor: C, logic: F, + /// Container for synthesized times, used by `map_times`. + times: TInner::Container, } -use crate::trace::implementations::{Layout, WithLayout}; -impl WithLayout for CursorEnter -where - C: Cursor, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - -impl CursorEnter { +impl CursorEnter { fn new(cursor: C, logic: F) -> Self { CursorEnter { - phantom: ::std::marker::PhantomData, cursor, logic, + times: Default::default(), } } } @@ -224,35 +189,51 @@ impl CursorEnter { impl Cursor for CursorEnter where C: Cursor, - TInner: Refines+Lattice, - F: FnMut(C::Key<'_>, C::Val<'_>, C::TimeGat<'_>)->TInner, + TInner: crate::trace::implementations::Data + Refines+Lattice, + F: FnMut(columnar::Ref<'_, C::Key>, columnar::Ref<'_, C::Val>, columnar::Ref<'_, C::Time>)->TInner, { + type Key = C::Key; + type Val = C::Val; + type Time = TInner; + type Diff = C::Diff; type Storage = C::Storage; #[inline] fn key_valid(&self, storage: &Self::Storage) -> bool { self.cursor.key_valid(storage) } #[inline] fn val_valid(&self, storage: &Self::Storage) -> bool { self.cursor.val_valid(storage) } - #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { self.cursor.key(storage) } - #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { self.cursor.val(storage) } + #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key> { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val> { self.cursor.val(storage) } - #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } - #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } + #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, mut logic: L) { + use columnar::{Clear, Borrow, Index}; let key = self.key(storage); let val = self.val(storage); + self.times.clear(); + let mut diffs = Vec::new(); let logic2 = &mut self.logic; self.cursor.map_times(storage, |time, diff| { - logic(&logic2(key, val, time), diff) - }) + let inner_time = logic2(key, val, time); + >::push(&mut self.times, &inner_time); + diffs.push(::into_owned(diff)); + }); + let times_borrowed = self.times.borrow(); + for (i, diff_owned) in diffs.iter().enumerate() { + + let mut diff_container = ::Container::default(); + columnar::Push::push(&mut diff_container, diff_owned); + logic(times_borrowed.get(i), diff_container.borrow().get(0)); + } } #[inline] fn step_key(&mut self, storage: &Self::Storage) { self.cursor.step_key(storage) } - #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { self.cursor.seek_key(storage, key) } + #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>) { self.cursor.seek_key(storage, key) } #[inline] fn step_val(&mut self, storage: &Self::Storage) { self.cursor.step_val(storage) } - #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>) { self.cursor.seek_val(storage, val) } + #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>) { self.cursor.seek_val(storage, val) } #[inline] fn rewind_keys(&mut self, storage: &Self::Storage) { self.cursor.rewind_keys(storage) } #[inline] fn rewind_vals(&mut self, storage: &Self::Storage) { self.cursor.rewind_vals(storage) } @@ -261,67 +242,70 @@ where /// Wrapper to provide cursor to nested scope. -pub struct BatchCursorEnter { - phantom: ::std::marker::PhantomData, +pub struct BatchCursorEnter { cursor: C, logic: F, + /// Container for synthesized times, used by `map_times`. + times: TInner::Container, } -impl WithLayout for BatchCursorEnter -where - C: Cursor, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - -impl BatchCursorEnter { +impl BatchCursorEnter { fn new(cursor: C, logic: F) -> Self { BatchCursorEnter { - phantom: ::std::marker::PhantomData, cursor, logic, + times: Default::default(), } } } impl Cursor for BatchCursorEnter where - TInner: Refines+Lattice, - F: FnMut(C::Key<'_>, C::Val<'_>, C::TimeGat<'_>)->TInner, + TInner: crate::trace::implementations::Data + Refines+Lattice, + F: FnMut(columnar::Ref<'_, C::Key>, columnar::Ref<'_, C::Val>, columnar::Ref<'_, C::Time>)->TInner, { + type Key = C::Key; + type Val = C::Val; + type Time = TInner; + type Diff = C::Diff; type Storage = BatchEnter; #[inline] fn key_valid(&self, storage: &Self::Storage) -> bool { self.cursor.key_valid(&storage.batch) } #[inline] fn val_valid(&self, storage: &Self::Storage) -> bool { self.cursor.val_valid(&storage.batch) } - #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { self.cursor.key(&storage.batch) } - #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { self.cursor.val(&storage.batch) } + #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key> { self.cursor.key(&storage.batch) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val> { self.cursor.val(&storage.batch) } - #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(&storage.batch) } - #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(&storage.batch) } + #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(&storage.batch) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(&storage.batch) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, mut logic: L) { + use columnar::{Clear, Borrow, Index}; let key = self.key(storage); let val = self.val(storage); + self.times.clear(); + let mut diffs = Vec::new(); let logic2 = &mut self.logic; self.cursor.map_times(&storage.batch, |time, diff| { - logic(&logic2(key, val, time), diff) - }) + let inner_time = logic2(key, val, time); + >::push(&mut self.times, &inner_time); + diffs.push(::into_owned(diff)); + }); + let times_borrowed = self.times.borrow(); + for (i, diff_owned) in diffs.iter().enumerate() { + + let mut diff_container = ::Container::default(); + columnar::Push::push(&mut diff_container, diff_owned); + logic(times_borrowed.get(i), diff_container.borrow().get(0)); + } } #[inline] fn step_key(&mut self, storage: &Self::Storage) { self.cursor.step_key(&storage.batch) } - #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { self.cursor.seek_key(&storage.batch, key) } + #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>) { self.cursor.seek_key(&storage.batch, key) } #[inline] fn step_val(&mut self, storage: &Self::Storage) { self.cursor.step_val(&storage.batch) } - #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>) { self.cursor.seek_val(&storage.batch, val) } + #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>) { self.cursor.seek_val(&storage.batch, val) } #[inline] fn rewind_keys(&mut self, storage: &Self::Storage) { self.cursor.rewind_keys(&storage.batch) } #[inline] fn rewind_vals(&mut self, storage: &Self::Storage) { self.cursor.rewind_vals(&storage.batch) } diff --git a/differential-dataflow/src/trace/wrappers/frontier.rs b/differential-dataflow/src/trace/wrappers/frontier.rs index 5d521831b..a52a8725d 100644 --- a/differential-dataflow/src/trace/wrappers/frontier.rs +++ b/differential-dataflow/src/trace/wrappers/frontier.rs @@ -31,18 +31,13 @@ impl Clone for TraceFrontier { } } -impl WithLayout for TraceFrontier { - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl TraceReader for TraceFrontier { + type Key = Tr::Key; + type Val = Tr::Val; + type Time = Tr::Time; + type Diff = Tr::Diff; + type Batch = BatchFrontier; type Storage = Tr::Storage; type Cursor = CursorFrontier; @@ -86,18 +81,13 @@ pub struct BatchFrontier { until: Antichain, } -impl WithLayout for BatchFrontier { - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl BatchReader for BatchFrontier { + type Key = B::Key; + type Val = B::Val; + type Time = B::Time; + type Diff = B::Diff; + type Cursor = BatchCursorFrontier; fn cursor(&self) -> Self::Cursor { @@ -119,65 +109,72 @@ impl BatchFrontier { } /// Wrapper to provide cursor to nested scope. -pub struct CursorFrontier { +pub struct CursorFrontier { cursor: C, since: Antichain, - until: Antichain -} - -use crate::trace::implementations::{Layout, WithLayout}; -impl WithLayout for CursorFrontier { - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); + until: Antichain, + /// Container for synthesized times, used by `map_times`. + times: T::Container, } -impl CursorFrontier { +impl CursorFrontier { fn new(cursor: C, since: AntichainRef, until: AntichainRef) -> Self { CursorFrontier { cursor, since: since.to_owned(), until: until.to_owned(), + times: Default::default(), } } } impl Cursor for CursorFrontier { + type Key = C::Key; + type Val = C::Val; + type Time = C::Time; + type Diff = C::Diff; + type Storage = C::Storage; #[inline] fn key_valid(&self, storage: &Self::Storage) -> bool { self.cursor.key_valid(storage) } #[inline] fn val_valid(&self, storage: &Self::Storage) -> bool { self.cursor.val_valid(storage) } - #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { self.cursor.key(storage) } - #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { self.cursor.val(storage) } + #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key> { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val> { self.cursor.val(storage) } - #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } - #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } + #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(storage) } #[inline] - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, mut logic: L) { + use columnar::{Push, Clear, Borrow, Index}; let since = self.since.borrow(); let until = self.until.borrow(); - let mut temp: C::Time = ::minimum(); + self.times.clear(); + let mut diffs = Vec::new(); self.cursor.map_times(storage, |time, diff| { - C::clone_time_onto(time, &mut temp); + let mut temp = ::into_owned(time); temp.advance_by(since); if !until.less_equal(&temp) { - logic(&temp, diff); + self.times.push(&temp); + diffs.push(::into_owned(diff)); } - }) + }); + let times_borrowed = self.times.borrow(); + for (i, diff_owned) in diffs.iter().enumerate() { + + let mut diff_container = ::Container::default(); + columnar::Push::push(&mut diff_container, diff_owned); + logic(times_borrowed.get(i), diff_container.borrow().get(0)); + } } #[inline] fn step_key(&mut self, storage: &Self::Storage) { self.cursor.step_key(storage) } - #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { self.cursor.seek_key(storage, key) } + #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>) { self.cursor.seek_key(storage, key) } #[inline] fn step_val(&mut self, storage: &Self::Storage) { self.cursor.step_val(storage) } - #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>) { self.cursor.seek_val(storage, val) } + #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>) { self.cursor.seek_val(storage, val) } #[inline] fn rewind_keys(&mut self, storage: &Self::Storage) { self.cursor.rewind_keys(storage) } #[inline] fn rewind_vals(&mut self, storage: &Self::Storage) { self.cursor.rewind_vals(storage) } @@ -190,16 +187,8 @@ pub struct BatchCursorFrontier { cursor: C, since: Antichain, until: Antichain, -} - -impl WithLayout for BatchCursorFrontier { - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); + /// Container for synthesized times, used by `map_times`. + times: ::Container, } impl BatchCursorFrontier { @@ -208,42 +197,58 @@ impl BatchCursorFrontier { cursor, since: since.to_owned(), until: until.to_owned(), + times: Default::default(), } } } impl> Cursor for BatchCursorFrontier { + type Key = C::Key; + type Val = C::Val; + type Time = C::Time; + type Diff = C::Diff; + type Storage = BatchFrontier; #[inline] fn key_valid(&self, storage: &Self::Storage) -> bool { self.cursor.key_valid(&storage.batch) } #[inline] fn val_valid(&self, storage: &Self::Storage) -> bool { self.cursor.val_valid(&storage.batch) } - #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> Self::Key<'a> { self.cursor.key(&storage.batch) } - #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> Self::Val<'a> { self.cursor.val(&storage.batch) } + #[inline] fn key<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Key> { self.cursor.key(&storage.batch) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> columnar::Ref<'a, Self::Val> { self.cursor.val(&storage.batch) } - #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(&storage.batch) } - #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(&storage.batch) } + #[inline] fn get_key<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_key(&storage.batch) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option> { self.cursor.get_val(&storage.batch) } #[inline] - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times, columnar::Ref<'_, Self::Diff>)>(&mut self, storage: &Self::Storage, mut logic: L) { + use columnar::{Push, Clear, Borrow, Index}; let since = self.since.borrow(); let until = self.until.borrow(); - let mut temp: C::Time = ::minimum(); + self.times.clear(); + let mut diffs = Vec::new(); self.cursor.map_times(&storage.batch, |time, diff| { - C::clone_time_onto(time, &mut temp); + let mut temp = ::into_owned(time); temp.advance_by(since); if !until.less_equal(&temp) { - logic(&temp, diff); + self.times.push(&temp); + diffs.push(::into_owned(diff)); } - }) + }); + let times_borrowed = self.times.borrow(); + for (i, diff_owned) in diffs.iter().enumerate() { + + let mut diff_container = ::Container::default(); + columnar::Push::push(&mut diff_container, diff_owned); + logic(times_borrowed.get(i), diff_container.borrow().get(0)); + } } #[inline] fn step_key(&mut self, storage: &Self::Storage) { self.cursor.step_key(&storage.batch) } - #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: Self::Key<'_>) { self.cursor.seek_key(&storage.batch, key) } + #[inline] fn seek_key(&mut self, storage: &Self::Storage, key: columnar::Ref<'_, Self::Key>) { self.cursor.seek_key(&storage.batch, key) } #[inline] fn step_val(&mut self, storage: &Self::Storage) { self.cursor.step_val(&storage.batch) } - #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: Self::Val<'_>) { self.cursor.seek_val(&storage.batch, val) } + #[inline] fn seek_val(&mut self, storage: &Self::Storage, val: columnar::Ref<'_, Self::Val>) { self.cursor.seek_val(&storage.batch, val) } #[inline] fn rewind_keys(&mut self, storage: &Self::Storage) { self.cursor.rewind_keys(&storage.batch) } #[inline] fn rewind_vals(&mut self, storage: &Self::Storage) { self.cursor.rewind_vals(&storage.batch) } diff --git a/differential-dataflow/src/trace/wrappers/rc.rs b/differential-dataflow/src/trace/wrappers/rc.rs index 76a44c129..d952b45b4 100644 --- a/differential-dataflow/src/trace/wrappers/rc.rs +++ b/differential-dataflow/src/trace/wrappers/rc.rs @@ -78,13 +78,13 @@ pub struct TraceRc { pub wrapper: Rc>>, } -use crate::trace::WithLayout; -impl WithLayout for TraceRc { - type Layout = Tr::Layout; -} - impl TraceReader for TraceRc { + type Key = Tr::Key; + type Val = Tr::Val; + type Time = Tr::Time; + type Diff = Tr::Diff; + type Batch = Tr::Batch; type Storage = Tr::Storage; type Cursor = Tr::Cursor; diff --git a/dogsdogsdogs/Cargo.toml b/dogsdogsdogs/Cargo.toml index 2db38d3e1..72e17555a 100644 --- a/dogsdogsdogs/Cargo.toml +++ b/dogsdogsdogs/Cargo.toml @@ -17,6 +17,7 @@ workspace = true [dependencies] timely = { workspace = true } differential-dataflow = { workspace = true } +columnar = { workspace = true } serde = { version = "1.0", features = ["derive"]} [dev-dependencies] diff --git a/dogsdogsdogs/examples/delta_query.rs b/dogsdogsdogs/examples/delta_query.rs index af9eb5bfd..22474ac50 100644 --- a/dogsdogsdogs/examples/delta_query.rs +++ b/dogsdogsdogs/examples/delta_query.rs @@ -55,9 +55,12 @@ fn main() { // 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.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)); + fn to_alt(_: (&u32, &u32), _: (), t: usize) -> AltNeu { AltNeu::alt(t) } + fn to_neu(_: (&u32, &u32), _: (), t: usize) -> AltNeu { AltNeu::neu(t) } + fn prior(t: &AltNeu) -> usize { t.time.saturating_sub(1) } + let reverse_self_alt = reverse_self.clone().enter_at(inner, to_alt, prior); + let forward_self_neu = forward_self.enter_at(inner, to_neu, prior); + let reverse_self_neu = reverse_self.enter_at(inner, to_neu, prior); // For each relation, we form a delta query driven by changes to that relation. // @@ -101,20 +104,20 @@ fn main() { .clone() .map(|(x,y)| (y,x)) .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))); + .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 .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))); + .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(reverse_self_alt, |(c,b), a, _| Some((*a,*b,*c))); let next_changes = changes1.concat(changes2).concat(changes3).integrate(); diff --git a/dogsdogsdogs/examples/ngo.rs b/dogsdogsdogs/examples/ngo.rs index 340b6c988..f9d16eac2 100644 --- a/dogsdogsdogs/examples/ngo.rs +++ b/dogsdogsdogs/examples/ngo.rs @@ -40,7 +40,7 @@ fn main() { fn triangles(edges: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { // only use forward-pointing edges. let edges = edges.filter(|&(src, dst)| src < dst); @@ -56,8 +56,8 @@ where .arrange_by_self(); // extract ((src, dst), idx) tuples with weights equal to the number of extensions. - 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))); + 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) @@ -79,13 +79,13 @@ where 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))) + .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.clone(), |&dst, &src, &ext| Some(((src, ext), dst))) - .join_core(as_self, |&(src, ext), &dst, &()| 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. diff --git a/dogsdogsdogs/src/altneu.rs b/dogsdogsdogs/src/altneu.rs index 95d69cbc0..063e0fcdc 100644 --- a/dogsdogsdogs/src/altneu.rs +++ b/dogsdogsdogs/src/altneu.rs @@ -14,7 +14,8 @@ use serde::{Deserialize, Serialize}; /// A pair of timestamps, partially ordered by the product order. -#[derive(Debug, Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Serialize, Deserialize)] +#[derive(Debug, Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Serialize, Deserialize, columnar::Columnar)] +#[columnar(derive(Eq, PartialEq, Ord, PartialOrd))] pub struct AltNeu { pub time: T, pub neu: bool, // alt < neu in timestamp comparisons. diff --git a/dogsdogsdogs/src/lib.rs b/dogsdogsdogs/src/lib.rs index f704d5988..69fad60c4 100644 --- a/dogsdogsdogs/src/lib.rs +++ b/dogsdogsdogs/src/lib.rs @@ -144,7 +144,7 @@ where // 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()) + .as_collection(|k,_v| { let kv: (K, V) = columnar::Columnar::into_owned(k); kv }) .distinct() .map(|(k, _v)| k) .arrange_by_self() @@ -187,6 +187,7 @@ where V: ExchangeData+Hash+Default, P: ExchangeData, R: Monoid+Multiply+ExchangeData, + for<'a> R: differential_dataflow::difference::Semigroup>, F: Fn(&P)->K+Clone+'static, { type Prefix = P; diff --git a/dogsdogsdogs/src/operators/count.rs b/dogsdogsdogs/src/operators/count.rs index 0cd363a76..393157614 100644 --- a/dogsdogsdogs/src/operators/count.rs +++ b/dogsdogsdogs/src/operators/count.rs @@ -19,9 +19,9 @@ pub fn count( ) -> VecCollection where G: Scope, - Tr: TraceReader+Clone+'static, - for<'a> Tr::Diff : Semigroup>, - K: Hashable + Ord + Default + 'static, + Tr: TraceReader+Clone+'static, + for<'a> Tr::Diff : Semigroup>, + K: Hashable + Ord + Default + differential_dataflow::Data + 'static, R: Monoid+Multiply+ExchangeData, F: Fn(&P)->K+Clone+'static, P: ExchangeData, diff --git a/dogsdogsdogs/src/operators/half_join.rs b/dogsdogsdogs/src/operators/half_join.rs index ddb2a5979..63aa6308a 100644 --- a/dogsdogsdogs/src/operators/half_join.rs +++ b/dogsdogsdogs/src/operators/half_join.rs @@ -49,7 +49,7 @@ use differential_dataflow::lattice::Lattice; use differential_dataflow::operators::arrange::Arranged; use differential_dataflow::trace::{Cursor, TraceReader}; use differential_dataflow::consolidation::{consolidate, consolidate_updates}; -use differential_dataflow::trace::implementations::BatchContainer; +use differential_dataflow::trace::implementations::{BatchContainer, Coltainer}; /// A binary equijoin that responds to updates on only its first input. /// @@ -85,14 +85,14 @@ where K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, - Tr: TraceReader+Clone+'static, + Tr: TraceReader+Clone+'static, R: Mul, FF: Fn(&G::Timestamp, &mut Antichain) + 'static, - CF: Fn(Tr::TimeGat<'_>, &G::Timestamp) -> bool + 'static, + CF: Fn(columnar::Ref<'_, Tr::Time>, &G::Timestamp) -> bool + 'static, DOut: Clone+'static, - S: FnMut(&K, &V, Tr::Val<'_>)->DOut+'static, + S: FnMut(&K, &V, columnar::Ref<'_, Tr::Val>)->DOut+'static, { - let output_func = move |session: &mut SessionFor, k: &K, v1: &V, v2: Tr::Val<'_>, initial: &G::Timestamp, diff1: &R, output: &mut Vec<(G::Timestamp, Tr::Diff)>| { + let output_func = move |session: &mut SessionFor, k: &K, v1: &V, v2: columnar::Ref<'_, Tr::Val>, initial: &G::Timestamp, diff1: &R, output: &mut Vec<(G::Timestamp, Tr::Diff)>| { for (time, diff2) in output.drain(..) { let diff = diff1.clone() * diff2.clone(); let dout = (output_func(k, v1, v2), time.clone()); @@ -151,11 +151,11 @@ where K: Hashable + ExchangeData, V: ExchangeData, R: ExchangeData + Monoid, - Tr: for<'a> TraceReader+Clone+'static, + Tr: for<'a> TraceReader+Clone+'static, FF: Fn(&G::Timestamp, &mut Antichain) + 'static, - CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, + CF: Fn(columnar::Ref<'_, Tr::Time>, &Tr::Time) -> bool + 'static, Y: Fn(std::time::Instant, usize) -> bool + 'static, - S: FnMut(&mut SessionFor, &K, &V, Tr::Val<'_>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, + S: FnMut(&mut SessionFor, &K, &V, columnar::Ref<'_, Tr::Val>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, CB: ContainerBuilder, { // No need to block physical merging for this operator. @@ -312,12 +312,12 @@ fn process_proposals( ) -> bool where G: Scope, - Tr: for<'a> TraceReader, - CF: Fn(Tr::TimeGat<'_>, &Tr::Time) -> bool + 'static, + Tr: for<'a> TraceReader, + CF: Fn(columnar::Ref<'_, Tr::Time>, &Tr::Time) -> bool + 'static, Y: Fn(Instant, usize) -> bool + 'static, - S: FnMut(&mut SessionFor, &K, &V, Tr::Val<'_>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, + S: FnMut(&mut SessionFor, &K, &V, columnar::Ref<'_, Tr::Val>, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, CB: ContainerBuilder, - K: Ord, + K: Ord + differential_dataflow::Data, V: Ord, R: Monoid, { @@ -327,8 +327,8 @@ where let (mut cursor, storage) = trace.cursor(); let mut yielded = false; - let mut key_con = Tr::KeyContainer::with_capacity(1); - let mut time_con = Tr::TimeContainer::with_capacity(1); + let mut key_con = Coltainer::::with_capacity(1); + let mut time_con = Coltainer::::with_capacity(1); for time in frontier.iter() { time_con.push_own(time); } @@ -345,9 +345,9 @@ where while let Some(val2) = cursor.get_val(&storage) { cursor.map_times(&storage, |t, d| { if comparison(t, initial) { - let mut t = Tr::owned_time(t); + let mut t: Tr::Time = columnar::Columnar::into_owned(t); t.join_assign(time); - output_buffer.push((t, Tr::owned_diff(d))) + output_buffer.push((t, columnar::Columnar::into_owned(d))) } }); consolidate(&mut output_buffer); diff --git a/dogsdogsdogs/src/operators/lookup_map.rs b/dogsdogsdogs/src/operators/lookup_map.rs index 7908ce858..52fd9da73 100644 --- a/dogsdogsdogs/src/operators/lookup_map.rs +++ b/dogsdogsdogs/src/operators/lookup_map.rs @@ -10,7 +10,7 @@ use differential_dataflow::{ExchangeData, VecCollection, AsCollection, Hashable} use differential_dataflow::difference::{IsZero, Semigroup, Monoid}; use differential_dataflow::operators::arrange::Arranged; use differential_dataflow::trace::{Cursor, TraceReader}; -use differential_dataflow::trace::implementations::BatchContainer; +use differential_dataflow::trace::implementations::{BatchContainer, Coltainer}; /// Proposes extensions to a stream of prefixes. /// @@ -28,18 +28,19 @@ pub fn lookup_map( ) -> VecCollection where G: Scope, - Tr: for<'a> TraceReader< - KeyOwn = K, + Tr: TraceReader< + Key = K, Time: std::hash::Hash, - Diff : Semigroup>+Monoid+ExchangeData, + Diff : Monoid+ExchangeData, >+Clone+'static, - K: Hashable + Ord + 'static, + for<'a> Tr::Diff: Semigroup>, + K: Hashable + Ord + differential_dataflow::Data + 'static, F: FnMut(&D, &mut K)+Clone+'static, D: ExchangeData, R: ExchangeData+Monoid, DOut: Clone+'static, ROut: Monoid + 'static, - S: FnMut(&D, &R, Tr::Val<'_>, &Tr::Diff)->(DOut, ROut)+'static, + S: FnMut(&D, &R, columnar::Ref<'_, Tr::Val>, &Tr::Diff)->(DOut, ROut)+'static, { // No need to block physical merging for this operator. arrangement.trace.set_physical_compaction(Antichain::new().borrow()); @@ -91,17 +92,20 @@ where let (mut cursor, storage) = trace.cursor(); // Key container to stage keys for comparison. - let mut key_con = Tr::KeyContainer::with_capacity(1); + let mut key_con = Coltainer::::with_capacity(1); for &mut (ref prefix, ref time, ref mut diff) in prefixes.iter_mut() { if !frontier2.less_equal(time) { logic2(prefix, &mut key1); key_con.clear(); key_con.push_own(&key1); - cursor.seek_key(&storage, key_con.index(1)); - if cursor.get_key(&storage) == Some(key_con.index(1)) { + cursor.seek_key(&storage, key_con.index(0)); + if cursor.get_key(&storage) == Some(key_con.index(0)) { while let Some(value) = cursor.get_val(&storage) { let mut count = Tr::Diff::zero(); cursor.map_times(&storage, |t, d| { - if Tr::owned_time(t).less_equal(time) { count.plus_equals(&d); } + if ::into_owned(t).less_equal(time) { + let d: Tr::Diff = columnar::Columnar::into_owned(d); + count.plus_equals(&d); + } }); if !count.is_zero() { let (dout, rout) = output_func(prefix, diff, value, &count); diff --git a/dogsdogsdogs/src/operators/propose.rs b/dogsdogsdogs/src/operators/propose.rs index 9f2e69f78..263e760e9 100644 --- a/dogsdogsdogs/src/operators/propose.rs +++ b/dogsdogsdogs/src/operators/propose.rs @@ -20,22 +20,23 @@ pub fn propose( ) -> VecCollection where G: Scope, - Tr: for<'a> TraceReader< - KeyOwn = K, - ValOwn = V, + Tr: TraceReader< + Key = K, + Val = V, Time: std::hash::Hash, - Diff: Monoid+Multiply+ExchangeData+Semigroup>, + Diff: Monoid+Multiply+ExchangeData, >+Clone+'static, - K: Hashable + Default + Ord + 'static, + for<'a> Tr::Diff: Semigroup>, + K: Hashable + Default + Ord + differential_dataflow::Data + 'static, F: Fn(&P)->K+Clone+'static, P: ExchangeData, - V: Clone + 'static, + V: Clone + differential_dataflow::Data + 'static, { crate::operators::lookup_map( prefixes, arrangement, move |p: &P, k: &mut K | { *k = key_selector(p); }, - move |prefix, diff, value, sum| ((prefix.clone(), Tr::owned_val(value)), diff.clone().multiply(sum)), + move |prefix, diff, value, sum| ((prefix.clone(), columnar::Columnar::into_owned(value)), diff.clone().multiply(sum)), Default::default(), Default::default(), Default::default(), @@ -54,22 +55,23 @@ pub fn propose_distinct( ) -> VecCollection where G: Scope, - Tr: for<'a> TraceReader< - KeyOwn = K, - ValOwn = V, + Tr: TraceReader< + Key = K, + Val = V, Time: std::hash::Hash, - Diff : Semigroup>+Monoid+Multiply+ExchangeData, + Diff : Monoid+Multiply+ExchangeData, >+Clone+'static, - K: Hashable + Default + Ord + 'static, + for<'a> Tr::Diff: Semigroup>, + K: Hashable + Default + Ord + differential_dataflow::Data + 'static, F: Fn(&P)->K+Clone+'static, P: ExchangeData, - V: Clone + 'static, + V: Clone + differential_dataflow::Data + 'static, { crate::operators::lookup_map( prefixes, arrangement, move |p: &P, k: &mut K| { *k = key_selector(p); }, - move |prefix, diff, value, _sum| ((prefix.clone(), Tr::owned_val(value)), diff.clone()), + move |prefix, diff, value, _sum| ((prefix.clone(), columnar::Columnar::into_owned(value)), diff.clone()), Default::default(), Default::default(), Default::default(), diff --git a/dogsdogsdogs/src/operators/validate.rs b/dogsdogsdogs/src/operators/validate.rs index f610d55be..406f026c9 100644 --- a/dogsdogsdogs/src/operators/validate.rs +++ b/dogsdogsdogs/src/operators/validate.rs @@ -19,13 +19,14 @@ pub fn validate( ) -> VecCollection where G: Scope, - Tr: for<'a> TraceReader< - KeyOwn = (K, V), + Tr: TraceReader< + Key = (K, V), Time: std::hash::Hash, - Diff : Semigroup>+Monoid+Multiply+ExchangeData, + Diff : Monoid+Multiply+ExchangeData, >+Clone+'static, - K: Ord+Hash+Clone+Default + 'static, - V: ExchangeData+Hash+Default, + for<'a> Tr::Diff: Semigroup>, + K: Ord+Hash+Clone+Default+differential_dataflow::Data + 'static, + V: ExchangeData+Hash+Default+differential_dataflow::Data, F: Fn(&P)->K+Clone+'static, P: ExchangeData, { diff --git a/experiments/Cargo.toml b/experiments/Cargo.toml index edcf55fb3..bc62fc571 100644 --- a/experiments/Cargo.toml +++ b/experiments/Cargo.toml @@ -10,6 +10,7 @@ publish = false workspace = true [dependencies] +columnar = { workspace = true } core_affinity = "0.5.9" rand="0.3.13" timely = { workspace = true } diff --git a/experiments/src/bin/deals-interactive.rs b/experiments/src/bin/deals-interactive.rs index 3ea53cd14..a73a38f6c 100644 --- a/experiments/src/bin/deals-interactive.rs +++ b/experiments/src/bin/deals-interactive.rs @@ -210,7 +210,7 @@ fn interactive( tc_2: VecCollection, sg_x: VecCollection ) -> VecCollection -where G::Timestamp: Lattice{ +where G::Timestamp: Lattice+differential_dataflow::Data{ // descendants of tc_1: let tc_1_enter = tc_1.clone(); @@ -233,7 +233,7 @@ where G::Timestamp: Lattice{ tc_2.map(|x| (x,x)) .iterate(|scope, inner| edges_q2 - .as_collection(|&k,&v| (v,k)) + .as_collection(|&k,&v| (v, k)) .enter(&scope) .join_core(inner.arrange_by_key(), |_,&y,&q| [(y,q)]) .concat(tc_2_enter.enter(&scope).map(|x| (x,x))) @@ -252,7 +252,7 @@ where G::Timestamp: Lattice{ let magic = sg_x.iterate(|scope, inner| edges_magic - .as_collection(|&k,&v| (v,k)) + .as_collection(|&k,&v| (v, k)) .enter(&scope) .semijoin(inner) .map(|(_x,y)| y) diff --git a/experiments/src/bin/graphs-interactive-alt.rs b/experiments/src/bin/graphs-interactive-alt.rs index 6e5b0ffaa..8296fdcf8 100644 --- a/experiments/src/bin/graphs-interactive-alt.rs +++ b/experiments/src/bin/graphs-interactive-alt.rs @@ -65,20 +65,20 @@ fn main() { // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(state_indexed.clone(), |&query, &(), &state| Some((query, state))) + .join_core(state_indexed.clone(), |query, (), state| Some((query, state))) .probe_with(&mut probe); // Q2: One-hop lookups on `state`: q2 .arrange_by_self() - .join_core(graph_indexed.clone(), |&query, &(), &friend| Some((friend, query))) - .join_core(state_indexed.clone(), |_friend, &query, &state| Some((query, state))) + .join_core(graph_indexed.clone(), |query, (), friend| Some((friend, query))) + .join_core(state_indexed.clone(), |_friend, query, state| Some((query, state))) .probe_with(&mut probe); // Q3: Two-hop lookups on `state`: q3 .arrange_by_self() - .join_core(graph_indexed.clone(), |&query, &(), &friend| Some((friend, query))) - .join_core(graph_indexed.clone(), |_friend, &query, &friend2| Some((friend2, query))) - .join_core(state_indexed, |_friend2, &query, &state| Some((query, state))) + .join_core(graph_indexed.clone(), |query, (), friend| Some((friend, query))) + .join_core(graph_indexed.clone(), |_friend, query, friend2| Some((friend2, query))) + .join_core(state_indexed, |_friend2, query, state| Some((query, state))) .probe_with(&mut probe); // Q4: Shortest path queries: @@ -97,20 +97,20 @@ fn main() { // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(state.clone().arrange_by_key(), |&query, &(), &state| Some((query, state))) + .join_core(state.clone().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.clone().arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(state.clone().arrange_by_key(), |_friend, &query, &state| Some((query, state))) + .join_core(graph.clone().arrange_by_key(), |query, (), friend| Some((friend, query))) + .join_core(state.clone().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.clone().arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(graph.clone().arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query))) - .join_core(state.arrange_by_key(), |_friend2, &query, &state| Some((query, state))) + .join_core(graph.clone().arrange_by_key(), |query, (), friend| Some((friend, query))) + .join_core(graph.clone().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: @@ -265,19 +265,19 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+differential_dataflow::Data { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.clone().join_core(forward_graph.clone(), |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.clone().join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.clone().join_core(forward_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let forward2 = forward1.clone().join_core(forward_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.clone().join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let reverse2 = reverse1.clone().join_core(reverse_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let forward = forward0.concat(forward1).concat(forward2); let reverse = reverse0.concat(reverse1).concat(reverse2); @@ -292,7 +292,7 @@ fn _bidijkstra( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+differential_dataflow::Data { goals.scope().iterative::(|inner| { @@ -316,7 +316,7 @@ where G::Timestamp: Lattice+Ord { let reached = forward_collection.clone() .join_map(reverse_collection.clone(), |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .semijoin(goals.clone()); let active = @@ -333,10 +333,10 @@ where G::Timestamp: Lattice+Ord { .map(|(med, (src, dist))| (src, (med, dist))) .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) - .join_core(forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) + .join_core(forward_graph, { fn step(_: usize, val: (usize, &u32), next: usize) -> Option<(usize, (usize, u32))> { Some((next, (val.0, *val.1+1))) } step }) .concat(forward_collection) .map(|(next, (src, dist))| ((next, src), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); forward.set(forward_next); @@ -348,10 +348,10 @@ where G::Timestamp: Lattice+Ord { .map(|(med, (rev, dist))| (rev, (med, dist))) .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) - .join_core(reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) + .join_core(reverse_graph, { fn step(_: usize, val: (usize, &u32), next: usize) -> Option<(usize, (usize, u32))> { Some((next, (val.0, *val.1+1))) } step }) .concat(reverse_collection) .map(|(next, (rev, dist))| ((next, rev), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); reverse.set(reverse_next); @@ -362,12 +362,12 @@ where G::Timestamp: Lattice+Ord { fn connected_components(graph: Arrange) -> VecCollection -where G::Timestamp: Lattice + std::hash::Hash { +where G::Timestamp: Lattice + std::hash::Hash + differential_dataflow::Data { // each edge (x,y) means that we need at least a label for the min of x and y. let nodes = graph.clone() - .as_collection(|&k,&v| { + .as_collection(|k,v| { let min = std::cmp::min(k,v); (min, min) }) @@ -383,7 +383,7 @@ 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) diff --git a/experiments/src/bin/graphs-interactive-neu-zwei.rs b/experiments/src/bin/graphs-interactive-neu-zwei.rs index 35a118cd9..d5f28931e 100644 --- a/experiments/src/bin/graphs-interactive-neu-zwei.rs +++ b/experiments/src/bin/graphs-interactive-neu-zwei.rs @@ -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.clone(), |&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.clone(), |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 => { @@ -233,19 +233,19 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+differential_dataflow::Data { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.clone().join_core(forward_graph.clone(), |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.clone().join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.clone().join_core(forward_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let forward2 = forward1.clone().join_core(forward_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.clone().join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let reverse2 = reverse1.clone().join_core(reverse_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let forward = forward0.concat(forward1).concat(forward2); let reverse = reverse0.concat(reverse1).concat(reverse2); diff --git a/experiments/src/bin/graphs-interactive-neu.rs b/experiments/src/bin/graphs-interactive-neu.rs index 7e27e7ed8..a83a755d8 100644 --- a/experiments/src/bin/graphs-interactive-neu.rs +++ b/experiments/src/bin/graphs-interactive-neu.rs @@ -64,20 +64,20 @@ fn main() { // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(state_indexed.clone(), |&query, &(), &state| Some((query, state))) + .join_core(state_indexed.clone(), |query, (), state| Some((query, state))) .probe_with(&mut probe); // Q2: One-hop lookups on `state`: q2 .arrange_by_self() - .join_core(graph_indexed.clone(), |&query, &(), &friend| Some((friend, query))) - .join_core(state_indexed.clone(), |_friend, &query, &state| Some((query, state))) + .join_core(graph_indexed.clone(), |query, (), friend| Some((friend, query))) + .join_core(state_indexed.clone(), |_friend, query, state| Some((query, state))) .probe_with(&mut probe); // Q3: Two-hop lookups on `state`: q3 .arrange_by_self() - .join_core(graph_indexed.clone(), |&query, &(), &friend| Some((friend, query))) - .join_core(graph_indexed.clone(), |_friend, &query, &friend2| Some((friend2, query))) - .join_core(state_indexed, |_friend2, &query, &state| Some((query, state))) + .join_core(graph_indexed.clone(), |query, (), friend| Some((friend, query))) + .join_core(graph_indexed.clone(), |_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,20 +93,20 @@ fn main() { // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(state.clone().arrange_by_key(), |&query, &(), &state| Some((query, state))) + .join_core(state.clone().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.clone().arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(state.clone().arrange_by_key(), |_friend, &query, &state| Some((query, state))) + .join_core(graph.clone().arrange_by_key(), |query, (), friend| Some((friend, query))) + .join_core(state.clone().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.clone().arrange_by_key(), |&query, &(), &friend| Some((friend, query))) - .join_core(graph.clone().arrange_by_key(), |_friend, &query, &friend2| Some((friend2, query))) - .join_core(state.arrange_by_key(), |_friend2, &query, &state| Some((query, state))) + .join_core(graph.clone().arrange_by_key(), |query, (), friend| Some((friend, query))) + .join_core(graph.clone().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: @@ -298,19 +298,19 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+differential_dataflow::Data { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.clone().join_core(forward_graph.clone(), |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.clone().join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.clone().join_core(forward_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let forward2 = forward1.clone().join_core(forward_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.clone().join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let reverse2 = reverse1.clone().join_core(reverse_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let forward = forward0.concat(forward1).concat(forward2); let reverse = reverse0.concat(reverse1).concat(reverse2); @@ -325,7 +325,7 @@ fn _bidijkstra( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+differential_dataflow::Data { goals.scope().iterative::(|inner| { @@ -349,7 +349,7 @@ where G::Timestamp: Lattice+Ord { let reached = forward_collection.clone() .join_map(reverse_collection.clone(), |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .semijoin(goals.clone()); let active = @@ -366,10 +366,10 @@ where G::Timestamp: Lattice+Ord { .map(|(med, (src, dist))| (src, (med, dist))) .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) - .join_core(forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) + .join_core(forward_graph, { fn step(_: usize, val: (usize, &u32), next: usize) -> Option<(usize, (usize, u32))> { Some((next, (val.0, *val.1+1))) } step }) .concat(forward_collection) .map(|(next, (src, dist))| ((next, src), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); forward.set(forward_next); @@ -381,10 +381,10 @@ where G::Timestamp: Lattice+Ord { .map(|(med, (rev, dist))| (rev, (med, dist))) .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) - .join_core(reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) + .join_core(reverse_graph, { fn step(_: usize, val: (usize, &u32), next: usize) -> Option<(usize, (usize, u32))> { Some((next, (val.0, *val.1+1))) } step }) .concat(reverse_collection) .map(|(next, (rev, dist))| ((next, rev), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); reverse.set(reverse_next); diff --git a/experiments/src/bin/graphs-interactive.rs b/experiments/src/bin/graphs-interactive.rs index 2ec74e93f..f2c172e63 100644 --- a/experiments/src/bin/graphs-interactive.rs +++ b/experiments/src/bin/graphs-interactive.rs @@ -57,24 +57,24 @@ fn main() { // Q1: Point lookups on `state`: q1 .arrange_by_self() - .join_core(state_indexed.clone(), |&query, &(), &state| Some((query, state))) + .join_core(state_indexed.clone(), |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.clone(), |&query, &(), &friend| Some((friend, query))) - .join_core(state_indexed.clone(), |_friend, &query, &state| Some((query, state))) + .join_core(graph_indexed.clone(), |query, (), friend| Some((friend, query))) + .join_core(state_indexed.clone(), |_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.clone(), |&query, &(), &friend| Some((friend, query))) - .join_core(graph_indexed.clone(), |_friend, &query, &friend2| Some((friend2, query))) - .join_core(state_indexed, |_friend2, &query, &state| Some((query, state))) + .join_core(graph_indexed.clone(), |query, (), friend| Some((friend, query))) + .join_core(graph_indexed.clone(), |_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)) @@ -202,19 +202,19 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+differential_dataflow::Data { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); // Q3: Two-hop lookups on `state`: let forward0 = sources.map(|x| (x, (x,0))); - let forward1 = forward0.clone().join_core(forward_graph.clone(), |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); - let forward2 = forward1.clone().join_core(forward_graph, |&_, &(source,dist), &friend| Some((friend, (source, dist+1)))); + let forward1 = forward0.clone().join_core(forward_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let forward2 = forward1.clone().join_core(forward_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let reverse0 = targets.map(|x| (x, (x,0))); - let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); - let reverse2 = reverse1.clone().join_core(reverse_graph, |&_, &(target,dist), &friend| Some((friend, (target, dist+1)))); + let reverse1 = reverse0.clone().join_core(reverse_graph.clone(), { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); + let reverse2 = reverse1.clone().join_core(reverse_graph, { fn step(_: usize, val: (usize, &u32), friend: usize) -> Option<(usize, (usize, u32))> { Some((friend, (val.0, *val.1+1))) } step }); let forward = forward0.concat(forward1).concat(forward2); let reverse = reverse0.concat(reverse1).concat(reverse2); @@ -229,7 +229,7 @@ fn _bidijkstra( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+differential_dataflow::Data { goals.scope().iterative::(|inner| { @@ -253,7 +253,7 @@ where G::Timestamp: Lattice+Ord { let reached = forward_collection.clone() .join_map(reverse_collection.clone(), |_, &(src,d1), &(dst,d2)| ((src, dst), d1 + d2)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .semijoin(goals.clone()); let active = @@ -270,10 +270,10 @@ where G::Timestamp: Lattice+Ord { .map(|(med, (src, dist))| (src, (med, dist))) .semijoin(forward_active) .map(|(src, (med, dist))| (med, (src, dist))) - .join_core(forward_graph, |_med, &(src, dist), &next| Some((next, (src, dist+1)))) + .join_core(forward_graph, { fn step(_: usize, val: (usize, &u32), next: usize) -> Option<(usize, (usize, u32))> { Some((next, (val.0, *val.1+1))) } step }) .concat(forward_collection) .map(|(next, (src, dist))| ((next, src), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .map(|((next, src), dist)| (next, (src, dist))); forward.set(forward_next); @@ -285,10 +285,10 @@ where G::Timestamp: Lattice+Ord { .map(|(med, (rev, dist))| (rev, (med, dist))) .semijoin(reverse_active) .map(|(rev, (med, dist))| (med, (rev, dist))) - .join_core(reverse_graph, |_med, &(rev, dist), &next| Some((next, (rev, dist+1)))) + .join_core(reverse_graph, { fn step(_: usize, val: (usize, &u32), next: usize) -> Option<(usize, (usize, u32))> { Some((next, (val.0, *val.1+1))) } step }) .concat(reverse_collection) .map(|(next, (rev, dist))| ((next, rev), dist)) - .reduce(|_key, s, t| t.push((*s[0].0, 1))) + .reduce(|_key, s, t: &mut Vec<(u32, isize)>| t.push((*s[0].0, 1))) .map(|((next,rev), dist)| (next, (rev, dist))); reverse.set(reverse_next); diff --git a/experiments/src/bin/graphs-static.rs b/experiments/src/bin/graphs-static.rs index 16cbbd6dd..5961528a0 100644 --- a/experiments/src/bin/graphs-static.rs +++ b/experiments/src/bin/graphs-static.rs @@ -85,7 +85,7 @@ fn main() { let mut reverse = worker.dataflow(|scope| { forward .import(scope) - .as_collection(|&k,&v| (v,k)) + .as_collection(|&k,&v| (v, k)) .arrange_by_key() .trace }); @@ -121,7 +121,7 @@ fn reach> ( let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); let result = - graph.join_core(inner_collection.arrange_by_self(), |_src,&dst,&()| Some(dst)) + graph.join_core(inner_collection.arrange_by_self(), |_src,&dst,()| Some(dst)) .concat(roots) .threshold_total(|_,_| 1); @@ -179,8 +179,8 @@ fn connected_components>( let (inner, inner_collection) = Variable::new(scope, Product::new(Default::default(), 1)); let labels = inner_collection.clone().arrange_by_key(); - let f_prop = labels.clone().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.clone().join_core(forward, |_k,l: &Node,d| Some((*d,*l))); + let r_prop = labels.join_core(reverse, |_k,l: &Node,d| Some((*d,*l))); use timely::dataflow::operators::vec::{Map, Delay}; use timely::dataflow::operators::Concat; diff --git a/experiments/src/bin/graphs.rs b/experiments/src/bin/graphs.rs index ea7420120..3bd6d6883 100644 --- a/experiments/src/bin/graphs.rs +++ b/experiments/src/bin/graphs.rs @@ -103,7 +103,7 @@ fn reach> ( // 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)) + graph.join_core(inner.arrange_by_self(), |_src,dst,()| Some(dst)) .concat(roots) .distinct_total() }) @@ -123,7 +123,7 @@ fn bfs> ( let graph = graph.enter(&scope); let roots = roots.enter(&scope); - graph.join_core(inner.arrange_by_key(), |_src,&dest,&dist| [(dest, dist+1)]) + 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))) }) diff --git a/experiments/src/bin/graspan1.rs b/experiments/src/bin/graspan1.rs index 507f5c46b..bbc483a14 100644 --- a/experiments/src/bin/graspan1.rs +++ b/experiments/src/bin/graspan1.rs @@ -44,7 +44,7 @@ fn main() { let (labels, labels_collection) = Variable::new(inner, Product::new(Default::default(), 1)); let next = - labels_collection.join_core(edges, |_b, a, c| Some((*c, *a))) + labels_collection.join_core(edges, |_b, a: &Node, c: &Node| Some((*c, *a))) .concat(nodes) .arrange::, ValBuilder<_,_,_,_>, ValSpine<_,_,_,_>>() // .distinct_total_core::(); diff --git a/experiments/src/bin/graspan2.rs b/experiments/src/bin/graspan2.rs index 4ce8574a9..a80a22939 100644 --- a/experiments/src/bin/graspan2.rs +++ b/experiments/src/bin/graspan2.rs @@ -1,3 +1,5 @@ +#![recursion_limit = "512"] + use std::io::{BufRead, BufReader}; use std::fs::File; diff --git a/experiments/src/bin/multitemporal.rs b/experiments/src/bin/multitemporal.rs index eb602efe2..6f10b570d 100644 --- a/experiments/src/bin/multitemporal.rs +++ b/experiments/src/bin/multitemporal.rs @@ -171,7 +171,8 @@ fn main() { mod pair { /// A pair of timestamps, partially ordered by the product order. - #[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Serialize, Deserialize)] + #[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Serialize, Deserialize, columnar::Columnar)] + #[columnar(derive(Eq, PartialEq, Ord, PartialOrd))] pub struct Pair { pub first: S, pub second: T, diff --git a/interactive/Cargo.toml b/interactive/Cargo.toml index 6a03116ac..4f7819ae7 100644 --- a/interactive/Cargo.toml +++ b/interactive/Cargo.toml @@ -11,6 +11,7 @@ workspace = true [dependencies] bincode = "1" +columnar = { workspace = true } serde = { version = "1", features = ["derive"]} differential-dataflow = { workspace = true } differential-dogs3 = { path = "../dogsdogsdogs" } diff --git a/interactive/src/plan/join.rs b/interactive/src/plan/join.rs index 956fab419..369f01a5d 100644 --- a/interactive/src/plan/join.rs +++ b/interactive/src/plan/join.rs @@ -96,12 +96,12 @@ impl Render for Join { arrange1 .join_core(arrange2, |keys, vals1, vals2| { - Some( - keys.iter().cloned() - .chain(vals1.iter().cloned()) - .chain(vals2.iter().cloned()) - .collect() - ) + use columnar::common::Index as ColumnarIndex; + let mut result = Vec::new(); + for i in 0..keys.len() { result.push(columnar::Columnar::into_owned(keys.get(i))); } + for i in 0..vals1.len() { result.push(columnar::Columnar::into_owned(vals1.get(i))); } + for i in 0..vals2.len() { result.push(columnar::Columnar::into_owned(vals2.get(i))); } + Some(result) }) } } diff --git a/interactive/src/plan/mod.rs b/interactive/src/plan/mod.rs index deff18a4a..ddcd52a48 100644 --- a/interactive/src/plan/mod.rs +++ b/interactive/src/plan/mod.rs @@ -172,7 +172,7 @@ impl Render for Plan { let output = input.reduce_abelian::<_,KeyBuilder<_,_,_>,KeySpine<_,_,_>>("Distinct", move |_,_,t| t.push(((), 1))); arrangements.set_unkeyed(&self, &output.trace); - output.as_collection(|k,&()| k.clone()) + output.as_collection(|k,()| columnar::Columnar::into_owned(k)) }, Plan::Concat(concat) => { @@ -192,7 +192,7 @@ impl Render for Plan { } Plan::Consolidate(consolidate) => { if let Some(mut trace) = arrangements.get_unkeyed(&self) { - trace.import(scope).as_collection(|k,&()| k.clone()) + trace.import(scope).as_collection(|k,()| columnar::Columnar::into_owned(k)) } else { consolidate.render(scope, collections, arrangements).consolidate() @@ -209,7 +209,7 @@ impl Render for Plan { .get_unkeyed(self) .expect(&format!("Failed to find source collection: {:?}", source)) .import(scope) - .as_collection(|k,()| k.to_vec()) + .as_collection(|k,()| columnar::Columnar::into_owned(k)) }, Plan::Inspect(text, plan) => { let text = text.clone(); diff --git a/interactive/src/plan/sfw.rs b/interactive/src/plan/sfw.rs index 6a5c82fc8..fdf81d030 100644 --- a/interactive/src/plan/sfw.rs +++ b/interactive/src/plan/sfw.rs @@ -141,8 +141,8 @@ impl Render for MultiwayJoin { .get_unkeyed(&plan) .expect("Surely we just ensured this") .import(scope) - .as_collection(|val,&()| val.clone()) - .map(move |tuple| attributes_init.iter().map(|&(attr,_)| + .as_collection(|val,()| columnar::Columnar::into_owned(val)) + .map(move |tuple: Vec| attributes_init.iter().map(|&(attr,_)| tuple[attr].clone()).collect::>() ); diff --git a/server/dataflows/degr_dist/src/lib.rs b/server/dataflows/degr_dist/src/lib.rs index 95d10bee1..593b9c7eb 100644 --- a/server/dataflows/degr_dist/src/lib.rs +++ b/server/dataflows/degr_dist/src/lib.rs @@ -16,7 +16,7 @@ pub fn build((dataflow, handles, probe, _timer, args): Environment) -> Result<() .get_mut::>>>(&args[0])? .borrow_mut().as_mut().unwrap() .import(dataflow) - .as_collection(|&src,_dst| src) + .as_collection(|src,_dst| src) .count_total() .map(|(_deg, cnt)| cnt as usize) .count_total() diff --git a/server/dataflows/neighborhood/src/lib.rs b/server/dataflows/neighborhood/src/lib.rs index 8d5be369b..dae762f50 100644 --- a/server/dataflows/neighborhood/src/lib.rs +++ b/server/dataflows/neighborhood/src/lib.rs @@ -23,13 +23,13 @@ pub fn build((dataflow, handles, probe, _timer, args): Environment) -> Result<() query .map(|x| (x, x)) - .join_core(edges.clone(), |_n, &q, &d| Some((d, q))) // one hop - .join_core(edges.clone(), |_n, &q, &d| Some((d, q))) // two hops - .join_core(edges, |_n, &q, &d| Some((d, q))) // three hops + .join_core(edges.clone(), |_n, q, d| Some((d, q))) // one hop + .join_core(edges.clone(), |_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)) .probe_with(probe); Ok(()) -} \ No newline at end of file +} diff --git a/server/dataflows/reachability/src/lib.rs b/server/dataflows/reachability/src/lib.rs index e5f46ad16..9fd0f164e 100644 --- a/server/dataflows/reachability/src/lib.rs +++ b/server/dataflows/reachability/src/lib.rs @@ -23,7 +23,7 @@ pub fn build((dataflow, handles, probe, _timer, args): Environment) -> Result<() let edges = edges.enter(&scope); let roots = roots.enter(&scope); dists.arrange_by_self() - .join_core(edges, |_src, _, &dst| Some(dst)) + .join_core(edges, |_src, _, dst| Some(dst)) .concat(roots) .distinct() })