diff --git a/differential-dataflow/Cargo.toml b/differential-dataflow/Cargo.toml index e961eb3ea..8637ef621 100644 --- a/differential-dataflow/Cargo.toml +++ b/differential-dataflow/Cargo.toml @@ -28,7 +28,6 @@ mimalloc = "0.1.48" [dependencies] columnar = { workspace = true } -columnation = "0.1.1" fnv="1.0.2" paste = "1.0" serde = { version = "1.0", features = ["derive"] } diff --git a/differential-dataflow/examples/bfs.rs b/differential-dataflow/examples/bfs.rs index af875d5d5..96518ca8e 100644 --- a/differential-dataflow/examples/bfs.rs +++ b/differential-dataflow/examples/bfs.rs @@ -93,7 +93,7 @@ fn main() { // returns pairs (n, s) indicating node n can be reached from a root in s steps. fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); diff --git a/differential-dataflow/examples/columnar.rs b/differential-dataflow/examples/columnar.rs.disabled similarity index 100% rename from differential-dataflow/examples/columnar.rs rename to differential-dataflow/examples/columnar.rs.disabled diff --git a/differential-dataflow/examples/dynamic.rs b/differential-dataflow/examples/dynamic.rs index a844c28bb..4d67f06a1 100644 --- a/differential-dataflow/examples/dynamic.rs +++ b/differential-dataflow/examples/dynamic.rs @@ -93,7 +93,7 @@ fn main() { // returns pairs (n, s) indicating node n can be reached from a root in s steps. fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { use timely::order::Product; use iterate::Variable; diff --git a/differential-dataflow/examples/graspan.rs b/differential-dataflow/examples/graspan.rs index 79c7fd184..273a27155 100644 --- a/differential-dataflow/examples/graspan.rs +++ b/differential-dataflow/examples/graspan.rs @@ -79,7 +79,7 @@ type Arrange = Arranged::Tim /// /// An edge variable provides arranged representations of its contents, even before they are /// completely defined, in support of recursively defined productions. -pub struct EdgeVariable> { +pub struct EdgeVariable> { variable: VecVariable, collection: VecCollection, current: VecCollection, @@ -87,7 +87,7 @@ pub struct EdgeVariable> { reverse: Option>, } -impl> EdgeVariable { +impl> EdgeVariable { /// Creates a new variable initialized with `source`. pub fn from(source: VecCollection, step: ::Summary) -> Self { let (variable, collection) = VecVariable::new(&mut source.scope(), step); @@ -129,7 +129,7 @@ impl> EdgeVariable { } /// Handles to inputs and outputs of a computation. -pub struct RelationHandles { +pub struct RelationHandles { /// An input handle supporting arbitrary changes. pub input: InputSession, /// An output trace handle which can be used in other computations. @@ -153,7 +153,7 @@ impl Query { /// Creates a dataflow implementing the query, and returns input and trace handles. pub fn render_in(&self, scope: &mut G) -> BTreeMap> where - G: Scope, + G: Scope, { // Create new input (handle, stream) pairs let mut input_map = BTreeMap::new(); diff --git a/differential-dataflow/examples/interpreted.rs b/differential-dataflow/examples/interpreted.rs index 4c6e3429c..e5233686b 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. diff --git a/differential-dataflow/examples/monoid-bfs.rs b/differential-dataflow/examples/monoid-bfs.rs index cce06674e..fc2c9d9bb 100644 --- a/differential-dataflow/examples/monoid-bfs.rs +++ b/differential-dataflow/examples/monoid-bfs.rs @@ -11,7 +11,7 @@ use differential_dataflow::lattice::Lattice; type Node = u32; type Edge = (Node, Node); -#[derive(Copy, Ord, PartialOrd, Eq, PartialEq, Debug, Clone, Serialize, Deserialize, Hash)] +#[derive(Copy, Ord, PartialOrd, Eq, PartialEq, Debug, Clone, Serialize, Deserialize, Hash, columnar::Columnar)] pub struct MinSum { value: u32, } @@ -124,7 +124,7 @@ fn main() { // returns pairs (n, s) indicating node n can be reached from a root in s steps. fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { // repeatedly update minimal distances each node can be reached from each root roots.scope().iterative::(|scope| { diff --git a/differential-dataflow/examples/multitemporal.rs b/differential-dataflow/examples/multitemporal.rs index 63040c6f6..2d68ce129 100644 --- a/differential-dataflow/examples/multitemporal.rs +++ b/differential-dataflow/examples/multitemporal.rs @@ -136,7 +136,7 @@ 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)] pub struct Pair { pub first: S, pub second: T, @@ -223,7 +223,7 @@ mod vector { use serde::{Deserialize, Serialize}; /// A pair of timestamps, partially ordered by the product order. - #[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Debug, Serialize, Deserialize)] + #[derive(Hash, Default, Clone, Eq, PartialEq, Ord, PartialOrd, Debug, Serialize, Deserialize, columnar::Columnar)] pub struct Vector { pub vector: Vec, } diff --git a/differential-dataflow/examples/pagerank.rs b/differential-dataflow/examples/pagerank.rs index d779ed36e..676d9c2ae 100644 --- a/differential-dataflow/examples/pagerank.rs +++ b/differential-dataflow/examples/pagerank.rs @@ -79,7 +79,7 @@ fn main() { // to its PageRank in the input graph `edges`. fn pagerank(iters: Iter, edges: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { // initialize many surfers at each node. let nodes = diff --git a/differential-dataflow/examples/progress.rs b/differential-dataflow/examples/progress.rs index 634e43d56..cebd304a9 100644 --- a/differential-dataflow/examples/progress.rs +++ b/differential-dataflow/examples/progress.rs @@ -13,6 +13,15 @@ use differential_dataflow::lattice::Lattice; use timely::progress::{Timestamp, Source, Target, Location}; use timely::progress::timestamp::PathSummary; +// Columnar-compatible representations of Source and Target as (node, port) tuples. +type Src = (usize, usize); +type Tgt = (usize, usize); + +fn src(s: Source) -> Src { (s.node, s.port) } +fn tgt(t: Target) -> Tgt { (t.node, t.port) } +fn to_source(s: &Src) -> Source { Source::new(s.0, s.1) } +fn to_target(t: &Tgt) -> Target { Target::new(t.0, t.1) } + fn main() { timely::execute_from_args(std::env::args(), move |worker| { @@ -46,15 +55,15 @@ fn main() { // A PageRank-like graph, as represented here: // https://github.com/TimelyDataflow/diagnostics/blob/master/examples/pagerank.png - nodes.insert((Target::new(2, 0), Source::new(2, 0), 1)); - nodes.insert((Target::new(3, 0), Source::new(3, 0), 0)); - nodes.insert((Target::new(3, 1), Source::new(3, 0), 0)); - nodes.insert((Target::new(4, 0), Source::new(4, 0), 0)); + nodes.insert((tgt(Target::new(2, 0)), src(Source::new(2, 0)), 1)); + nodes.insert((tgt(Target::new(3, 0)), src(Source::new(3, 0)), 0)); + nodes.insert((tgt(Target::new(3, 1)), src(Source::new(3, 0)), 0)); + nodes.insert((tgt(Target::new(4, 0)), src(Source::new(4, 0)), 0)); - edges.insert((Source::new(1, 0), Target::new(3, 0))); - edges.insert((Source::new(3, 0), Target::new(4, 0))); - edges.insert((Source::new(4, 0), Target::new(2, 0))); - edges.insert((Source::new(2, 0), Target::new(3, 1))); + edges.insert((src(Source::new(1, 0)), tgt(Target::new(3, 0)))); + edges.insert((src(Source::new(3, 0)), tgt(Target::new(4, 0)))); + edges.insert((src(Source::new(4, 0)), tgt(Target::new(2, 0)))); + edges.insert((src(Source::new(2, 0)), tgt(Target::new(3, 1)))); // Initially no capabilities. nodes.advance_to(1); nodes.flush(); @@ -115,17 +124,17 @@ fn main() { /// The computation to determine this, and to maintain it as times change, is an iterative /// computation that propagates times and maintains the minimal elements at each location. fn frontier( - nodes: VecCollection, - edges: VecCollection, + nodes: VecCollection, + edges: VecCollection, times: VecCollection, ) -> VecCollection where - G: Scope, - T: Timestamp+std::hash::Hash, + G: Scope, + T: Timestamp+std::hash::Hash+columnar::Columnar, { // Translate node and edge transitions into a common Location to Location edge with an associated Summary. - let nodes = nodes.map(|(target, source, summary)| (Location::from(target), (Location::from(source), summary))); - let edges = edges.map(|(source, target)| (Location::from(source), (Location::from(target), Default::default()))); + let nodes = nodes.map(|(target, source, summary)| (Location::from(to_target(&target)), (Location::from(to_source(&source)), summary))); + let edges = edges.map(|(source, target)| (Location::from(to_source(&source)), (Location::from(to_target(&target)), Default::default()))); let transitions: VecCollection = nodes.concat(edges); times @@ -150,24 +159,24 @@ where /// Summary paths from locations to operator zero inputs. fn summarize( - nodes: VecCollection, - edges: VecCollection, + nodes: VecCollection, + edges: VecCollection, ) -> VecCollection where - G: Scope, - T: Timestamp, + G: Scope, + T: Timestamp+columnar::Columnar, { // Start from trivial reachability from each input to itself. let zero_inputs = edges .clone() - .map(|(_source, target)| Location::from(target)) + .map(|(_source, target)| Location::from(to_target(&target))) .filter(|location| location.node == 0) .map(|location| (location, (location, Default::default()))); // Retain node connections along "default" timestamp summaries. - let nodes = nodes.map(|(target, source, summary)| (Location::from(source), (Location::from(target), summary))); - let edges = edges.map(|(source, target)| (Location::from(target), (Location::from(source), Default::default()))); + let nodes = nodes.map(|(target, source, summary)| (Location::from(to_source(&source)), (Location::from(to_target(&target)), summary))); + let edges = edges.map(|(source, target)| (Location::from(to_target(&target)), (Location::from(to_source(&source)), Default::default()))); let transitions: VecCollection = nodes.concat(edges); zero_inputs @@ -195,23 +204,23 @@ where /// Identifies cycles along paths that do not increment timestamps. fn find_cycles( - nodes: VecCollection, - edges: VecCollection, + nodes: VecCollection, + edges: VecCollection, ) -> VecCollection where - G: Scope, - T: Timestamp, + G: Scope, + T: Timestamp+columnar::Columnar, { // Retain node connections along "default" timestamp summaries. let nodes = nodes.flat_map(|(target, source, summary)| { if summary == Default::default() { - Some((Location::from(target), Location::from(source))) + Some((Location::from(to_target(&target)), Location::from(to_source(&source)))) } else { None } }); - let edges = edges.map(|(source, target)| (Location::from(source), Location::from(target))); + let edges = edges.map(|(source, target)| (Location::from(to_source(&source)), Location::from(to_target(&target)))); let transitions: VecCollection = nodes.concat(edges); // Repeatedly restrict to locations with an incoming path. diff --git a/differential-dataflow/examples/spines.rs b/differential-dataflow/examples/spines.rs index a72a6f34b..fd85a642d 100644 --- a/differential-dataflow/examples/spines.rs +++ b/differential-dataflow/examples/spines.rs @@ -33,9 +33,9 @@ fn main() { match mode.as_str() { "new" => { - use differential_dataflow::trace::implementations::ord_neu::{ColKeyBatcher, ColKeyBuilder, ColKeySpine}; - let data = data.arrange::, ColKeyBuilder<_,_,_>, ColKeySpine<_,_,_>>(); - let keys = keys.arrange::, ColKeyBuilder<_,_,_>, ColKeySpine<_,_,_>>(); + use differential_dataflow::trace::implementations::ord_neu::{OrdKeyBatcher, RcOrdKeyBuilder, OrdKeySpine}; + let data = data.arrange::, RcOrdKeyBuilder<_,_,_>, OrdKeySpine<_,_,_>>(); + let keys = keys.arrange::, RcOrdKeyBuilder<_,_,_>, OrdKeySpine<_,_,_>>(); keys.join_core(data, |_k, &(), &()| Option::<()>::None) .probe_with(&mut probe); }, diff --git a/differential-dataflow/examples/stackoverflow.rs b/differential-dataflow/examples/stackoverflow.rs index 3fe521ee2..93f53b1df 100644 --- a/differential-dataflow/examples/stackoverflow.rs +++ b/differential-dataflow/examples/stackoverflow.rs @@ -107,7 +107,7 @@ fn main() { // returns pairs (n, s) indicating node n can be reached from a root in s steps. fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); diff --git a/differential-dataflow/src/algorithms/graphs/bfs.rs b/differential-dataflow/src/algorithms/graphs/bfs.rs index 1513622bd..f41cd4244 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)); diff --git a/differential-dataflow/src/algorithms/graphs/bijkstra.rs b/differential-dataflow/src/algorithms/graphs/bijkstra.rs index 9c61716ef..f612c0230 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 diff --git a/differential-dataflow/src/algorithms/graphs/propagate.rs b/differential-dataflow/src/algorithms/graphs/propagate.rs index 3d5d8c65c..3d566b3cf 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 diff --git a/differential-dataflow/src/algorithms/graphs/scc.rs b/differential-dataflow/src/algorithms/graphs/scc.rs index f7bca525f..13c00d96d 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, diff --git a/differential-dataflow/src/algorithms/graphs/sequential.rs b/differential-dataflow/src/algorithms/graphs/sequential.rs index 354ad32c2..afa066b08 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 diff --git a/differential-dataflow/src/algorithms/identifiers.rs b/differential-dataflow/src/algorithms/identifiers.rs index 267e4c453..7626396c5 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, { diff --git a/differential-dataflow/src/algorithms/prefix_sum.rs b/differential-dataflow/src/algorithms/prefix_sum.rs index f76f59d9c..42e31bb66 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, @@ -79,7 +79,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, diff --git a/differential-dataflow/src/collection.rs b/differential-dataflow/src/collection.rs index c13227f2c..a703df1c3 100644 --- a/differential-dataflow/src/collection.rs +++ b/differential-dataflow/src/collection.rs @@ -509,7 +509,7 @@ pub mod vec { /// ``` pub fn join_function(self, mut logic: L) -> Collection>::Output> where - G::Timestamp: Lattice, + G::Timestamp: Lattice+columnar::Columnar, D2: Clone+'static, R2: Semigroup+Multiply, I: IntoIterator, @@ -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+columnar::Columnar, { 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+columnar::Columnar, { 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,13 +741,13 @@ pub mod vec { /// }); /// }); /// ``` - pub fn reduce(self, logic: L) -> Collection + pub fn reduce(self, logic: L) -> Collection where L: FnMut(&K, &[(&V, R)], &mut Vec<(V2, R2)>)+'static { self.reduce_named("Reduce", logic) } /// As `reduce` with the ability to name the operator. - pub fn reduce_named(self, name: &str, logic: L) -> Collection + pub fn reduce_named(self, name: &str, logic: L) -> Collection where L: FnMut(&K, &[(&V, R)], &mut Vec<(V2, R2)>)+'static { use crate::trace::implementations::{ValBuilder, ValSpine}; @@ -782,7 +782,7 @@ 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, { @@ -801,7 +801,7 @@ 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, { @@ -812,7 +812,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, K: crate::ExchangeData+Hashable, R1: crate::ExchangeData+Semigroup { @@ -840,7 +840,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,12 +862,12 @@ pub mod vec { /// .threshold(|_,c| c % 2); /// }); /// ``` - pub fn thresholdR2+'static>(self, thresh: F) -> Collection { + pub fn thresholdR2+'static>(self, thresh: F) -> Collection { self.threshold_named("Threshold", thresh) } /// A `threshold` with the ability to name the operator. - pub fn threshold_namedR2+'static>(self, name: &str, mut thresh: F) -> Collection { + pub fn threshold_namedR2+'static>(self, name: &str, mut thresh: F) -> Collection { use crate::trace::implementations::{KeyBuilder, KeySpine}; self.arrange_by_self_named(&format!("Arrange: {}", name)) @@ -879,7 +879,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, K: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup { @@ -905,7 +905,7 @@ 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)))) @@ -916,7 +916,7 @@ pub mod vec { /// Methods which require data be arrangeable. impl Collection where - G: Scope, + G: Scope, D: crate::ExchangeData+Hashable, R: crate::ExchangeData+Semigroup, { @@ -953,7 +953,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(&Tr::Key, &Tr::Val) -> D + 'static, { use crate::operators::arrange::arrangement::Arrange; self.map(|k| (k, ())) @@ -1011,7 +1011,7 @@ pub mod vec { impl Arrange> for Collection where - G: Scope, + G: Scope, K: crate::ExchangeData + Hashable, V: crate::ExchangeData, R: crate::ExchangeData + Semigroup, @@ -1029,7 +1029,7 @@ pub mod vec { impl Arrange> for Collection where - G: Scope, + G: Scope, { fn arrange_named(self, name: &str) -> Arranged> where @@ -1045,7 +1045,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, { /// Arranges a collection of `(Key, Val)` records by `Key`. /// @@ -1064,7 +1064,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, { /// Arranges a collection of `Key` records by `Key`. /// @@ -1084,7 +1084,7 @@ pub mod vec { impl Collection where - G: Scope, + G: Scope, K: crate::ExchangeData+Hashable, V: crate::ExchangeData, R: crate::ExchangeData+Semigroup, @@ -1230,10 +1230,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(&K,&V,&Tr2::Val)->I+'static, { self.arrange_by_key() .join_core(stream2, result) diff --git a/differential-dataflow/src/consolidation.rs b/differential-dataflow/src/consolidation.rs index 0d5271b9e..881fc1335 100644 --- a/differential-dataflow/src/consolidation.rs +++ b/differential-dataflow/src/consolidation.rs @@ -312,8 +312,8 @@ pub trait ConsolidateLayout: DrainContainer { impl ConsolidateLayout for Vec<(D, T, R)> where - D: Ord + Clone + 'static, - T: Ord + Clone + 'static, + D: Data, + T: Data, R: Semigroup + Clone + 'static, { type Key<'a> = (D, T) where Self: 'a; diff --git a/differential-dataflow/src/containers.rs b/differential-dataflow/src/containers.rs deleted file mode 100644 index 4d3f684e5..000000000 --- a/differential-dataflow/src/containers.rs +++ /dev/null @@ -1,302 +0,0 @@ -//! A columnar container based on the columnation library. - -use std::iter::FromIterator; - -pub use columnation::*; -use timely::container::PushInto; - -/// An append-only vector that store records as columns. -/// -/// This container maintains elements that might conventionally own -/// memory allocations, but instead the pointers to those allocations -/// reference larger regions of memory shared with multiple instances -/// of the type. Elements can be retrieved as references, and care is -/// taken when this type is dropped to ensure that the correct memory -/// is returned (rather than the incorrect memory, from running the -/// elements `Drop` implementations). -pub struct TimelyStack { - local: Vec, - inner: T::InnerRegion, -} - -impl TimelyStack { - /// Construct a [TimelyStack], reserving space for `capacity` elements - /// - /// Note that the associated region is not initialized to a specific capacity - /// because we can't generally know how much space would be required. - pub fn with_capacity(capacity: usize) -> Self { - Self { - local: Vec::with_capacity(capacity), - inner: T::InnerRegion::default(), - } - } - - /// Ensures `Self` can absorb `items` without further allocations. - /// - /// The argument `items` may be cloned and iterated multiple times. - /// Please be careful if it contains side effects. - #[inline(always)] - pub fn reserve_items<'a, I>(&mut self, items: I) - where - I: Iterator + Clone, - T: 'a, - { - self.local.reserve(items.clone().count()); - self.inner.reserve_items(items); - } - - /// Ensures `Self` can absorb `items` without further allocations. - /// - /// The argument `items` may be cloned and iterated multiple times. - /// Please be careful if it contains side effects. - #[inline(always)] - pub fn reserve_regions<'a, I>(&mut self, regions: I) - where - Self: 'a, - I: Iterator + Clone, - { - self.local.reserve(regions.clone().map(|cs| cs.local.len()).sum()); - self.inner.reserve_regions(regions.map(|cs| &cs.inner)); - } - - - - /// Copies an element in to the region. - /// - /// The element can be read by indexing - pub fn copy(&mut self, item: &T) { - // TODO: Some types `T` should just be cloned. - // E.g. types that are `Copy` or vecs of ZSTs. - unsafe { - self.local.push(self.inner.copy(item)); - } - } - /// Empties the collection. - pub fn clear(&mut self) { - unsafe { - // Unsafety justified in that setting the length to zero exposes - // no invalid data. - self.local.set_len(0); - self.inner.clear(); - } - } - /// Retain elements that pass a predicate, from a specified offset. - /// - /// This method may or may not reclaim memory in the inner region. - pub fn retain_from bool>(&mut self, index: usize, mut predicate: P) { - let mut write_position = index; - for position in index..self.local.len() { - if predicate(&self[position]) { - // TODO: compact the inner region and update pointers. - self.local.swap(position, write_position); - write_position += 1; - } - } - unsafe { - // Unsafety justified in that `write_position` is no greater than - // `self.local.len()` and so this exposes no invalid data. - self.local.set_len(write_position); - } - } - - /// Unsafe access to `local` data. The slices stor data that is backed by a region - /// allocation. Therefore, it is undefined behavior to mutate elements of the `local` slice. - /// - /// # Safety - /// Elements within `local` can be reordered, but not mutated, removed and/or dropped. - pub unsafe fn local(&mut self) -> &mut [T] { - &mut self.local[..] - } - - /// Estimate the memory capacity in bytes. - #[inline] - pub fn heap_size(&self, mut callback: impl FnMut(usize, usize)) { - let size_of = std::mem::size_of::(); - callback(self.local.len() * size_of, self.local.capacity() * size_of); - self.inner.heap_size(callback); - } - - /// Estimate the consumed memory capacity in bytes, summing both used and total capacity. - #[inline] - pub fn summed_heap_size(&self) -> (usize, usize) { - let (mut length, mut capacity) = (0, 0); - self.heap_size(|len, cap| { - length += len; - capacity += cap - }); - (length, capacity) - } - - /// The length in items. - #[inline] - pub fn len(&self) -> usize { - self.local.len() - } - - /// Returns `true` if the stack is empty. - pub fn is_empty(&self) -> bool { - self.local.is_empty() - } - - /// The capacity of the local vector. - #[inline] - pub fn capacity(&self) -> usize { - self.local.capacity() - } - - /// Reserve space for `additional` elements. - #[inline] - pub fn reserve(&mut self, additional: usize) { - self.local.reserve(additional) - } -} - -impl TimelyStack<(A, B)> { - /// Copies a destructured tuple `(A, B)` into this column stack. - /// - /// This serves situations where a tuple should be constructed from its constituents but - /// not all elements are available as owned data. - /// - /// The element can be read by indexing - pub fn copy_destructured(&mut self, t1: &A, t2: &B) { - unsafe { - self.local.push(self.inner.copy_destructured(t1, t2)); - } - } -} - -impl TimelyStack<(A, B, C)> { - /// Copies a destructured tuple `(A, B, C)` into this column stack. - /// - /// This serves situations where a tuple should be constructed from its constituents but - /// not all elements are available as owned data. - /// - /// The element can be read by indexing - pub fn copy_destructured(&mut self, r0: &A, r1: &B, r2: &C) { - unsafe { - self.local.push(self.inner.copy_destructured(r0, r1, r2)); - } - } -} - -impl std::ops::Deref for TimelyStack { - type Target = [T]; - #[inline(always)] - fn deref(&self) -> &Self::Target { - &self.local[..] - } -} - -impl Drop for TimelyStack { - fn drop(&mut self) { - self.clear(); - } -} - -impl Default for TimelyStack { - fn default() -> Self { - Self { - local: Vec::new(), - inner: T::InnerRegion::default(), - } - } -} - -impl<'a, A: 'a + Columnation> FromIterator<&'a A> for TimelyStack { - fn from_iter>(iter: T) -> Self { - let iter = iter.into_iter(); - let mut c = TimelyStack::::with_capacity(iter.size_hint().0); - for element in iter { - c.copy(element); - } - - c - } -} - -impl PartialEq for TimelyStack { - fn eq(&self, other: &Self) -> bool { - PartialEq::eq(&self[..], &other[..]) - } -} - -impl Eq for TimelyStack {} - -impl std::fmt::Debug for TimelyStack { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - self[..].fmt(f) - } -} - -impl Clone for TimelyStack { - fn clone(&self) -> Self { - let mut new: Self = Default::default(); - for item in &self[..] { - new.copy(item); - } - new - } - - fn clone_from(&mut self, source: &Self) { - self.clear(); - for item in &source[..] { - self.copy(item); - } - } -} - -impl PushInto for TimelyStack { - #[inline] - fn push_into(&mut self, item: T) { - self.copy(&item); - } -} - -impl PushInto<&T> for TimelyStack { - #[inline] - fn push_into(&mut self, item: &T) { - self.copy(item); - } -} - - -impl PushInto<&&T> for TimelyStack { - #[inline] - fn push_into(&mut self, item: &&T) { - self.copy(*item); - } -} - -mod container { - use columnation::Columnation; - - use crate::containers::TimelyStack; - - impl timely::container::Accountable for TimelyStack { - #[inline] fn record_count(&self) -> i64 { i64::try_from(self.local.len()).unwrap() } - #[inline] fn is_empty(&self) -> bool { self.local.is_empty() } - } - impl timely::container::DrainContainer for TimelyStack { - type Item<'a> = &'a T where Self: 'a; - type DrainIter<'a> = std::slice::Iter<'a, T> where Self: 'a; - #[inline] fn drain(&mut self) -> Self::DrainIter<'_> { - (*self).iter() - } - } - - impl timely::container::SizableContainer for TimelyStack { - fn at_capacity(&self) -> bool { - self.len() == self.capacity() - } - fn ensure_capacity(&mut self, stash: &mut Option) { - if self.capacity() == 0 { - *self = stash.take().unwrap_or_default(); - self.clear(); - } - let preferred = timely::container::buffer::default_capacity::(); - if self.capacity() < preferred { - self.reserve(preferred - self.capacity()); - } - } - } -} diff --git a/differential-dataflow/src/difference.rs b/differential-dataflow/src/difference.rs index 401bf565f..349dff8ed 100644 --- a/differential-dataflow/src/difference.rs +++ b/differential-dataflow/src/difference.rs @@ -181,10 +181,6 @@ mod present { impl super::Semigroup for Present { fn plus_equals(&mut self, _rhs: &Self) { } } - - impl columnation::Columnation for Present { - type InnerRegion = columnation::CopyRegion; - } } // Pair implementations. diff --git a/differential-dataflow/src/dynamic/pointstamp.rs b/differential-dataflow/src/dynamic/pointstamp.rs index 095acc2b7..f540c0551 100644 --- a/differential-dataflow/src/dynamic/pointstamp.rs +++ b/differential-dataflow/src/dynamic/pointstamp.rs @@ -254,47 +254,3 @@ impl Lattice for PointStamp { } } -mod columnation { - use columnation::{Columnation, Region}; - use smallvec::SmallVec; - use crate::dynamic::pointstamp::PointStamp; - - impl Columnation for PointStamp { - type InnerRegion = PointStampStack; - } - - /// Stack for PointStamp. Part of Columnation implementation. - pub struct PointStampStack>( as Columnation>::InnerRegion); - - impl> Default for PointStampStack { - #[inline] - fn default() -> Self { - Self(Default::default()) - } - } - - impl> Region for PointStampStack { - type Item = PointStamp; - - #[inline] - unsafe fn copy(&mut self, item: &Self::Item) -> Self::Item { - Self::Item { vector: self.0.copy(&item.vector) } - } - - fn clear(&mut self) { - self.0.clear(); - } - - fn reserve_items<'a, I>(&mut self, items: I) where Self: 'a, I: Iterator + Clone { - self.0.reserve_items(items.map(|x| &x.vector)); - } - - fn reserve_regions<'a, I>(&mut self, regions: I) where Self: 'a, I: Iterator + Clone { - self.0.reserve_regions(regions.map(|r| &r.0)); - } - - fn heap_size(&self, callback: impl FnMut(usize, usize)) { - self.0.heap_size(callback); - } - } -} diff --git a/differential-dataflow/src/input.rs b/differential-dataflow/src/input.rs index 3fd612107..07e7f1f52 100644 --- a/differential-dataflow/src/input.rs +++ b/differential-dataflow/src/input.rs @@ -98,7 +98,7 @@ pub trait Input : TimelyInput { } use crate::lattice::Lattice; -impl Input for G where ::Timestamp: Lattice { +impl Input for G where ::Timestamp: Lattice + columnar::Columnar { fn new_collection(&mut self) -> (InputSession<::Timestamp, D, R>, VecCollection) where D: Data, R: Semigroup+'static, diff --git a/differential-dataflow/src/lib.rs b/differential-dataflow/src/lib.rs index 615aa5ded..34e05e439 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 : Ord + Debug + Clone + columnar::Columnar + 'static { } +impl Data for T { } /// Data types exchangeable in differential dataflow. pub trait ExchangeData : timely::ExchangeData + Data { } @@ -104,7 +104,6 @@ pub mod collection; pub mod logging; pub mod consolidation; pub mod capture; -pub mod containers; /// Configuration options for differential dataflow. #[derive(Default)] 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..9d31af26d 100644 --- a/differential-dataflow/src/operators/arrange/arrangement.rs +++ b/differential-dataflow/src/operators/arrange/arrangement.rs @@ -44,7 +44,7 @@ use super::TraceAgent; /// computation, memory) required to produce and maintain an indexed representation of a collection. pub struct Arranged where - G: Scope, + G: Scope, Tr: TraceReader+Clone, { /// A stream containing arranged updates. @@ -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+columnar::Columnar, { 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+columnar::Columnar+'static, + F: FnMut(&Tr::Key, &Tr::Val, &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(&Tr::Key, &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| [(key.clone(), val.clone())]) } /// 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(&Tr::Key, &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(&Tr::Key, &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(), time.clone(), diff.clone())); }); } 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(&T1::Key,&T1::Val,&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: &T1::Key, v1: &T1::Val, v2: &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(&T1::Key, &[(&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(&T1::Key, &[(&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) @@ -311,7 +309,7 @@ where /// A type that can be arranged as if a collection of updates. pub trait Arrange : Sized where - G: Scope, + G: Scope, { /// Arranges updates into a shared trace. fn arrange(self) -> Arranged> @@ -339,7 +337,7 @@ where /// be consistently by key (though this is the most common). pub fn arrange_core(stream: Stream, pact: P, name: &str) -> Arranged> where - G: Scope, + G: Scope, P: ParallelizationContract, Ba: Batcher + 'static, Bu: Builder, diff --git a/differential-dataflow/src/operators/arrange/upsert.rs b/differential-dataflow/src/operators/arrange/upsert.rs index e9dbe9cdb..4b9e9cc80 100644 --- a/differential-dataflow/src/operators/arrange/upsert.rs +++ b/differential-dataflow/src/operators/arrange/upsert.rs @@ -113,8 +113,6 @@ use crate::trace::{Builder, Description}; use crate::trace::{self, Trace, TraceReader, Cursor}; use crate::{ExchangeData, Hashable}; -use crate::trace::implementations::containers::BatchContainer; - use super::TraceAgent; /// Arrange data from a stream of keyed upserts. @@ -128,18 +126,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 +146,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 +172,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,25 +232,22 @@ 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); 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)); - if trace_cursor.get_key(&trace_storage).map(|k| k.eq(&key_con.index(0))).unwrap_or(false) { + trace_cursor.seek_key(&trace_storage, &key); + if trace_cursor.get_key(&trace_storage).map(|k| k.eq(&key)).unwrap_or(false) { // 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 += diff.clone()); assert!(count == 0 || count == 1); if count == 1 { assert!(prev_value.is_none()); - prev_value = Some(Tr::owned_val(val)); + prev_value = Some(val.clone()); } trace_cursor.step_val(&trace_storage); } diff --git a/differential-dataflow/src/operators/count.rs b/differential-dataflow/src/operators/count.rs index 713c73c86..197af83a3 100644 --- a/differential-dataflow/src/operators/count.rs +++ b/differential-dataflow/src/operators/count.rs @@ -44,7 +44,7 @@ pub trait CountTotal, K: ExchangeDat impl CountTotal for VecCollection where - G: Scope, + G: Scope, { fn count_total_core + 'static>(self) -> VecCollection { self.arrange_by_self_named("Arrange: CountTotal") @@ -55,11 +55,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 + for<'a> Semigroup<&'a T1::Diff>, >+Clone+'static, K: ExchangeData, { @@ -109,7 +109,7 @@ where if trace_cursor.get_key(&trace_storage) == Some(key) { trace_cursor.map_times(&trace_storage, |_, 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.clone()); } }); } @@ -117,14 +117,14 @@ where 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(((key.clone(), count.clone()), time.clone(), 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.clone()); } 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(((key.clone(), count.clone()), time.clone(), R2::from(1i8))); } } }); diff --git a/differential-dataflow/src/operators/iterate.rs b/differential-dataflow/src/operators/iterate.rs index e5486bb6e..41f291310 100644 --- a/differential-dataflow/src/operators/iterate.rs +++ b/differential-dataflow/src/operators/iterate.rs @@ -46,7 +46,7 @@ use crate::difference::{Semigroup, Abelian}; use crate::lattice::Lattice; /// An extension trait for the `iterate` method. -pub trait Iterate, D: Data, R: Semigroup> { +pub trait Iterate, D: Data, R: Semigroup> { /// Iteratively apply `logic` to the source collection until convergence. /// /// Importantly, this method does not automatically consolidate results. @@ -78,7 +78,7 @@ pub trait Iterate, D: Data, R: Semigroup> { for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>; } -impl, D: Ord+Data+Debug, R: Abelian+'static> Iterate for VecCollection { +impl, D: Ord+Data+Debug, R: Abelian+'static> Iterate for VecCollection { fn iterate(self, logic: F) -> VecCollection where for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, @@ -98,7 +98,7 @@ impl, D: Ord+Data+Debug, R: Abelian+'static> Iterat } } -impl, D: Ord+Data+Debug, R: Semigroup+'static> Iterate for G { +impl, D: Ord+Data+Debug, R: Semigroup+'static> Iterate for G { fn iterate(mut self, logic: F) -> VecCollection where for<'a> F: FnOnce(Iterative<'a, G, u64>, VecCollection, D, R>)->VecCollection, D, R>, @@ -189,7 +189,7 @@ impl, D: Ord+Data+Debug, R: Semigroup+'static> Iter /// large, and the edits to perform are relatively smaller. pub struct Variable where - G: Scope, + G: Scope, C: Container, { feedback: Handle, @@ -202,7 +202,7 @@ pub type VecVariable = Variable::Timestam impl Variable where - G: Scope, + G: Scope, C: crate::collection::containers::ResultsIn<::Summary>, { /// Creates a new initially empty `Variable` and its associated `Collection`. diff --git a/differential-dataflow/src/operators/join.rs b/differential-dataflow/src/operators/join.rs index dae4d31b3..6abc72bc6 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(&T1::Key,&T1::Val,&T2::Val,&G::Timestamp,&T1::Diff,&T2::Diff,&mut JoinSession>)+'static, CB: ContainerBuilder, { // Rename traces for symmetry from here on out. @@ -313,7 +313,7 @@ struct Deferred where T: Timestamp+Lattice+Ord, C1: Cursor, - C2: for<'a> Cursor=C1::Key<'a>, Time=T>, + C2: Cursor, { trace: C1, trace_storage: C1::Storage, @@ -326,7 +326,7 @@ where impl Deferred where C1: Cursor, - C2: for<'a> Cursor=C1::Key<'a>, Time=T>, + C2: Cursor, T: Timestamp+Lattice+Ord, { fn new(trace: C1, trace_storage: C1::Storage, batch: C2, batch_storage: C2::Storage, capability: Capability) -> Self { @@ -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: FnMut(&C1::Key, &C1::Val, &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 = time.clone(); 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| time.clone()); // 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(&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..deb3c901d 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<(&'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(&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), diff1.clone())); 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: &'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(&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<((&'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(&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: &C::Key, logic: L ) -> HistoryReplay<'storage, 'history, C> where - L: Fn(C::TimeGat<'_>)->C::Time, + L: Fn(&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<(&'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) -> &[((&'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..8d1f75b3f 100644 --- a/differential-dataflow/src/operators/reduce.rs +++ b/differential-dataflow/src/operators/reduce.rs @@ -19,8 +19,6 @@ 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::containers::BatchContainer; -use crate::trace::implementations::merge_batcher::container::MergerChunk; use crate::trace::TraceReader; /// A key-wise reduction of values in an input trace. @@ -29,10 +27,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(&T1::Key, &[(&T1::Val, T1::Diff)], &mut Vec<(T2::Val,T2::Diff)>, &mut Vec<(T2::Val, T2::Diff)>)+'static, { let mut result_trace = None; @@ -67,7 +65,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,13 +144,13 @@ 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 = Vec::::new(); + let mut exposed_time = Vec::::new(); // Keep pairs greater or equal to `upper_limit`, and "expose" other pairs. interesting.retain(|(key, time)| { if upper_limit.less_equal(time) { true } else { - exposed_keys.push_own(key); - exposed_time.push_own(time); + exposed_keys.push(key.clone()); + exposed_time.push(time.clone()); false } }); @@ -166,7 +164,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 +209,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(exposed_time[exposed_position].clone()); exposed_position += 1; } @@ -238,7 +236,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((key.clone(), time)); } // Sort each buffer by value and move into the corresponding builder. @@ -248,9 +246,9 @@ 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(((key.clone(), val), time, diff)); builders[index].push(&mut buffer); - buffer.clear(); + buffer = Default::default(); } } } @@ -345,31 +343,30 @@ fn sort_dedup(list: &mut Vec) { list.dedup(); } -trait PerKeyCompute<'a, C1, C2, C3, V> +trait PerKeyCompute<'a, C1, C2, C3> 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, { fn new() -> Self; fn compute( &mut self, - key: C1::Key<'a>, + key: &C1::Key, source_cursor: (&mut C1, &'a C1::Storage), output_cursor: (&mut C2, &'a C2::Storage), batch_cursor: (&mut C3, &'a C3::Storage), times: &mut Vec, logic: &mut L, upper_limit: &Antichain, - outputs: &mut [(C2::Time, Vec<(V, C2::Time, C2::Diff)>)], + outputs: &mut [(C2::Time, Vec<(C2::Val, C2::Time, C2::Diff)>)], new_interesting: &mut Vec) -> (usize, usize) where L: FnMut( - C1::Key<'a>, - &[(C1::Val<'a>, C1::Diff)], - &mut Vec<(V, C2::Diff)>, - &mut Vec<(V, C2::Diff)>, + &C1::Key, + &[(&C1::Val, C1::Diff)], + &mut Vec<(C2::Val, C2::Diff)>, + &mut Vec<(C2::Val, C2::Diff)>, ); } @@ -388,32 +385,30 @@ mod history_replay { /// The `HistoryReplayer` is a compute strategy based on moving through existing inputs, interesting times, etc in /// time order, maintaining consolidated representations of updates with respect to future interesting times. - pub struct HistoryReplayer<'a, C1, C2, C3, V> + pub struct HistoryReplayer<'a, C1, C2, C3> 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>, - V: Clone + Ord, + C2: Cursor, + C3: Cursor, { input_history: ValueHistory<'a, C1>, output_history: ValueHistory<'a, C2>, batch_history: ValueHistory<'a, C3>, - input_buffer: Vec<(C1::Val<'a>, C1::Diff)>, - output_buffer: Vec<(V, C2::Diff)>, - update_buffer: Vec<(V, C2::Diff)>, - output_produced: Vec<((V, C2::Time), C2::Diff)>, + input_buffer: Vec<(&'a C1::Val, C1::Diff)>, + output_buffer: Vec<(C2::Val, C2::Diff)>, + update_buffer: Vec<(C2::Val, C2::Diff)>, + output_produced: Vec<((C2::Val, C2::Time), C2::Diff)>, synth_times: Vec, meets: Vec, times_current: Vec, temporary: Vec, } - impl<'a, C1, C2, C3, V> PerKeyCompute<'a, C1, C2, C3, V> for HistoryReplayer<'a, C1, C2, C3, V> + impl<'a, C1, C2, C3> PerKeyCompute<'a, C1, C2, C3> for HistoryReplayer<'a, C1, C2, C3> 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, { fn new() -> Self { HistoryReplayer { @@ -433,21 +428,21 @@ mod history_replay { #[inline(never)] fn compute( &mut self, - key: C1::Key<'a>, + key: &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), times: &mut Vec, logic: &mut L, upper_limit: &Antichain, - outputs: &mut [(C2::Time, Vec<(V, C2::Time, C2::Diff)>)], + outputs: &mut [(C2::Time, Vec<(C2::Val, C2::Time, C2::Diff)>)], new_interesting: &mut Vec) -> (usize, usize) where L: FnMut( - C1::Key<'a>, - &[(C1::Val<'a>, C1::Diff)], - &mut Vec<(V, C2::Diff)>, - &mut Vec<(V, C2::Diff)>, + &C1::Key, + &[(&C1::Val, C1::Diff)], + &mut Vec<(C2::Val, C2::Diff)>, + &mut Vec<(C2::Val, C2::Diff)>, ) { @@ -460,7 +455,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| time.clone()); // 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 @@ -477,18 +472,6 @@ mod history_replay { let mut meet = None; update_meet(&mut meet, self.meets.get(0)); update_meet(&mut meet, batch_replay.meet()); - // if let Some(time) = self.meets.get(0) { - // meet = match meet { - // None => Some(self.meets[0].clone()), - // Some(x) => Some(x.meet(&self.meets[0])), - // }; - // } - // if let Some(time) = batch_replay.meet() { - // meet = match meet { - // None => Some(time.clone()), - // Some(x) => Some(x.meet(&time)), - // }; - // } // Having determined the meet, we can load the input and output histories, where we // advance all times by joining them with `meet`. The resulting times are more compact @@ -497,23 +480,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 = time.clone(); 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| time.clone()) }; 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 = time.clone(); 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| time.clone()) }; self.synth_times.clear(); @@ -618,7 +601,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((value.clone(), diff.clone())); } else { self.temporary.push(next_time.join(time)); @@ -641,24 +624,6 @@ mod history_replay { self.output_buffer.clear(); } - // output_replay.advance_buffer_by(&meet); - // for &((ref value, ref time), diff) in output_replay.buffer().iter() { - // if time.less_equal(&next_time) { - // self.output_buffer.push(((*value).clone(), -diff)); - // } - // else { - // self.temporary.push(next_time.join(time)); - // } - // } - // for &((ref value, ref time), diff) in self.output_produced.iter() { - // if time.less_equal(&next_time) { - // self.output_buffer.push(((*value).clone(), -diff)); - // } - // else { - // self.temporary.push(next_time.join(&time)); - // } - // } - // Having subtracted output updates from user output, consolidate the results to determine // if there is anything worth reporting. Note: this also orders the results by value, so // that could make the above merging plan even easier. @@ -752,12 +717,7 @@ mod history_replay { update_meet(&mut meet, input_replay.meet()); update_meet(&mut meet, output_replay.meet()); for time in self.synth_times.iter() { update_meet(&mut meet, Some(time)); } - // if let Some(time) = batch_replay.meet() { meet = meet.meet(time); } - // if let Some(time) = input_replay.meet() { meet = meet.meet(time); } - // if let Some(time) = output_replay.meet() { meet = meet.meet(time); } - // for time in self.synth_times.iter() { meet = meet.meet(time); } update_meet(&mut meet, meets_slice.first()); - // if let Some(time) = meets_slice.first() { meet = meet.meet(time); } // Update `times_current` by the frontier. if let Some(meet) = meet.as_ref() { diff --git a/differential-dataflow/src/operators/threshold.rs b/differential-dataflow/src/operators/threshold.rs index 91bbd16fe..81b230fbc 100644 --- a/differential-dataflow/src/operators/threshold.rs +++ b/differential-dataflow/src/operators/threshold.rs @@ -17,7 +17,7 @@ use crate::operators::arrange::Arranged; use crate::trace::{BatchReader, Cursor, TraceReader}; /// Extension trait for the `distinct` differential dataflow method. -pub trait ThresholdTotal, K: ExchangeData, R: ExchangeData+Semigroup> : Sized { +pub trait ThresholdTotal, K: ExchangeData, R: ExchangeData+Semigroup> : Sized { /// Reduces the collection to one occurrence of each distinct element. fn threshold_semigroup(self, thresh: F) -> VecCollection where @@ -86,7 +86,7 @@ pub trait ThresholdTotal, K: Exchang impl ThresholdTotal for VecCollection where - G: Scope, + G: Scope, { fn threshold_semigroup(self, thresh: F) -> VecCollection where @@ -101,18 +101,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 + for<'a> Semigroup<&'a T1::Diff>, >+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: FnMut(&T1::Key,&T1::Diff,Option<&T1::Diff>)->Option+'static, { let mut trace = self.trace.clone(); @@ -160,7 +160,7 @@ where if trace_cursor.get_key(&trace_storage) == Some(key) { trace_cursor.map_times(&trace_storage, |_, 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.clone()); } }); } @@ -175,7 +175,7 @@ where temp.plus_equals(&diff); thresh(key, &temp, Some(old)) }, - None => { thresh(key, &T1::owned_diff(diff), None) }, + None => { thresh(key, &diff.clone(), None) }, }; // Either add or assign `diff` to `count`. @@ -183,12 +183,12 @@ where count.plus_equals(&diff); } else { - count = Some(T1::owned_diff(diff)); + count = Some(diff.clone()); } if let Some(difference) = difference { if !difference.is_zero() { - session.give((key.clone(), T1::owned_time(time), difference)); + session.give((key.clone(), time.clone(), difference)); } } }); diff --git a/differential-dataflow/src/trace/cursor/cursor_list.rs b/differential-dataflow/src/trace/cursor/cursor_list.rs index 6d2d5b57b..45a34e285 100644 --- a/differential-dataflow/src/trace/cursor/cursor_list.rs +++ b/differential-dataflow/src/trace/cursor/cursor_list.rs @@ -94,13 +94,13 @@ 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 @@ -111,29 +111,29 @@ impl Cursor for CursorList { // accessors #[inline] - fn key<'a>(&self, storage: &'a Vec) -> Self::Key<'a> { + fn key<'a>(&self, storage: &'a Vec) -> &'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) -> &'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<&'a Self::Key> { 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<&'a Self::Val> { 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(&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)); } @@ -148,7 +148,7 @@ 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: &Self::Key) { for (cursor, storage) in self.cursors.iter_mut().zip(storage) { cursor.seek_key(storage, key); } @@ -164,7 +164,7 @@ 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: &Self::Val) { for (cursor, storage) in self.cursors.iter_mut().zip(storage) { cursor.seek_val(storage, val); } diff --git a/differential-dataflow/src/trace/cursor/mod.rs b/differential-dataflow/src/trace/cursor/mod.rs index d0d973249..c052ada1c 100644 --- a/differential-dataflow/src/trace/cursor/mod.rs +++ b/differential-dataflow/src/trace/cursor/mod.rs @@ -9,10 +9,17 @@ pub mod cursor_list; pub use self::cursor_list::CursorList; -use crate::trace::implementations::LayoutExt; - /// A cursor for navigating ordered `(key, val, time, diff)` updates. -pub trait Cursor : LayoutExt { +pub trait Cursor { + + /// The key type. + type Key: crate::Data; + /// The value type. + type Val: crate::Data; + /// The time type. + type Time: crate::lattice::Lattice + timely::progress::Timestamp + crate::Data; + /// The diff type. + type Diff: crate::difference::Semigroup + crate::Data; /// Storage required by the cursor. type Storage; @@ -27,28 +34,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) -> &'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) -> &'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<&'a Self::Key>; /// 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<&'a Self::Val>; /// 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(&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: &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: &Self::Val); /// Rewinds the cursor to the first key. fn rewind_keys(&mut self, storage: &Self::Storage); @@ -58,8 +65,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(&'a Self::Key) -> K, + IV: for<'a> Fn(&'a Self::Val) -> V, { let mut out = Vec::new(); self.rewind_keys(storage); @@ -68,7 +75,7 @@ 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((ts.clone(), r.clone())); }); out.push(((into_key(key), into_val(val)), kv_out)); self.step_val(storage); diff --git a/differential-dataflow/src/trace/implementations/chunker.rs b/differential-dataflow/src/trace/implementations/chunker.rs index 71d7aeb41..ed8caa005 100644 --- a/differential-dataflow/src/trace/implementations/chunker.rs +++ b/differential-dataflow/src/trace/implementations/chunker.rs @@ -2,11 +2,9 @@ use std::collections::VecDeque; -use columnation::Columnation; use timely::Container; use timely::container::{ContainerBuilder, DrainContainer, PushInto, SizableContainer}; -use crate::containers::TimelyStack; use crate::consolidation::{consolidate_updates, ConsolidateLayout}; use crate::difference::Semigroup; @@ -124,117 +122,6 @@ where } } -/// Chunk a stream of vectors into chains of vectors. -pub struct ColumnationChunker { - pending: Vec, - ready: VecDeque>, - empty: Option>, -} - -impl Default for ColumnationChunker { - fn default() -> Self { - Self { - pending: Vec::default(), - ready: VecDeque::default(), - empty: None, - } - } -} - -impl ColumnationChunker<(D, T, R)> -where - D: Columnation + Ord, - T: Columnation + Ord, - R: Columnation + Semigroup, -{ - const BUFFER_SIZE_BYTES: usize = 64 << 10; - fn chunk_capacity() -> usize { - let size = ::std::mem::size_of::<(D, T, R)>(); - if size == 0 { - Self::BUFFER_SIZE_BYTES - } else if size <= Self::BUFFER_SIZE_BYTES { - Self::BUFFER_SIZE_BYTES / size - } else { - 1 - } - } - - /// Form chunks out of pending data, if needed. This function is meant to be applied to - /// potentially full buffers, and ensures that if the buffer was full when called it is at most - /// half full when the function returns. - /// - /// `form_chunk` does the following: - /// * If pending is full, consolidate. - /// * If after consolidation it's more than half full, peel off chunks, - /// leaving behind any partial chunk in pending. - fn form_chunk(&mut self) { - consolidate_updates(&mut self.pending); - if self.pending.len() >= Self::chunk_capacity() { - while self.pending.len() > Self::chunk_capacity() { - let mut chunk = TimelyStack::with_capacity(Self::chunk_capacity()); - for item in self.pending.drain(..chunk.capacity()) { - chunk.copy(&item); - } - self.ready.push_back(chunk); - } - } - } -} - -impl<'a, D, T, R> PushInto<&'a mut Vec<(D, T, R)>> for ColumnationChunker<(D, T, R)> -where - D: Columnation + Ord + Clone, - T: Columnation + Ord + Clone, - R: Columnation + Semigroup + Clone, -{ - fn push_into(&mut self, container: &'a mut Vec<(D, T, R)>) { - // Ensure `self.pending` has the desired capacity. We should never have a larger capacity - // because we don't write more than capacity elements into the buffer. - if self.pending.capacity() < Self::chunk_capacity() * 2 { - self.pending.reserve(Self::chunk_capacity() * 2 - self.pending.len()); - } - - let mut drain = container.drain(..).peekable(); - while drain.peek().is_some() { - self.pending.extend((&mut drain).take(self.pending.capacity() - self.pending.len())); - if self.pending.len() == self.pending.capacity() { - self.form_chunk(); - } - } - } -} - -impl ContainerBuilder for ColumnationChunker<(D, T, R)> -where - D: Columnation + Ord + Clone + 'static, - T: Columnation + Ord + Clone + 'static, - R: Columnation + Semigroup + Clone + 'static, -{ - type Container = TimelyStack<(D,T,R)>; - - fn extract(&mut self) -> Option<&mut Self::Container> { - if let Some(ready) = self.ready.pop_front() { - self.empty = Some(ready); - self.empty.as_mut() - } else { - None - } - } - - fn finish(&mut self) -> Option<&mut Self::Container> { - consolidate_updates(&mut self.pending); - while !self.pending.is_empty() { - let mut chunk = TimelyStack::with_capacity(Self::chunk_capacity()); - for item in self.pending.drain(..std::cmp::min(self.pending.len(), chunk.capacity())) { - chunk.copy(&item); - } - self.ready.push_back(chunk); - } - self.empty = self.ready.pop_front(); - self.empty.as_mut() - } -} - /// Chunk a stream of containers into chains of vectors. pub struct ContainerChunker { pending: Output, diff --git a/differential-dataflow/src/trace/implementations/merge_batcher.rs b/differential-dataflow/src/trace/implementations/merge_batcher.rs index 5c932af9f..b8c84dd70 100644 --- a/differential-dataflow/src/trace/implementations/merge_batcher.rs +++ b/differential-dataflow/src/trace/implementations/merge_batcher.rs @@ -219,7 +219,7 @@ pub trait Merger: Default { fn account(chunk: &Self::Chunk) -> (usize, usize, usize, usize); } -pub use container::{VecMerger, ColMerger}; +pub use container::VecMerger; pub mod container { @@ -516,91 +516,4 @@ pub mod container { } } - pub use columnation::ColMerger; - /// Implementations of `ContainerQueue` and `MergerChunk` for `TimelyStack` containers (columnation). - pub mod columnation { - - use timely::progress::{Antichain, frontier::AntichainRef}; - use columnation::Columnation; - - use crate::containers::TimelyStack; - use crate::difference::Semigroup; - - use super::{ContainerQueue, MergerChunk}; - - /// A `Merger` implementation backed by `TimelyStack` containers (columnation). - pub type ColMerger = super::ContainerMerger,TimelyStackQueue<(D, T, R)>>; - - /// TODO - pub struct TimelyStackQueue { - list: TimelyStack, - head: usize, - } - - impl ContainerQueue> for TimelyStackQueue<(D, T, R)> { - fn next_or_alloc(&mut self) -> Result<&(D, T, R), TimelyStack<(D, T, R)>> { - if self.is_empty() { - Err(std::mem::take(&mut self.list)) - } - else { - Ok(self.pop()) - } - } - fn is_empty(&self) -> bool { - self.head == self.list[..].len() - } - fn cmp_heads(&self, other: &Self) -> std::cmp::Ordering { - let (data1, time1, _) = self.peek(); - let (data2, time2, _) = other.peek(); - (data1, time1).cmp(&(data2, time2)) - } - fn from(list: TimelyStack<(D, T, R)>) -> Self { - TimelyStackQueue { list, head: 0 } - } - } - - impl TimelyStackQueue { - fn pop(&mut self) -> &T { - self.head += 1; - &self.list[self.head - 1] - } - - fn peek(&self) -> &T { - &self.list[self.head] - } - } - - impl MergerChunk for TimelyStack<(D, T, R)> { - type TimeOwned = T; - type DiffOwned = R; - - fn time_kept((_, time, _): &Self::Item<'_>, upper: &AntichainRef, frontier: &mut Antichain) -> bool { - if upper.less_equal(time) { - frontier.insert_with(&time, |time| time.clone()); - true - } - else { false } - } - fn push_and_add<'a>(&mut self, item1: Self::Item<'a>, item2: Self::Item<'a>, stash: &mut Self::DiffOwned) { - let (data, time, diff1) = item1; - let (_data, _time, diff2) = item2; - stash.clone_from(diff1); - stash.plus_equals(&diff2); - if !stash.is_zero() { - self.copy_destructured(data, time, stash); - } - } - fn account(&self) -> (usize, usize, usize, usize) { - let (mut size, mut capacity, mut allocations) = (0, 0, 0); - let cb = |siz, cap| { - size += siz; - capacity += cap; - allocations += 1; - }; - self.heap_size(cb); - (self.len(), size, capacity, allocations) - } - #[inline] fn clear(&mut self) { TimelyStack::clear(self) } - } - } } diff --git a/differential-dataflow/src/trace/implementations/mod.rs b/differential-dataflow/src/trace/implementations/mod.rs index d73eb71f4..523189b73 100644 --- a/differential-dataflow/src/trace/implementations/mod.rs +++ b/differential-dataflow/src/trace/implementations/mod.rs @@ -57,33 +57,30 @@ 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::progress::Timestamp; +use timely::container::PushInto; -use crate::containers::TimelyStack; use crate::lattice::Lattice; use crate::difference::Semigroup; /// A type that names constituent update types. pub trait Update { /// Key by which data are grouped. - type Key: Ord + Clone + 'static; + type Key: crate::Data; /// Values associated with the key. - type Val: Ord + Clone + 'static; + type Val: crate::Data; /// Time at which updates occur. - type Time: Ord + Clone + Lattice + timely::progress::Timestamp; + type Time: crate::Data + Lattice + timely::progress::Timestamp; /// Way in which updates occur. - type Diff: Ord + Semigroup + 'static; + type Diff: crate::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: crate::Data, + V: crate::Data, + T: crate::Data+Lattice+timely::progress::Timestamp, + R: crate::Data+Semigroup, { type Key = K; type Val = V; @@ -91,165 +88,6 @@ 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>; -} - -/// A type bearing a layout. -pub trait WithLayout { - /// The layout. - type Layout: Layout; -} - -/// 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 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) } - -} - -// 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>, -{ - type KeyContainer = KC; - type ValContainer = VC; - type TimeContainer = TC; - type DiffContainer = DC; - type OffsetContainer = OC; -} - -/// 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>; -} - -/// A layout that uses vectors -pub struct Vector { - phantom: std::marker::PhantomData, -} - -impl> Layout for Vector { - type KeyContainer = Vec; - type ValContainer = Vec; - type TimeContainer = Vec; - type DiffContainer = Vec; - type OffsetContainer = OffsetList; -} - -/// A layout based on timely stacks -pub struct TStack { - phantom: std::marker::PhantomData, -} - -impl Layout for TStack -where - U: Update< - Key: Columnation, - Val: Columnation, - Time: Columnation, - Diff: Columnation + Ord, - >, -{ - type KeyContainer = TimelyStack; - type ValContainer = TimelyStack; - type TimeContainer = TimelyStack; - type DiffContainer = TimelyStack; - type OffsetContainer = OffsetList; -} - /// 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)] pub struct OffsetList { @@ -377,148 +215,13 @@ 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()) - } - - 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) - } -} - -pub use self::containers::{BatchContainer, SliceContainer}; +pub use self::containers::BatchContainer; /// 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<'_>`. @@ -656,118 +359,4 @@ 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. - /// - /// The length will be one greater than the number of contained slices, - /// starting with zero and ending with `self.inner.len()`. - offsets: Vec, - /// An inner container for sequences of `B` that dereferences to a slice. - inner: Vec, - } - - impl PushInto<&[B]> for SliceContainer { - fn push_into(&mut self, item: &[B]) { - for x in item.iter() { - self.inner.push_into(x); - } - self.offsets.push(self.inner.len()); - } - } - - impl PushInto<&Vec> for SliceContainer { - fn push_into(&mut self, item: &Vec) { - self.push_into(&item[..]); - } - } - - impl BatchContainer for SliceContainer - where - B: Ord + Clone + Sized + 'static, - { - type Owned = Vec; - type ReadItem<'a> = &'a [B]; - - #[inline(always)] fn into_owned<'a>(item: Self::ReadItem<'a>) -> Self::Owned { item.to_vec() } - #[inline(always)] fn clone_onto<'a>(item: Self::ReadItem<'a>, other: &mut Self::Owned) { other.clone_from_slice(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.offsets.clear(); - self.offsets.push(0); - self.inner.clear(); - } - - fn with_capacity(size: usize) -> Self { - let mut offsets = Vec::with_capacity(size + 1); - offsets.push(0); - Self { - offsets, - inner: Vec::with_capacity(size), - } - } - fn merge_capacity(cont1: &Self, cont2: &Self) -> Self { - let mut offsets = Vec::with_capacity(cont1.inner.len() + cont2.inner.len() + 1); - offsets.push(0); - Self { - offsets, - inner: Vec::with_capacity(cont1.inner.len() + cont2.inner.len()), - } - } - fn index(&self, index: usize) -> Self::ReadItem<'_> { - let lower = self.offsets[index]; - let upper = self.offsets[index+1]; - &self.inner[lower .. upper] - } - fn len(&self) -> usize { - self.offsets.len() - 1 - } - } - - /// Default implementation introduces a first offset. - impl Default for SliceContainer { - fn default() -> Self { - Self { - offsets: vec![0], - inner: Default::default(), - } - } - } } diff --git a/differential-dataflow/src/trace/implementations/ord_neu.rs b/differential-dataflow/src/trace/implementations/ord_neu.rs index b6f9a198e..7ead06442 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. @@ -267,52 +243,45 @@ pub mod layers { /// Types related to forming batches with values. 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 crate::trace::implementations::BatchContainer; + use crate::trace::implementations::OffsetList; - use super::{Layout, Vals, Upds, layers::UpdsBuilder}; + 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>, + U::Key: Serialize + for<'a> Deserialize<'a>, + U::Val: Serialize + for<'a> Deserialize<'a>, + U::Time: Serialize + for<'a> Deserialize<'a>, + U::Diff: Serialize + for<'a> Deserialize<'a>, ")] - pub struct OrdValStorage { + pub struct OrdValStorage { /// An ordered list of keys. - pub keys: L::KeyContainer, + pub keys: Vec, /// 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, Vec>, } /// 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>, + U::Key: Serialize + for<'a> Deserialize<'a>, + U::Val: Serialize + for<'a> Deserialize<'a>, + U::Time: Serialize + for<'a> Deserialize<'a>, + U::Diff: 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, @@ -321,18 +290,19 @@ pub mod val_batch { pub updates: usize, } - impl WithLayout for OrdValBatch { - type Layout = L; - } + 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, val_cursor: 0, - phantom: PhantomData, + phantom: std::marker::PhantomData, } } fn len(&self) -> usize { @@ -340,13 +310,13 @@ pub 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 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) } @@ -354,7 +324,7 @@ pub mod val_batch { use timely::progress::Timestamp; Self { storage: OrdValStorage { - keys: L::KeyContainer::with_capacity(0), + keys: Vec::new(), vals: Default::default(), upds: Default::default(), }, @@ -365,24 +335,24 @@ pub mod val_batch { } /// State for an in-progress merge. - pub struct OrdValMerger { + pub struct OrdValMerger { /// 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: OrdValStorage, + result: OrdValStorage, /// description - description: Description>, + description: Description, /// Staging area to consolidate owned times and diffs, before sealing. - staging: UpdsBuilder, + staging: UpdsBuilder, Vec>, } - impl Merger> for OrdValMerger + impl Merger> for OrdValMerger where - OrdValBatch: Batch>, + OrdValBatch: Batch, { - fn new(batch1: &OrdValBatch, batch2: &OrdValBatch, compaction_frontier: AntichainRef>) -> Self { + fn new(batch1: &OrdValBatch, batch2: &OrdValBatch, compaction_frontier: AntichainRef) -> Self { assert!(batch1.upper() == batch2.lower()); use crate::lattice::Lattice; @@ -398,7 +368,7 @@ pub mod val_batch { key_cursor1: 0, key_cursor2: 0, result: OrdValStorage { - keys: L::KeyContainer::merge_capacity(&batch1.keys, &batch2.keys), + keys: Vec::with_capacity(batch1.keys.len() + batch2.keys.len()), vals: Vals::merge_capacity(&batch1.vals, &batch2.vals), upds: Upds::merge_capacity(&batch1.upds, &batch2.upds), }, @@ -406,14 +376,14 @@ 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) { + fn work(&mut self, source1: &OrdValBatch, source2: &OrdValBatch, fuel: &mut isize) { // An (incomplete) indication of the amount of work we've done so far. let starting_updates = self.staging.total(); @@ -444,7 +414,7 @@ pub mod val_batch { } // Helper methods in support of merging batches. - impl OrdValMerger { + impl OrdValMerger { /// Copy the next key in `source`. /// /// The method extracts the key in `source` at `cursor`, and merges it in to `self`. @@ -452,7 +422,7 @@ pub 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: &OrdValStorage, cursor: usize) { + fn copy_key(&mut self, source: &OrdValStorage, cursor: usize) { // Capture the initial number of values to determine if the merge was ultimately non-empty. let init_vals = self.result.vals.vals.len(); let (mut lower, upper) = source.vals.bounds(cursor); @@ -474,7 +444,7 @@ pub 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: &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 => { @@ -505,8 +475,8 @@ pub 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): (&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(); @@ -568,53 +538,53 @@ 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 = time.clone(); new_time.advance_by(self.description.since().borrow()); - self.staging.push(new_time, L::DiffContainer::into_owned(diff)); + self.staging.push(new_time, diff.clone()); } } } /// A cursor for navigating a single layer. - pub struct OrdValCursor { + pub struct OrdValCursor { /// 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: std::marker::PhantomData, } - use crate::trace::implementations::WithLayout; - impl WithLayout for OrdValCursor { - type Layout = L; - } + 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<&'a Self::Key> { storage.storage.keys.get(self.key_cursor) } + fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { 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) -> &'a Self::Key { storage.storage.keys.index(self.key_cursor) } + fn val<'a>(&self, storage: &'a OrdValBatch) -> &'a Self::Val { storage.storage.vals.get_abs(self.val_cursor) } + fn map_times(&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 +593,93 @@ 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: &Self::Key) { + self.key_cursor += storage.storage.keys.advance(self.key_cursor, storage.storage.keys.len(), |x| x.lt(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: &Self::Val) { + self.val_cursor += storage.storage.vals.vals.advance(self.val_cursor, storage.storage.vals.bounds(self.key_cursor).1, |x| x.lt(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 { + 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, Vec>, } - impl Builder for OrdValBuilder - where - L: for<'a> Layout< - KeyContainer: PushInto>, - ValContainer: PushInto>, - >, - CI: for<'a> BuilderInput, Diff=layout::Diff>, - { + impl Builder for OrdValBuilder { - type Input = CI; - type Time = layout::Time; - type Output = OrdValBatch; + 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: Vec::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); + // TODO: could iterate by reference to avoid moving data unnecessarily. + for ((key, val), time, diff) in chunk.drain(..) { // Pre-load the first update. if self.result.keys.is_empty() { - self.result.vals.vals.push_into(val); - self.result.keys.push_into(key); + self.result.vals.vals.push(val); + self.result.keys.push(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() == Some(&key) { // 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) { + if self.result.vals.vals.last() == Some(&val) { 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(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(val); + self.result.keys.push(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,7 +690,29 @@ 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; + 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; + } + } + upds += 1; + prev_keyval = Some((key, val)); + } + } + if prev_keyval.is_some() { + keys += 1; + vals += 1; + } + let mut builder = Self::with_capacity(keys, vals, upds); for mut chunk in chain.drain(..) { builder.push(&mut chunk); @@ -743,49 +726,42 @@ pub mod val_batch { /// Types related to forming batches of keys. 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 crate::trace::implementations::BatchContainer; + use crate::trace::implementations::OffsetList; - use super::{Layout, Upds, layers::UpdsBuilder}; + 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>, + U::Key: Serialize + for<'a> Deserialize<'a>, + U::Time: Serialize + for<'a> Deserialize<'a>, + U::Diff: Serialize + for<'a> Deserialize<'a>, ")] - pub struct OrdKeyStorage { + pub struct OrdKeyStorage { /// An ordered list of keys, corresponding to entries in `keys_offs`. - pub keys: L::KeyContainer, + pub keys: Vec, /// For each key in `keys`, a list of (time, diff) updates. - pub upds: Upds, + pub upds: Upds, Vec>, } /// 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>, + U::Key: Serialize + for<'a> Deserialize<'a>, + U::Val: Serialize + for<'a> Deserialize<'a>, + U::Time: Serialize + for<'a> Deserialize<'a>, + U::Diff: 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, @@ -794,25 +770,26 @@ pub mod key_batch { pub updates: usize, /// Single value to return if asked. - pub value: L::ValContainer, + pub value: Vec, } - 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); - value.push_own(&Default::default()); + pub fn create_value() -> Vec { + let mut value = Vec::with_capacity(1); + value.push(Default::default()); value } } - impl>> WithLayout for OrdKeyBatch { - type Layout = L; - } + impl> BatchReader for OrdKeyBatch { - impl>> BatchReader for OrdKeyBatch { + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; - type Cursor = OrdKeyCursor; + type Cursor = OrdKeyCursor; fn cursor(&self) -> Self::Cursor { OrdKeyCursor { key_cursor: 0, @@ -825,13 +802,13 @@ pub mod key_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 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) } @@ -839,7 +816,7 @@ pub mod key_batch { use timely::progress::Timestamp; Self { storage: OrdKeyStorage { - keys: L::KeyContainer::with_capacity(0), + keys: Vec::new(), upds: Upds::default(), }, description: Description::new(lower, upper, Antichain::from_elem(Self::Time::minimum())), @@ -850,25 +827,25 @@ pub mod key_batch { } /// State for an in-progress merge. - pub struct OrdKeyMerger { + pub struct OrdKeyMerger { /// 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: OrdKeyStorage, + result: OrdKeyStorage, /// description - description: Description>, + description: Description, /// Local stash of updates, to use for consolidation. - staging: UpdsBuilder, + staging: UpdsBuilder, Vec>, } - impl>> Merger> for OrdKeyMerger + impl> Merger> for OrdKeyMerger where - OrdKeyBatch: Batch>, + OrdKeyBatch: Batch, { - fn new(batch1: &OrdKeyBatch, batch2: &OrdKeyBatch, compaction_frontier: AntichainRef>) -> Self { + fn new(batch1: &OrdKeyBatch, batch2: &OrdKeyBatch, compaction_frontier: AntichainRef) -> Self { assert!(batch1.upper() == batch2.lower()); use crate::lattice::Lattice; @@ -884,22 +861,22 @@ pub mod key_batch { key_cursor1: 0, key_cursor2: 0, result: OrdKeyStorage { - keys: L::KeyContainer::merge_capacity(&batch1.keys, &batch2.keys), + keys: Vec::with_capacity(batch1.keys.len() + batch2.keys.len()), 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) { + fn work(&mut self, source1: &OrdKeyBatch, source2: &OrdKeyBatch, fuel: &mut isize) { // An (incomplete) indication of the amount of work we've done so far. let starting_updates = self.staging.total(); @@ -930,7 +907,7 @@ pub mod key_batch { } // Helper methods in support of merging batches. - impl OrdKeyMerger { + impl OrdKeyMerger { /// Copy the next key in `source`. /// /// The method extracts the key in `source` at `cursor`, and merges it in to `self`. @@ -938,7 +915,7 @@ pub mod key_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: &OrdKeyStorage, cursor: usize) { + 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)); @@ -948,7 +925,7 @@ pub mod key_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: &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 => { @@ -974,44 +951,44 @@ 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 = time.clone(); new_time.advance_by(self.description.since().borrow()); - self.staging.push(new_time, L::DiffContainer::into_owned(diff)); + self.staging.push(new_time, diff.clone()); } } } /// A cursor for navigating a single layer. - pub struct OrdKeyCursor { + pub struct OrdKeyCursor { /// Absolute position of the current key. 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, + phantom: std::marker::PhantomData, } - use crate::trace::implementations::WithLayout; - impl> WithLayout for OrdKeyCursor { - type Layout = L; - } + impl> Cursor for OrdKeyCursor { - impl Layout>> Cursor for OrdKeyCursor { + type Key = U::Key; + type Val = U::Val; + type Time = U::Time; + type Diff = U::Diff; - type Storage = OrdKeyBatch; + 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<&'a Self::Key> { storage.storage.keys.get(self.key_cursor) } + fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { 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 Self::Storage) -> &'a Self::Key { storage.storage.keys.index(self.key_cursor) } + fn val<'a>(&self, storage: &'a Self::Storage) -> &'a Self::Val { storage.value.index(0) } + fn map_times(&mut self, storage: &Self::Storage, 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); @@ -1029,8 +1006,8 @@ 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: &Self::Storage, key: &Self::Key) { + self.key_cursor += storage.storage.keys.advance(self.key_cursor, storage.storage.keys.len(), |x| x.lt(key)); if self.key_valid(storage) { self.rewind_vals(storage); } @@ -1038,7 +1015,7 @@ pub mod key_batch { fn step_val(&mut self, _storage: &Self::Storage) { self.val_stepped = true; } - fn seek_val(&mut self, _storage: &Self::Storage, _val: Self::Val<'_>) { } + fn seek_val(&mut self, _storage: &Self::Storage, _val: &Self::Val) { } fn rewind_keys(&mut self, storage: &Self::Storage) { self.key_cursor = 0; if self.key_valid(storage) { @@ -1051,69 +1028,84 @@ pub mod key_batch { } /// A builder for creating layers from unsorted update tuples. - pub struct OrdKeyBuilder { + 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, Vec>, } - impl Builder for OrdKeyBuilder - where - L: for<'a> Layout>>, - L: Layout>, - CI: BuilderInput, Diff=layout::Diff>, - { + impl> Builder for OrdKeyBuilder { - type Input = CI; - type Time = layout::Time; - type Output = OrdKeyBatch; + 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: Vec::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); + // TODO: could iterate by reference to avoid moving data unnecessarily. + for ((key, _val), time, diff) in chunk.drain(..) { if self.result.keys.is_empty() { - self.result.keys.push_into(key); + self.result.keys.push(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() == Some(&key) { 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(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 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; + } + } + upds += 1; + prev_keyval = Some((key, val)); + } + } + if prev_keyval.is_some() { + keys += 1; + vals += 1; + } + 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/rhh.rs b/differential-dataflow/src/trace/implementations/rhh.rs index 71d20b69d..c7197dfaf 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 { } @@ -47,7 +33,7 @@ pub trait HashOrdered: Hashable { } impl<'a, T: std::hash::Hash + HashOrdered> HashOrdered for &'a T { } /// A hash-ordered wrapper that modifies `Ord` and `PartialOrd`. -#[derive(Copy, Clone, Eq, PartialEq, Debug, Default, Serialize, Deserialize)] +#[derive(Copy, Clone, Eq, PartialEq, Debug, Default, Serialize, Deserialize, columnar::Columnar)] pub struct HashWrapper { /// The inner value, freely modifiable. pub inner: T @@ -86,15 +72,13 @@ 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::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)`. /// @@ -111,9 +95,15 @@ mod val_batch { /// 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 + #[serde(bound = " + U::Key: Serialize + for<'a> Deserialize<'a>, + U::Val: Serialize + for<'a> Deserialize<'a>, + U::Time: Serialize + for<'a> Deserialize<'a>, + U::Diff: Serialize + for<'a> Deserialize<'a>, + ")] + 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 +117,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: Vec, /// 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: Vec, /// Offsets used to provide indexes from values to updates. /// /// This list has a special representation that any empty range indicates the singleton @@ -142,17 +132,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: Vec, /// Concatenated ordered lists of update diffs, bracketed by offsets in `vals_offs`. - pub diffs: L::DiffContainer, + pub diffs: Vec, } - impl RhhValStorage + impl RhhValStorage where - layout::Key: Default + HashOrdered, - for<'a> layout::KeyRef<'a, L>: HashOrdered, + U::Key: Default + HashOrdered, + for<'a> &'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,33 +174,26 @@ 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) { - let desired = self.desired_location(&key); + fn insert_key(&mut self, key: &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_offs.push_ref(current_offset); + self.keys.push(U::Key::default()); + self.keys_offs.push(current_offset); } // Now we insert the key. Even if it is no longer the desired location because of contention. // If an offset has been supplied we insert it, and otherwise leave it for future determination. - self.keys.push_ref(key); + self.keys.push(key.clone()); if let Some(offset) = offset { - self.keys_offs.push_ref(offset); + self.keys_offs.push(offset); } self.key_count += 1; } - /// 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); - key_con.push_own(&key); - self.insert_key(key_con.index(0), offset) - } - /// Indicates both the desired location and the hash signature of the key. fn desired_location(&self, key: &K) -> usize { if self.divisor == 0 { 0 } @@ -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: &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].lt(key) } /// Indicates that a key is valid, rather than dead space, by looking for a valid offset range. @@ -253,25 +236,21 @@ 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>, + U::Key: Serialize + for<'a> Deserialize<'a>, + U::Val: Serialize + for<'a> Deserialize<'a>, + U::Time: Serialize + for<'a> Deserialize<'a>, + U::Diff: Serialize + for<'a> Deserialize<'a>, ")] - pub struct RhhValBatch + 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 +259,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> &'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 +284,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> &'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 +302,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: Vec::new(), + keys_offs: OffsetList::with_capacity(0), + vals: Vec::new(), + vals_offs: OffsetList::with_capacity(0), + times: Vec::new(), + diffs: Vec::new(), key_count: 0, key_capacity: 0, divisor: 0, @@ -343,35 +319,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> &'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; @@ -388,23 +363,25 @@ mod val_batch { let batch1 = &batch1.storage; let batch2 = &batch2.storage; + let keys_capacity = batch1.keys.len() + batch2.keys.len(); + let vals_capacity = batch1.vals.len() + batch2.vals.len(); + let times_capacity = batch1.times.len() + batch2.times.len(); + let diffs_capacity = batch1.diffs.len() + batch2.diffs.len(); + 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: Vec::with_capacity(keys_capacity), + keys_offs: OffsetList::with_capacity(batch1.keys_offs.len() + batch2.keys_offs.len()), + vals: Vec::with_capacity(vals_capacity), + vals_offs: OffsetList::with_capacity(batch1.vals_offs.len() + batch2.vals_offs.len()), + times: Vec::with_capacity(times_capacity), + diffs: Vec::with_capacity(diffs_capacity), 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; - keys_offs.push_ref(0); - let vals_offs: &mut L::OffsetContainer = &mut storage.vals_offs; - vals_offs.push_ref(0); + storage.keys_offs.push(0); + storage.vals_offs.push(0); RhhValMerger { key_cursor1: 0, @@ -415,14 +392,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 +437,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> &'a U::Key: HashOrdered, { /// Copy the next key in `source`. /// @@ -472,32 +449,32 @@ 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); while lower < upper { self.stash_updates_for_val(source, lower); if let Some(off) = self.consolidate_updates() { - self.result.vals_offs.push_ref(off); - self.result.vals.push_ref(source.vals.index(lower)); + self.result.vals_offs.push(off); + self.result.vals.push(source.vals[lower].clone()); } lower += 1; } // If we have pushed any values, copy the key as well. if self.result.vals.len() > init_vals { - self.result.insert_key(source.keys.index(cursor), Some(self.result.vals.len())); + self.result.insert_key(&source.keys[cursor], Some(self.result.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: &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)) { + match source1.keys[self.key_cursor1].cmp(&source2.keys[self.key_cursor2]) { Ordering::Less => { self.copy_key(source1, self.key_cursor1); self.key_cursor1 += 1; @@ -507,7 +484,7 @@ mod val_batch { let (lower1, upper1) = source1.values_for_key(self.key_cursor1); let (lower2, upper2) = source2.values_for_key(self.key_cursor2); if let Some(off) = self.merge_vals((source1, lower1, upper1), (source2, lower2, upper2)) { - self.result.insert_key(source1.keys.index(self.key_cursor1), Some(off)); + self.result.insert_key(&source1.keys[self.key_cursor1], Some(off)); } // Increment cursors in either case; the keys are merged. self.key_cursor1 += 1; @@ -525,8 +502,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(); @@ -535,13 +512,13 @@ mod val_batch { // 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.index(lower1).cmp(&source2.vals.index(lower2)) { + match source1.vals[lower1].cmp(&source2.vals[lower2]) { Ordering::Less => { // Extend stash by updates, with logical compaction applied. self.stash_updates_for_val(source1, lower1); if let Some(off) = self.consolidate_updates() { - self.result.vals_offs.push_ref(off); - self.result.vals.push_ref(source1.vals.index(lower1)); + self.result.vals_offs.push(off); + self.result.vals.push(source1.vals[lower1].clone()); } lower1 += 1; }, @@ -549,8 +526,8 @@ mod val_batch { self.stash_updates_for_val(source1, lower1); self.stash_updates_for_val(source2, lower2); if let Some(off) = self.consolidate_updates() { - self.result.vals_offs.push_ref(off); - self.result.vals.push_ref(source1.vals.index(lower1)); + self.result.vals_offs.push(off); + self.result.vals.push(source1.vals[lower1].clone()); } lower1 += 1; lower2 += 1; @@ -559,8 +536,8 @@ mod val_batch { // Extend stash by updates, with logical compaction applied. self.stash_updates_for_val(source2, lower2); if let Some(off) = self.consolidate_updates() { - self.result.vals_offs.push_ref(off); - self.result.vals.push_ref(source2.vals.index(lower2)); + self.result.vals_offs.push(off); + self.result.vals.push(source2.vals[lower2].clone()); } lower2 += 1; }, @@ -570,16 +547,16 @@ mod val_batch { while lower1 < upper1 { self.stash_updates_for_val(source1, lower1); if let Some(off) = self.consolidate_updates() { - self.result.vals_offs.push_ref(off); - self.result.vals.push_ref(source1.vals.index(lower1)); + self.result.vals_offs.push(off); + self.result.vals.push(source1.vals[lower1].clone()); } lower1 += 1; } while lower2 < upper2 { self.stash_updates_for_val(source2, lower2); if let Some(off) = self.consolidate_updates() { - self.result.vals_offs.push_ref(off); - self.result.vals.push_ref(source2.vals.index(lower2)); + self.result.vals_offs.push(off); + self.result.vals.push(source2.vals[lower2].clone()); } lower2 += 1; } @@ -593,16 +570,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 time = &source.times[i]; + let diff = &source.diffs[i]; + let mut new_time = time.clone(); 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, diff.clone())); } } @@ -615,8 +592,7 @@ mod val_batch { // we push nothing and report an unincremented offset to encode this case. 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 == *t2 && *d1 == *d2 }); if self.update_stash.len() == 1 && last_eq.unwrap_or(false) { // Just clear out update_stash, as we won't drain it here. @@ -626,8 +602,8 @@ mod val_batch { else { // Conventional; move `update_stash` into `updates`. for (time, diff) in self.update_stash.drain(..) { - self.result.times.push_own(&time); - self.result.diffs.push_own(&diff); + self.result.times.push(time); + self.result.diffs.push(diff); } } Some(self.result.times.len()) @@ -645,49 +621,45 @@ 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> &'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 key<'a>(&self, storage: &'a RhhValBatch) -> &'a U::Key { &storage.storage.keys[self.key_cursor] } + fn val<'a>(&self, storage: &'a RhhValBatch) -> &'a U::Val { &storage.storage.vals[self.val_cursor] } + fn get_key<'a>(&self, storage: &'a RhhValBatch) -> Option<&'a U::Key> { storage.storage.keys.get(self.key_cursor) } + fn get_val<'a>(&self, storage: &'a RhhValBatch) -> Option<&'a U::Val> { if self.val_valid(storage) { storage.storage.vals.get(self.val_cursor) } else { None } } + fn map_times(&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); - let diff = storage.storage.diffs.index(index); + let time = &storage.storage.times[index]; + let diff = &storage.storage.diffs[index]; 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,9 +671,8 @@ mod val_batch { self.key_cursor = storage.storage.keys.len(); } } - fn seek_key(&mut self, storage: &RhhValBatch, key: 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); + fn seek_key(&mut self, storage: &RhhValBatch, key: &U::Key) { + let desired = storage.storage.desired_location(key); // Advance the cursor, if `desired` is ahead of it. if self.key_cursor < desired { self.key_cursor = desired; @@ -719,16 +690,19 @@ 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: &U::Val) { + let (_, upper) = storage.storage.values_for_key(self.key_cursor); + while self.val_cursor < upper && storage.storage.vals[self.val_cursor].lt(val) { + self.val_cursor += 1; + } } - 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 +710,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,41 +744,38 @@ 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| *t == time).unwrap_or(false) && self.result.diffs.last().map(|d| *d == diff).unwrap_or(false) { assert!(self.singleton.is_none()); self.singleton = Some((time, diff)); } else { // If we have pushed a single element, we need to copy it out to meet this one. if let Some((time, diff)) = self.singleton.take() { - self.result.times.push_own(&time); - self.result.diffs.push_own(&diff); + self.result.times.push(time); + self.result.diffs.push(diff); } - self.result.times.push_own(&time); - self.result.diffs.push_own(&diff); + self.result.times.push(time); + self.result.diffs.push(diff); } } } - 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> &'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,58 +784,56 @@ 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: Vec::with_capacity(keys), + keys_offs: OffsetList::with_capacity(keys + 1), + vals: Vec::with_capacity(vals), + vals_offs: OffsetList::with_capacity(vals + 1), + times: Vec::with_capacity(upds), + diffs: Vec::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| key == *k).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| val == *v).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()); + self.result.vals_offs.push(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(val); } } else { // New key; complete representation of prior key. - self.result.vals_offs.push_ref(self.result.times.len()); + self.result.vals_offs.push(self.result.times.len()); if self.singleton.take().is_some() { self.singletons += 1; } - self.result.keys_offs.push_ref(self.result.vals.len()); + self.result.keys_offs.push(self.result.vals.len()); self.push_update(time, diff); - self.result.vals.push_into(val); + self.result.vals.push(val); // Insert the key, but with no specified offset. - self.result.insert_key_own(&key, None); + self.result.insert_key(&key, None); } } } #[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()); + self.result.vals_offs.push(self.result.times.len()); // Remove any pending singleton, and if it was set increment our count. if self.singleton.take().is_some() { self.singletons += 1; } - self.result.keys_offs.push_ref(self.result.vals.len()); + self.result.keys_offs.push(self.result.vals.len()); RhhValBatch { updates: self.result.times.len() + self.singletons, storage: self.result, @@ -874,7 +842,14 @@ 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 chunk in chain.iter() { + keys += chunk.len(); + vals += chunk.len(); + upds += chunk.len(); + } 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..51a29fc11 100644 --- a/differential-dataflow/src/trace/mod.rs +++ b/differential-dataflow/src/trace/mod.rs @@ -19,8 +19,6 @@ use crate::logging::Logger; pub use self::cursor::Cursor; pub use self::description::Description; -use crate::trace::implementations::LayoutExt; - /// 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>; @@ -44,52 +42,29 @@ pub type ExertionLogic = std::sync::Arc Fn(&'a [(usize, usize, usize /// 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 key type. + type Key: crate::Data; + /// The value type. + type Val: crate::Data; + /// The time type. + type Time: crate::lattice::Lattice + timely::progress::Timestamp + crate::Data; + /// The diff type. + type Diff: crate::difference::Semigroup + crate::Data; /// 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, - 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, - >; - + BatchReader; /// 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, - 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, - >; - + Cursor; /// Provides a cursor over updates contained in the trace. fn cursor(&mut self) -> (Self::Cursor, Self::Storage) { @@ -237,36 +212,28 @@ pub trait Trace : TraceReader { 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 key type. + type Key: crate::Data; + /// The value type. + type Val: crate::Data; + /// The time type. + type Time: crate::lattice::Lattice + timely::progress::Timestamp + crate::Data; + /// The diff type. + type Diff: crate::difference::Semigroup + crate::Data; /// 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, - 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, - >; - - /// Acquires a cursor to the batch's contents. + Cursor; + + /// Acquires a cursor to the batch's contents. fn cursor(&self) -> Self::Cursor; /// The number of updates in the batch. fn len(&self) -> usize; @@ -377,12 +344,13 @@ 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 { + type Key = B::Key; + type Val = B::Val; + type Time = B::Time; + type Diff = B::Diff; + /// The type used to enumerate the batch's contents. type Cursor = RcBatchCursor; /// Acquires a cursor to the batch's contents. @@ -401,11 +369,6 @@ 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 { @@ -416,27 +379,32 @@ pub mod rc_blanket_impls { 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) -> &'a Self::Key { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> &'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<&'a Self::Key> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { self.cursor.get_val(storage) } #[inline] - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, logic: L) { + fn map_times(&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: &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: &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..627311394 100644 --- a/differential-dataflow/src/trace/wrappers/enter.rs +++ b/differential-dataflow/src/trace/wrappers/enter.rs @@ -25,25 +25,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: Refines+Lattice+columnar::Columnar, { + 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; @@ -96,7 +87,7 @@ where impl TraceEnter where Tr: TraceReader, - TInner: Refines+Lattice, + TInner: Refines+Lattice+columnar::Columnar, { /// Makes a new trace wrapper pub fn make_from(trace: Tr) -> Self { @@ -116,25 +107,16 @@ pub struct BatchEnter { description: Description, } -impl WithLayout for BatchEnter -where - B: BatchReader, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl BatchReader for BatchEnter where B: BatchReader, - TInner: Refines+Lattice, + TInner: Refines+Lattice+columnar::Columnar, { + type Key = B::Key; + type Val = B::Val; + type Time = TInner; + type Diff = B::Diff; + type Cursor = BatchCursorEnter; fn cursor(&self) -> Self::Cursor { @@ -147,7 +129,7 @@ where impl BatchEnter where B: BatchReader, - TInner: Refines+Lattice, + TInner: Refines+Lattice+columnar::Columnar, { /// Makes a new batch wrapper pub fn make_from(batch: B) -> Self { @@ -168,21 +150,6 @@ pub struct CursorEnter { cursor: C, } -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 { fn new(cursor: C) -> Self { CursorEnter { @@ -195,31 +162,37 @@ impl CursorEnter { impl Cursor for CursorEnter where C: Cursor, - TInner: Refines+Lattice, + TInner: Refines+Lattice+columnar::Columnar, { + 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) -> &'a Self::Key { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> &'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<&'a Self::Key> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { self.cursor.get_val(storage) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times(&mut self, storage: &Self::Storage, mut logic: L) { self.cursor.map_times(storage, |time, diff| { - logic(&TInner::to_inner(C::owned_time(time)), diff) + let inner_time = TInner::to_inner(time.clone()); + logic(&inner_time, diff) }) } #[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: &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: &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) } @@ -242,47 +215,39 @@ impl BatchCursorEnter { } } -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: Refines+Lattice+columnar::Columnar, { + 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) -> &'a Self::Key { self.cursor.key(&storage.batch) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> &'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<&'a Self::Key> { self.cursor.get_key(&storage.batch) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { self.cursor.get_val(&storage.batch) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times(&mut self, storage: &Self::Storage, mut logic: L) { self.cursor.map_times(&storage.batch, |time, diff| { - logic(&TInner::to_inner(C::owned_time(time)), diff) + let inner_time = TInner::to_inner(time.clone()); + logic(&inner_time, diff) }) } #[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: &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: &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..09a452390 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: Refines+Lattice+columnar::Columnar, F: 'static, - F: FnMut(Tr::Key<'_>, Tr::Val<'_>, Tr::TimeGat<'_>)->TInner+Clone, + F: FnMut(&Tr::Key, &Tr::Val, &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; @@ -117,7 +106,7 @@ where impl TraceEnter where Tr: TraceReader, - TInner: Refines+Lattice, + TInner: Refines+Lattice+columnar::Columnar, { /// Makes a new trace wrapper pub fn make_from(trace: Tr, logic: F, prior: G) -> Self { @@ -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: Refines+Lattice+columnar::Columnar, + F: FnMut(&B::Key, &B::Val, &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 { @@ -173,7 +152,7 @@ where impl BatchEnter where B: BatchReader, - TInner: Refines+Lattice, + TInner: Refines+Lattice+columnar::Columnar, { /// Makes a new batch wrapper pub fn make_from(batch: B, logic: F) -> Self { @@ -196,21 +175,6 @@ pub struct CursorEnter { logic: F, } -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 { fn new(cursor: C, logic: F) -> Self { CursorEnter { @@ -224,22 +188,27 @@ impl CursorEnter { impl Cursor for CursorEnter where C: Cursor, - TInner: Refines+Lattice, - F: FnMut(C::Key<'_>, C::Val<'_>, C::TimeGat<'_>)->TInner, + TInner: Refines+Lattice+columnar::Columnar, + F: FnMut(&C::Key, &C::Val, &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) -> &'a Self::Key { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> &'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<&'a Self::Key> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { self.cursor.get_val(storage) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times(&mut self, storage: &Self::Storage, mut logic: L) { let key = self.key(storage); let val = self.val(storage); let logic2 = &mut self.logic; @@ -249,10 +218,10 @@ where } #[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: &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: &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) } @@ -267,20 +236,6 @@ pub struct BatchCursorEnter { logic: F, } -impl WithLayout for BatchCursorEnter -where - C: Cursor, - TInner: Refines+Lattice, -{ - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl BatchCursorEnter { fn new(cursor: C, logic: F) -> Self { BatchCursorEnter { @@ -293,22 +248,27 @@ impl BatchCursorEnter { impl Cursor for BatchCursorEnter where - TInner: Refines+Lattice, - F: FnMut(C::Key<'_>, C::Val<'_>, C::TimeGat<'_>)->TInner, + TInner: Refines+Lattice+columnar::Columnar, + F: FnMut(&C::Key, &C::Val, &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) -> &'a Self::Key { self.cursor.key(&storage.batch) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> &'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<&'a Self::Key> { self.cursor.get_key(&storage.batch) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { self.cursor.get_val(&storage.batch) } #[inline] - fn map_times)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times(&mut self, storage: &Self::Storage, mut logic: L) { let key = self.key(storage); let val = self.val(storage); let logic2 = &mut self.logic; @@ -318,10 +278,10 @@ where } #[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: &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: &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..ed5cbda9c 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 { @@ -125,17 +115,6 @@ pub struct CursorFrontier { until: Antichain } -use crate::trace::implementations::{Layout, WithLayout}; -impl WithLayout for CursorFrontier { - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl CursorFrontier { fn new(cursor: C, since: AntichainRef, until: AntichainRef) -> Self { CursorFrontier { @@ -148,24 +127,29 @@ impl CursorFrontier { 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) -> &'a Self::Key { self.cursor.key(storage) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> &'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<&'a Self::Key> { self.cursor.get_key(storage) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { self.cursor.get_val(storage) } #[inline] - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times(&mut self, storage: &Self::Storage, mut logic: L) { let since = self.since.borrow(); let until = self.until.borrow(); let mut temp: C::Time = ::minimum(); self.cursor.map_times(storage, |time, diff| { - C::clone_time_onto(time, &mut temp); + temp.clone_from(time); temp.advance_by(since); if !until.less_equal(&temp) { logic(&temp, diff); @@ -174,10 +158,10 @@ impl Cursor for CursorFrontier { } #[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: &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: &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) } @@ -192,16 +176,6 @@ pub struct BatchCursorFrontier { until: Antichain, } -impl WithLayout for BatchCursorFrontier { - type Layout = ( - ::KeyContainer, - ::ValContainer, - Vec, - ::DiffContainer, - ::OffsetContainer, - ); -} - impl BatchCursorFrontier { fn new(cursor: C, since: AntichainRef, until: AntichainRef) -> Self { BatchCursorFrontier { @@ -214,24 +188,29 @@ impl BatchCursorFrontier { 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) -> &'a Self::Key { self.cursor.key(&storage.batch) } + #[inline] fn val<'a>(&self, storage: &'a Self::Storage) -> &'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<&'a Self::Key> { self.cursor.get_key(&storage.batch) } + #[inline] fn get_val<'a>(&self, storage: &'a Self::Storage) -> Option<&'a Self::Val> { self.cursor.get_val(&storage.batch) } #[inline] - fn map_times, Self::DiffGat<'_>)>(&mut self, storage: &Self::Storage, mut logic: L) { + fn map_times(&mut self, storage: &Self::Storage, mut logic: L) { let since = self.since.borrow(); let until = self.until.borrow(); let mut temp: C::Time = ::minimum(); self.cursor.map_times(&storage.batch, |time, diff| { - C::clone_time_onto(time, &mut temp); + temp.clone_from(time); temp.advance_by(since); if !until.less_equal(&temp) { logic(&temp, diff); @@ -240,10 +219,10 @@ impl> Cursor for BatchCursorFrontier { } #[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: &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: &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/differential-dataflow/tests/bfs.rs b/differential-dataflow/tests/bfs.rs index fd6aea26c..f6dcdfabe 100644 --- a/differential-dataflow/tests/bfs.rs +++ b/differential-dataflow/tests/bfs.rs @@ -204,7 +204,7 @@ fn bfs_differential( // returns pairs (n, s) indicating node n can be reached from a root in s steps. fn bfs(edges: VecCollection, roots: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { // initialize roots as reaching themselves at distance 0 let nodes = roots.map(|x| (x, 0)); diff --git a/differential-dataflow/tests/scc.rs b/differential-dataflow/tests/scc.rs index 52eecdf1d..7f4663d88 100644 --- a/differential-dataflow/tests/scc.rs +++ b/differential-dataflow/tests/scc.rs @@ -217,7 +217,7 @@ fn scc_differential( fn _strongly_connected(graph: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { graph.clone().iterate(|scope, inner| { let edges = graph.enter(&scope); @@ -228,7 +228,7 @@ where fn _trim_edges(cycle: VecCollection, edges: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { let nodes = edges.clone() .map_in_place(|x| x.0 = x.1) @@ -246,7 +246,7 @@ where fn _reachability(edges: VecCollection, nodes: VecCollection) -> VecCollection where - G: Scope, + G: Scope, { edges.clone() // <-- wth is this. .filter(|_| false) diff --git a/dogsdogsdogs/Cargo.toml b/dogsdogsdogs/Cargo.toml index 2db38d3e1..4f875fc17 100644 --- a/dogsdogsdogs/Cargo.toml +++ b/dogsdogsdogs/Cargo.toml @@ -18,6 +18,7 @@ workspace = true timely = { workspace = true } differential-dataflow = { workspace = true } serde = { version = "1.0", features = ["derive"]} +columnar = { workspace = true } [dev-dependencies] graph_map = "0.1" diff --git a/dogsdogsdogs/examples/ngo.rs b/dogsdogsdogs/examples/ngo.rs index 340b6c988..dcddde3d5 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); diff --git a/dogsdogsdogs/src/altneu.rs b/dogsdogsdogs/src/altneu.rs index 95d69cbc0..f2fe52560 100644 --- a/dogsdogsdogs/src/altneu.rs +++ b/dogsdogsdogs/src/altneu.rs @@ -12,9 +12,10 @@ //! the join. use serde::{Deserialize, Serialize}; +use columnar::Columnar; /// 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)] pub struct AltNeu { pub time: T, pub neu: bool, // alt < neu in timestamp comparisons. diff --git a/dogsdogsdogs/src/operators/count.rs b/dogsdogsdogs/src/operators/count.rs index 0cd363a76..1027fd9c7 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<&'a Tr::Diff>, + K: Hashable + Ord + Clone + Default + '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..b601a6484 100644 --- a/dogsdogsdogs/src/operators/half_join.rs +++ b/dogsdogsdogs/src/operators/half_join.rs @@ -49,7 +49,6 @@ 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; /// A binary equijoin that responds to updates on only its first input. /// @@ -85,14 +84,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(&Tr::Time, &G::Timestamp) -> bool + 'static, DOut: Clone+'static, - S: FnMut(&K, &V, Tr::Val<'_>)->DOut+'static, + S: FnMut(&K, &V, &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: &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 +150,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(&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, &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 +311,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(&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, &Tr::Val, &G::Timestamp, &R, &mut Vec<(G::Timestamp, Tr::Diff)>) + 'static, CB: ContainerBuilder, - K: Ord, + K: Ord + Clone + 'static, V: Ord, R: Monoid, { @@ -327,10 +326,10 @@ 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 = Vec::::with_capacity(1); + let mut time_con = Vec::::with_capacity(1); for time in frontier.iter() { - time_con.push_own(time); + time_con.push(time.clone()); } // Process proposals one at a time, stopping if we should yield. @@ -338,16 +337,16 @@ where // Use TOTAL ORDER to allow the release of `time`. yielded = yielded || yield_function(timer, *work); - if !yielded && !(0 .. time_con.len()).any(|i| comparison(time_con.index(i), initial)) { - key_con.clear(); key_con.push_own(&key); - cursor.seek_key(&storage, key_con.index(0)); + if !yielded && !(0 .. time_con.len()).any(|i| comparison(&time_con[i], initial)) { + key_con.clear(); key_con.push(key.clone()); + cursor.seek_key(&storage, &key_con[0]); if cursor.get_key(&storage) == key_con.get(0) { 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 = t.clone(); t.join_assign(time); - output_buffer.push((t, Tr::owned_diff(d))) + output_buffer.push((t, d.clone())) } }); consolidate(&mut output_buffer); diff --git a/dogsdogsdogs/src/operators/lookup_map.rs b/dogsdogsdogs/src/operators/lookup_map.rs index 7908ce858..29e410642 100644 --- a/dogsdogsdogs/src/operators/lookup_map.rs +++ b/dogsdogsdogs/src/operators/lookup_map.rs @@ -10,7 +10,6 @@ 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; /// Proposes extensions to a stream of prefixes. /// @@ -28,10 +27,11 @@ pub fn lookup_map( ) -> VecCollection where G: Scope, + K: Clone, Tr: for<'a> TraceReader< - KeyOwn = K, + Key = K, Time: std::hash::Hash, - Diff : Semigroup>+Monoid+ExchangeData, + Diff : Semigroup<&'a Tr::Diff>+Monoid+ExchangeData, >+Clone+'static, K: Hashable + Ord + 'static, F: FnMut(&D, &mut K)+Clone+'static, @@ -39,7 +39,7 @@ where R: ExchangeData+Monoid, DOut: Clone+'static, ROut: Monoid + 'static, - S: FnMut(&D, &R, Tr::Val<'_>, &Tr::Diff)->(DOut, ROut)+'static, + S: FnMut(&D, &R, &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 +91,17 @@ 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 = Vec::::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)) { + key_con.clear(); key_con.push(key1.clone()); + cursor.seek_key(&storage, &key_con[1]); + if cursor.get_key(&storage) == Some(&key_con[1]) { 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 t.clone().less_equal(time) { 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..d85793af9 100644 --- a/dogsdogsdogs/src/operators/propose.rs +++ b/dogsdogsdogs/src/operators/propose.rs @@ -21,12 +21,12 @@ pub fn propose( where G: Scope, Tr: for<'a> TraceReader< - KeyOwn = K, - ValOwn = V, + Key = K, + Val = V, Time: std::hash::Hash, - Diff: Monoid+Multiply+ExchangeData+Semigroup>, + Diff: Monoid+Multiply+ExchangeData+Semigroup<&'a Tr::Diff>, >+Clone+'static, - K: Hashable + Default + Ord + 'static, + K: Hashable + Default + Ord + Clone + 'static, F: Fn(&P)->K+Clone+'static, P: ExchangeData, V: Clone + 'static, @@ -35,7 +35,7 @@ where 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(), value.clone()), diff.clone().multiply(sum)), Default::default(), Default::default(), Default::default(), @@ -55,12 +55,12 @@ pub fn propose_distinct( where G: Scope, Tr: for<'a> TraceReader< - KeyOwn = K, - ValOwn = V, + Key = K, + Val = V, Time: std::hash::Hash, - Diff : Semigroup>+Monoid+Multiply+ExchangeData, + Diff : Semigroup<&'a Tr::Diff>+Monoid+Multiply+ExchangeData, >+Clone+'static, - K: Hashable + Default + Ord + 'static, + K: Hashable + Default + Ord + Clone + 'static, F: Fn(&P)->K+Clone+'static, P: ExchangeData, V: Clone + 'static, @@ -69,7 +69,7 @@ where 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(), value.clone()), diff.clone()), Default::default(), Default::default(), Default::default(), diff --git a/dogsdogsdogs/src/operators/validate.rs b/dogsdogsdogs/src/operators/validate.rs index f610d55be..681b04b5b 100644 --- a/dogsdogsdogs/src/operators/validate.rs +++ b/dogsdogsdogs/src/operators/validate.rs @@ -20,9 +20,9 @@ pub fn validate( where G: Scope, Tr: for<'a> TraceReader< - KeyOwn = (K, V), + Key = (K, V), Time: std::hash::Hash, - Diff : Semigroup>+Monoid+Multiply+ExchangeData, + Diff : Semigroup<&'a Tr::Diff>+Monoid+Multiply+ExchangeData, >+Clone+'static, K: Ord+Hash+Clone+Default + 'static, V: ExchangeData+Hash+Default, diff --git a/experiments/Cargo.toml b/experiments/Cargo.toml index edcf55fb3..3155d6906 100644 --- a/experiments/Cargo.toml +++ b/experiments/Cargo.toml @@ -16,3 +16,4 @@ timely = { workspace = true } differential-dataflow = { workspace = true } graph_map = { git = "https://github.com/frankmcsherry/graph-map" } serde = { version = "1.0", features = ["derive"] } +columnar = { workspace = true } diff --git a/experiments/src/bin/deals-interactive.rs b/experiments/src/bin/deals-interactive.rs index 3ea53cd14..2e074dfcb 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+columnar::Columnar{ // descendants of tc_1: let tc_1_enter = tc_1.clone(); diff --git a/experiments/src/bin/graphs-interactive-alt.rs b/experiments/src/bin/graphs-interactive-alt.rs index 6e5b0ffaa..4c79b123d 100644 --- a/experiments/src/bin/graphs-interactive-alt.rs +++ b/experiments/src/bin/graphs-interactive-alt.rs @@ -265,7 +265,7 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+columnar::Columnar { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -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+columnar::Columnar { goals.scope().iterative::(|inner| { @@ -362,7 +362,7 @@ 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 + columnar::Columnar { // each edge (x,y) means that we need at least a label for the min of x and y. let nodes = diff --git a/experiments/src/bin/graphs-interactive-neu-zwei.rs b/experiments/src/bin/graphs-interactive-neu-zwei.rs index 35a118cd9..fd49da170 100644 --- a/experiments/src/bin/graphs-interactive-neu-zwei.rs +++ b/experiments/src/bin/graphs-interactive-neu-zwei.rs @@ -233,7 +233,7 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+columnar::Columnar { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); diff --git a/experiments/src/bin/graphs-interactive-neu.rs b/experiments/src/bin/graphs-interactive-neu.rs index 7e27e7ed8..c39e92bf9 100644 --- a/experiments/src/bin/graphs-interactive-neu.rs +++ b/experiments/src/bin/graphs-interactive-neu.rs @@ -298,7 +298,7 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+columnar::Columnar { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -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+columnar::Columnar { goals.scope().iterative::(|inner| { diff --git a/experiments/src/bin/graphs-interactive.rs b/experiments/src/bin/graphs-interactive.rs index 2ec74e93f..c56a83be9 100644 --- a/experiments/src/bin/graphs-interactive.rs +++ b/experiments/src/bin/graphs-interactive.rs @@ -202,7 +202,7 @@ fn three_hop( forward_graph: Arrange, reverse_graph: Arrange, goals: VecCollection) -> VecCollection -where G::Timestamp: Lattice+Ord { +where G::Timestamp: Lattice+Ord+columnar::Columnar { let sources = goals.clone().map(|(x,_)| x); let targets = goals.map(|(_,y)| y); @@ -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+columnar::Columnar { goals.scope().iterative::(|inner| { diff --git a/experiments/src/bin/multitemporal.rs b/experiments/src/bin/multitemporal.rs index eb602efe2..21ff29a7b 100644 --- a/experiments/src/bin/multitemporal.rs +++ b/experiments/src/bin/multitemporal.rs @@ -170,8 +170,10 @@ fn main() { /// type and its implementations, you can use it as a timestamp type. mod pair { + use columnar::Columnar; + /// 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)] pub struct Pair { pub first: S, pub second: T, diff --git a/interactive/Cargo.toml b/interactive/Cargo.toml index 6a03116ac..f556b0ea5 100644 --- a/interactive/Cargo.toml +++ b/interactive/Cargo.toml @@ -15,3 +15,4 @@ serde = { version = "1", features = ["derive"]} differential-dataflow = { workspace = true } differential-dogs3 = { path = "../dogsdogsdogs" } timely = { workspace = true } +columnar = { workspace = true } diff --git a/interactive/src/concrete.rs b/interactive/src/concrete.rs index 5d3c8e9ed..65e60be02 100644 --- a/interactive/src/concrete.rs +++ b/interactive/src/concrete.rs @@ -102,6 +102,83 @@ impl VectorFrom for Value { } } +/// Manual `Columnar` implementation for `Value`. +/// +/// Because `Value` is recursive (the `Vector` variant contains `Vec`), +/// the derive macro overflows. We store each `Value` as bincode-serialized bytes, +/// reusing the columnar representation of `Vec>` (`Vecs>`). +mod value_columnar { + use super::Value; + + /// Columnar container for `Value`, wrapping `Vecs>`. + /// Each `Value` is bincode-serialized to bytes on push. + #[derive(Clone, Default)] + pub struct ValueColumns(columnar::Vecs>); + + // Delegate Borrow, using the same Ref and Borrowed types as Vecs>. + impl columnar::Borrow for ValueColumns { + // Ref is a slice of bytes: Slice<&'a [u8]> + type Ref<'a> = columnar::Slice<&'a [u8]>; + type Borrowed<'a> = columnar::Vecs<&'a [u8], &'a [u64]>; + fn borrow<'a>(&'a self) -> Self::Borrowed<'a> { + columnar::Borrow::borrow(&self.0) + } + fn reborrow<'b, 'a: 'b>(item: Self::Borrowed<'a>) -> Self::Borrowed<'b> { + > as columnar::Borrow>::reborrow(item) + } + fn reborrow_ref<'b, 'a: 'b>(item: Self::Ref<'a>) -> Self::Ref<'b> { + > as columnar::Borrow>::reborrow_ref(item) + } + } + + impl columnar::Len for ValueColumns { + fn len(&self) -> usize { columnar::Len::len(&self.0) } + } + impl columnar::Clear for ValueColumns { + fn clear(&mut self) { columnar::Clear::clear(&mut self.0); } + } + impl columnar::HeapSize for ValueColumns { + fn heap_size(&self) -> (usize, usize) { columnar::HeapSize::heap_size(&self.0) } + } + + // Push &Value by serializing to bytes and pushing as Vec. + impl<'a> columnar::Push<&'a Value> for ValueColumns { + fn push(&mut self, item: &'a Value) { + let bytes = bincode::serialize(item).expect("bincode serialization of Value failed"); + columnar::Push::push(&mut self.0, &bytes); + } + } + + // Push the Ref type (Slice<&[u8]>) by re-encoding as bytes for the inner Vecs. + impl<'a> columnar::Push> for ValueColumns { + fn push(&mut self, item: columnar::Slice<&'a [u8]>) { + columnar::Push::push(&mut self.0, item); + } + } + + impl columnar::Container for ValueColumns { + fn extend_from_self(&mut self, other: Self::Borrowed<'_>, range: std::ops::Range) { + columnar::Container::extend_from_self(&mut self.0, other, range); + } + fn reserve_for<'a, I: Iterator>>(&mut self, items: I) where I: Clone { + columnar::Container::reserve_for(&mut self.0, items); + } + } + + impl columnar::Columnar for Value { + fn into_owned<'a>(other: columnar::Ref<'a, Self>) -> Self { + // Ref is Slice<&'a [u8]>; access the underlying byte slice. + let bytes: &[u8] = &other.slice[other.lower..other.upper]; + bincode::deserialize(bytes).expect("bincode deserialization of Value failed") + } + fn copy_from<'a>(&mut self, other: columnar::Ref<'a, Self>) { + let bytes: &[u8] = &other.slice[other.lower..other.upper]; + *self = bincode::deserialize(bytes).expect("bincode deserialization of Value failed"); + } + type Container = ValueColumns; + } +} + use differential_dataflow::logging::DifferentialEvent; impl VectorFrom for Value { diff --git a/mdbook/Cargo.toml b/mdbook/Cargo.toml index 950082787..53034c0fa 100644 --- a/mdbook/Cargo.toml +++ b/mdbook/Cargo.toml @@ -9,5 +9,6 @@ publish = false workspace = true [dependencies] +columnar = { workspace = true } differential-dataflow = { path = "../differential-dataflow" } timely.workspace = true diff --git a/mdbook/src/chapter_2/chapter_2_1.md b/mdbook/src/chapter_2/chapter_2_1.md index ecc749447..a19c48524 100644 --- a/mdbook/src/chapter_2/chapter_2_1.md +++ b/mdbook/src/chapter_2/chapter_2_1.md @@ -7,11 +7,12 @@ As an example, our example program used `map` to reverse the pairs of identifier ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .clone() @@ -26,11 +27,12 @@ If instead we had just written ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .map(|(m2, m1)| m2); diff --git a/mdbook/src/chapter_2/chapter_2_2.md b/mdbook/src/chapter_2/chapter_2_2.md index b30d779ef..1d0709134 100644 --- a/mdbook/src/chapter_2/chapter_2_2.md +++ b/mdbook/src/chapter_2/chapter_2_2.md @@ -7,11 +7,12 @@ As an example, we might select out those management relation where the manager h ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .filter(|&(m2, m1)| m2 > m1); diff --git a/mdbook/src/chapter_2/chapter_2_3.md b/mdbook/src/chapter_2/chapter_2_3.md index b61a02a57..1b0c7687b 100644 --- a/mdbook/src/chapter_2/chapter_2_3.md +++ b/mdbook/src/chapter_2/chapter_2_3.md @@ -7,11 +7,12 @@ For example, we might form the symmetric "management relation" by concatenating ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .clone() diff --git a/mdbook/src/chapter_2/chapter_2_4.md b/mdbook/src/chapter_2/chapter_2_4.md index 2c2ad4f4b..8a28c4603 100644 --- a/mdbook/src/chapter_2/chapter_2_4.md +++ b/mdbook/src/chapter_2/chapter_2_4.md @@ -9,11 +9,12 @@ As an example, if we were to inspect ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .clone() @@ -35,11 +36,12 @@ However, by introducing `consolidate` ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .clone() diff --git a/mdbook/src/chapter_2/chapter_2_5.md b/mdbook/src/chapter_2/chapter_2_5.md index 8de590eb2..da8b73ba2 100644 --- a/mdbook/src/chapter_2/chapter_2_5.md +++ b/mdbook/src/chapter_2/chapter_2_5.md @@ -7,11 +7,12 @@ Our example from earlier uses a join to match up pairs `(m2, m1)` and `(m1, p)` ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .clone() diff --git a/mdbook/src/chapter_2/chapter_2_6.md b/mdbook/src/chapter_2/chapter_2_6.md index 74b5020f4..85919cb53 100644 --- a/mdbook/src/chapter_2/chapter_2_6.md +++ b/mdbook/src/chapter_2/chapter_2_6.md @@ -7,11 +7,12 @@ For example, to produce for each manager their managee with the lowest identifie ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages .reduce(|_key, input, output| { diff --git a/mdbook/src/chapter_2/chapter_2_7.md b/mdbook/src/chapter_2/chapter_2_7.md index def1c4dcd..6c29e0405 100644 --- a/mdbook/src/chapter_2/chapter_2_7.md +++ b/mdbook/src/chapter_2/chapter_2_7.md @@ -7,12 +7,13 @@ As an example, we can take our `manages` relation and determine for all employee ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::operators::Iterate; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages // transitive contains (manager, person) for many hops. .iterate(|_scope, transitive| { @@ -44,12 +45,13 @@ In the example above, we could rewrite ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use differential_dataflow::VecCollection; # use differential_dataflow::operators::{Iterate, iterate::VecVariable}; # use differential_dataflow::lattice::Lattice; # fn example(manages: VecCollection) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { manages // transitive contains (manager, person) for many hops. .clone() @@ -84,6 +86,7 @@ As an example, the implementation of the `iterate` operator looks something like ```rust # extern crate timely; # extern crate differential_dataflow; +# extern crate columnar; # use timely::dataflow::Scope; # use timely::dataflow::scopes::Child; # use timely::progress::Antichain; @@ -91,12 +94,12 @@ As an example, the implementation of the `iterate` operator looks something like # use differential_dataflow::operators::{Iterate, iterate::VecVariable}; # use differential_dataflow::lattice::Lattice; # fn logic<'a, G: Scope>(collection: VecCollection, (u64, u64), isize>) -> VecCollection, (u64, u64)> -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { # collection # } # fn example<'a, G: Scope>(collection: VecCollection) //, logic: impl Fn(&VecVariable, (u64, u64), isize>) -> VecCollection, (u64, u64)>) -# where G::Timestamp: Lattice +# where G::Timestamp: Lattice + columnar::Columnar # { collection.scope().scoped("inner", |subgraph| { let (variable, collection) = VecVariable::new_from(collection.enter(subgraph), 1);